From f302c79b9e6491d0f5ed397fdccce42b07d1da5b Mon Sep 17 00:00:00 2001
From: David Li
Date: Mon, 24 Jun 2019 11:13:09 -0400
Subject: [PATCH] Rework interface for accessing server-sent metadata during
DoPut
---
.../apache/arrow/flight/AsyncPutListener.java | 63 +++++++
.../org/apache/arrow/flight/FlightClient.java | 51 +++--
.../apache/arrow/flight/SyncPutListener.java | 114 ++++++++++++
.../integration/IntegrationTestClient.java | 14 +-
.../arrow/flight/TestApplicationMetadata.java | 174 ++++++++++++------
.../arrow/flight/TestBasicOperation.java | 3 +-
.../apache/arrow/flight/TestLargeMessage.java | 2 +-
.../java/org/apache/arrow/flight/TestTls.java | 4 +-
.../flight/example/TestExampleServer.java | 4 +-
9 files changed, 338 insertions(+), 91 deletions(-)
create mode 100644 java/flight/src/main/java/org/apache/arrow/flight/AsyncPutListener.java
create mode 100644 java/flight/src/main/java/org/apache/arrow/flight/SyncPutListener.java
diff --git a/java/flight/src/main/java/org/apache/arrow/flight/AsyncPutListener.java b/java/flight/src/main/java/org/apache/arrow/flight/AsyncPutListener.java
new file mode 100644
index 0000000000000..c8214e3195310
--- /dev/null
+++ b/java/flight/src/main/java/org/apache/arrow/flight/AsyncPutListener.java
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.arrow.flight;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+
+/**
+ * A handler for server-sent application metadata messages during a Flight DoPut operation.
+ *
+ *
To handle messages, create an instance of this class overriding {@link #onNext(PutResult)}. The other methods
+ * should not be overridden.
+ */
+public class AsyncPutListener implements FlightClient.PutListener {
+
+ private CompletableFuture completed;
+
+ public AsyncPutListener() {
+ completed = new CompletableFuture<>();
+ }
+
+ /**
+ * Wait for the stream to finish on the server side. You must call this to be notified of any errors that may have
+ * happened during the upload.
+ */
+ @Override
+ public final void getResult() {
+ try {
+ completed.get();
+ } catch (InterruptedException | ExecutionException e) {
+ throw new RuntimeException(e);
+ }
+ }
+
+ @Override
+ public void onNext(PutResult val) {
+ }
+
+ @Override
+ public final void onError(Throwable t) {
+ completed.completeExceptionally(t);
+ }
+
+ @Override
+ public final void onCompleted() {
+ completed.complete(null);
+ }
+}
diff --git a/java/flight/src/main/java/org/apache/arrow/flight/FlightClient.java b/java/flight/src/main/java/org/apache/arrow/flight/FlightClient.java
index 673e4d78ef010..9ac3686fea377 100644
--- a/java/flight/src/main/java/org/apache/arrow/flight/FlightClient.java
+++ b/java/flight/src/main/java/org/apache/arrow/flight/FlightClient.java
@@ -20,7 +20,6 @@
import java.io.InputStream;
import java.net.URISyntaxException;
import java.util.Iterator;
-import java.util.concurrent.CompletableFuture;
import java.util.concurrent.TimeUnit;
import java.util.stream.Collectors;
@@ -44,7 +43,6 @@
import org.apache.arrow.vector.ipc.message.ArrowRecordBatch;
import com.google.common.base.Preconditions;
-import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Iterators;
@@ -168,7 +166,7 @@ public void authenticate(ClientAuthHandler handler, CallOption... options) {
* @return ClientStreamListener an interface to control uploading data
*/
public ClientStreamListener startPut(FlightDescriptor descriptor, VectorSchemaRoot root,
- StreamListener metadataListener, CallOption... options) {
+ PutListener metadataListener, CallOption... options) {
return startPut(descriptor, root, new MapDictionaryProvider(), metadataListener, options);
}
@@ -181,7 +179,7 @@ public ClientStreamListener startPut(FlightDescriptor descriptor, VectorSchemaRo
* @return ClientStreamListener an interface to control uploading data
*/
public ClientStreamListener startPut(FlightDescriptor descriptor, VectorSchemaRoot root, DictionaryProvider provider,
- StreamListener metadataListener, CallOption... options) {
+ PutListener metadataListener, CallOption... options) {
Preconditions.checkNotNull(descriptor);
Preconditions.checkNotNull(root);
@@ -194,7 +192,7 @@ public ClientStreamListener startPut(FlightDescriptor descriptor, VectorSchemaRo
DictionaryUtils.generateSchemaMessages(root.getSchema(), descriptor, provider, observer::onNext);
return new PutObserver(new VectorUnloader(
root, true /* include # of nulls in vectors */, true /* must align buffers to be C++-compatible */),
- observer, resultObserver);
+ observer, metadataListener);
}
/**
@@ -257,7 +255,7 @@ public void onCompleted() {
return stream;
}
- private static class SetStreamObserver extends CompletableFuture implements StreamObserver {
+ private static class SetStreamObserver implements StreamObserver {
private final BufferAllocator allocator;
private final StreamListener listener;
@@ -277,13 +275,11 @@ public void onNext(Flight.PutResult value) {
@Override
public void onError(Throwable t) {
listener.onError(t);
- completeExceptionally(t);
}
@Override
public void onCompleted() {
listener.onCompleted();
- complete(null);
}
}
@@ -291,13 +287,13 @@ private static class PutObserver implements ClientStreamListener {
private final ClientCallStreamObserver observer;
private final VectorUnloader unloader;
- private final CompletableFuture futureResult;
+ private final PutListener listener;
public PutObserver(VectorUnloader unloader, ClientCallStreamObserver observer,
- CompletableFuture futureResult) {
+ PutListener listener) {
this.observer = observer;
this.unloader = unloader;
- this.futureResult = futureResult;
+ this.listener = listener;
}
@Override
@@ -308,8 +304,7 @@ public void putNext() {
@Override
public void putNext(ArrowBuf appMetadata) {
ArrowRecordBatch batch = unloader.getRecordBatch();
- // Check the futureResult in case server sent an exception
- while (!observer.isReady() && !futureResult.isDone()) {
+ while (!observer.isReady()) {
/* busy wait */
}
// Takes ownership of appMetadata
@@ -328,11 +323,7 @@ public void completed() {
@Override
public void getResult() {
- try {
- futureResult.get();
- } catch (Exception ex) {
- throw Throwables.propagate(ex);
- }
+ listener.getResult();
}
}
@@ -367,6 +358,30 @@ public interface ClientStreamListener {
void getResult();
}
+ /**
+ * A handler for server-sent application metadata messages during a Flight DoPut operation.
+ *
+ *
Generally, instead of implementing this yourself, you should use {@link AsyncPutListener} or {@link
+ * SyncPutListener}.
+ */
+ public interface PutListener extends StreamListener {
+
+ /**
+ * Wait for the stream to finish on the server side. You must call this to be notified of any errors that may have
+ * happened during the upload.
+ */
+ void getResult();
+
+ /**
+ * Called when a message from the server is received.
+ *
+ * @param val The application metadata. This buffer will be reclaimed once onNext returns; you must retain a
+ * reference to use it outside this method.
+ */
+ @Override
+ void onNext(PutResult val);
+ }
+
/**
* Shut down this client.
*/
diff --git a/java/flight/src/main/java/org/apache/arrow/flight/SyncPutListener.java b/java/flight/src/main/java/org/apache/arrow/flight/SyncPutListener.java
new file mode 100644
index 0000000000000..f1246a1d07911
--- /dev/null
+++ b/java/flight/src/main/java/org/apache/arrow/flight/SyncPutListener.java
@@ -0,0 +1,114 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.arrow.flight;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+
+import io.netty.buffer.ArrowBuf;
+
+/**
+ * A listener for server-sent application metadata messages during a Flight DoPut. This class wraps the messages in a
+ * synchronous interface.
+ */
+public final class SyncPutListener implements FlightClient.PutListener, AutoCloseable {
+
+ private final LinkedBlockingQueue