You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by "stevenzwu (via GitHub)" <gi...@apache.org> on 2023/01/31 03:47:43 UTC

[GitHub] [iceberg] stevenzwu commented on a diff in pull request #6614: Flink:fix flink streaming query problem [ Cannot get a client from a closed pool]

stevenzwu commented on code in PR #6614:
URL: https://github.com/apache/iceberg/pull/6614#discussion_r1091413180


##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlannerImpl.java:
##########
@@ -62,11 +64,26 @@ public ContinuousSplitPlannerImpl(Table table, ScanContext scanContext, String t
                 "iceberg-plan-worker-pool-" + threadName, scanContext.planParallelism());
   }
 
+  public ContinuousSplitPlannerImpl(

Review Comment:
   we can probably remove or directly change the other constructor. this is an `@Internal` class. not user facing.



##########
flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/TableLoader.java:
##########
@@ -34,12 +34,16 @@
  * the cluster (for example, to get splits), not just on the client side. So we need an Iceberg
  * table loader to get the {@link Table} object.
  */
-public interface TableLoader extends Closeable, Serializable {
+public interface TableLoader extends Closeable, Serializable, Cloneable {
 
   void open();
 
   Table loadTable();
 
+  /** By shallow cloning, clone a tableLoader. */
+  @SuppressWarnings("checkstyle:NoClone")

Review Comment:
   instead of shallow cloning and suppress checkstyle warnings, can we implement proper clone. that would require `CatalogLoader` also implements `Cloneable` interface. From my quick look, it doesn't seem too bad.
   
   cc @hililiwei @pvary for more comments. 



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org