Skip to content

Commit

Permalink
Rework interface for accessing server-sent metadata during DoPut
Browse files Browse the repository at this point in the history
  • Loading branch information
lidavidm committed Jun 24, 2019
1 parent 8e3bc48 commit f302c79
Show file tree
Hide file tree
Showing 9 changed files with 338 additions and 91 deletions.
Original file line number Diff line number Diff line change
@@ -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.
*
* <p>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<Void> 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);
}
}
51 changes: 33 additions & 18 deletions java/flight/src/main/java/org/apache/arrow/flight/FlightClient.java
Original file line number Diff line number Diff line change
Expand Up @@ -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;

Expand All @@ -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;

Expand Down Expand Up @@ -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<PutResult> metadataListener, CallOption... options) {
PutListener metadataListener, CallOption... options) {
return startPut(descriptor, root, new MapDictionaryProvider(), metadataListener, options);
}

Expand All @@ -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<PutResult> metadataListener, CallOption... options) {
PutListener metadataListener, CallOption... options) {
Preconditions.checkNotNull(descriptor);
Preconditions.checkNotNull(root);

Expand All @@ -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);
}

/**
Expand Down Expand Up @@ -257,7 +255,7 @@ public void onCompleted() {
return stream;
}

private static class SetStreamObserver extends CompletableFuture<Void> implements StreamObserver<Flight.PutResult> {
private static class SetStreamObserver implements StreamObserver<Flight.PutResult> {
private final BufferAllocator allocator;
private final StreamListener<PutResult> listener;

Expand All @@ -277,27 +275,25 @@ 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);
}
}

private static class PutObserver implements ClientStreamListener {

private final ClientCallStreamObserver<ArrowMessage> observer;
private final VectorUnloader unloader;
private final CompletableFuture<Void> futureResult;
private final PutListener listener;

public PutObserver(VectorUnloader unloader, ClientCallStreamObserver<ArrowMessage> observer,
CompletableFuture<Void> futureResult) {
PutListener listener) {
this.observer = observer;
this.unloader = unloader;
this.futureResult = futureResult;
this.listener = listener;
}

@Override
Expand All @@ -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
Expand All @@ -328,11 +323,7 @@ public void completed() {

@Override
public void getResult() {
try {
futureResult.get();
} catch (Exception ex) {
throw Throwables.propagate(ex);
}
listener.getResult();
}
}

Expand Down Expand Up @@ -367,6 +358,30 @@ public interface ClientStreamListener {
void getResult();
}

/**
* A handler for server-sent application metadata messages during a Flight DoPut operation.
*
* <p>Generally, instead of implementing this yourself, you should use {@link AsyncPutListener} or {@link
* SyncPutListener}.
*/
public interface PutListener extends StreamListener<PutResult> {

/**
* 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.
*/
Expand Down
114 changes: 114 additions & 0 deletions java/flight/src/main/java/org/apache/arrow/flight/SyncPutListener.java
Original file line number Diff line number Diff line change
@@ -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<Object> queue;
private final CompletableFuture<Void> completed;
private static final Object DONE = new Object();
private static final Object DONE_WITH_EXCEPTION = new Object();

public SyncPutListener() {
queue = new LinkedBlockingQueue<>();
completed = new CompletableFuture<>();
}

private PutResult unwrap(Object queueItem) throws InterruptedException, ExecutionException {
if (queueItem == DONE) {
queue.put(queueItem);
return null;
} else if (queueItem == DONE_WITH_EXCEPTION) {
queue.put(queueItem);
completed.get();
}
return (PutResult) queueItem;
}

/**
* Get the next message from the server, blocking until it is available.
*
* @return The next message, or null if the server is done sending messages. The caller assumes ownership of the
* metadata and must remember to close it.
* @throws InterruptedException if interrupted while waiting.
* @throws ExecutionException if the server sent an error, or if there was an internal error.
*/
public PutResult read() throws InterruptedException, ExecutionException {
return unwrap(queue.take());
}

/**
* Get the next message from the server, blocking for the specified amount of time until it is available.
*
* @return The next message, or null if the server is done sending messages or no message arrived before the timeout.
* The caller assumes ownership of the metadata and must remember to close it.
* @throws InterruptedException if interrupted while waiting.
* @throws ExecutionException if the server sent an error, or if there was an internal error.
*/
public PutResult poll(long timeout, TimeUnit unit) throws InterruptedException, ExecutionException {
return unwrap(queue.poll(timeout, unit));
}

@Override
public void getResult() {
try {
completed.get();
} catch (InterruptedException | ExecutionException e) {
throw new RuntimeException(e);
}
}

@Override
public void onNext(PutResult val) {
final ArrowBuf metadata = val.getApplicationMetadata();
metadata.getReferenceManager().retain();
queue.add(PutResult.metadata(metadata));
}

@Override
public void onError(Throwable t) {
completed.completeExceptionally(t);
queue.add(DONE_WITH_EXCEPTION);
}

@Override
public void onCompleted() {
completed.complete(null);
queue.add(DONE);
}

@Override
public void close() {
queue.forEach(o -> {
if (o instanceof PutResult) {
((PutResult) o).close();
}
});
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -27,11 +27,11 @@
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;

import org.apache.arrow.flight.AsyncPutListener;
import org.apache.arrow.flight.FlightClient;
import org.apache.arrow.flight.FlightDescriptor;
import org.apache.arrow.flight.FlightEndpoint;
import org.apache.arrow.flight.FlightInfo;
import org.apache.arrow.flight.FlightProducer.StreamListener;
import org.apache.arrow.flight.FlightStream;
import org.apache.arrow.flight.Location;
import org.apache.arrow.flight.PutResult;
Expand Down Expand Up @@ -104,7 +104,7 @@ private void run(String[] args) throws ParseException, IOException {
VectorLoader jsonLoader = new VectorLoader(jsonRoot);
final CompletableFuture<Void> completed = new CompletableFuture<>();
FlightClient.ClientStreamListener stream = client.startPut(descriptor, root, reader,
new StreamListener<PutResult>() {
new AsyncPutListener() {
int counter = 0;

@Override
Expand All @@ -118,16 +118,6 @@ public void onNext(PutResult val) {
}
counter++;
}

@Override
public void onError(Throwable t) {
completed.completeExceptionally(t);
}

@Override
public void onCompleted() {
completed.complete(null);
}
});
int counter = 0;
while (reader.read(root)) {
Expand Down
Loading

0 comments on commit f302c79

Please sign in to comment.