gianm commented on code in PR #12848:
URL: https://github.com/apache/druid/pull/12848#discussion_r936943460


##########
processing/src/main/java/org/apache/druid/frame/channel/ReadableInputStreamFrameChannel.java:
##########
@@ -0,0 +1,197 @@
+/*
+ * 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.druid.frame.channel;
+
+import com.google.common.util.concurrent.ListenableFuture;
+import com.google.errorprone.annotations.concurrent.GuardedBy;
+import org.apache.druid.frame.Frame;
+import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.java.util.common.concurrent.Execs;
+import org.apache.druid.utils.CloseableUtils;
+
+import java.io.InputStream;
+import java.util.Arrays;
+import java.util.Optional;
+import java.util.concurrent.ExecutorService;
+
+/**
+ * Channel backed by an {@link InputStream}.
+ *
+ * Frame channels are expected to be nonblocking, but InputStreams cannot be 
read in nonblocking fashion.
+ * This implementation deals with that by using an {@link ExecutorService} to 
read from the stream in a
+ * separate thread.
+ */
+public class ReadableInputStreamFrameChannel implements ReadableFrameChannel
+{
+  private final InputStream inputStream;
+  private final ReadableByteChunksFrameChannel delegate;
+  private final ExecutorService executorService;
+  private final Object lock = new Object();
+
+  @GuardedBy("lock")
+  private final byte[] buffer = new byte[8 * 1024];
+
+  @GuardedBy("lock")
+  private long totalInputStreamBytesRead = 0;
+
+  @GuardedBy("lock")
+  private boolean inputStreamFinished = false;
+
+  @GuardedBy("lock")
+  private boolean inputStreamError = false;
+
+  private volatile boolean readingStarted = false;
+  private volatile boolean keepReading = true;
+
+  public ReadableInputStreamFrameChannel(InputStream inputStream, String id, 
ExecutorService executorService)
+  {
+    this.inputStream = inputStream;
+    this.delegate = ReadableByteChunksFrameChannel.create(id);
+    this.executorService = executorService;
+  }
+
+  /**
+   * Method needs to be called for reading of input streams into 
ByteChunksFrameChannel
+   */
+  public void startReading()

Review Comment:
   Hmm. I don't there there is a good reason. All current and future planned 
usages call `startReading()` immediately after construction. I'll combine the 
functionality, possibly into something like 
`ReadableInputStreamFrameChannel.open(...)`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to