You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2020/10/27 09:42:49 UTC

[GitHub] [iceberg] simonsssu opened a new pull request #1669: WIP add rewrite file operator after iceberg committer

simonsssu opened a new pull request #1669:
URL: https://github.com/apache/iceberg/pull/1669


   


----------------------------------------------------------------
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.

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


[GitHub] [iceberg] rdblue commented on pull request #1669: WIP add rewrite file operator after iceberg committer

Posted by GitBox <gi...@apache.org>.
rdblue commented on pull request #1669:
URL: https://github.com/apache/iceberg/pull/1669#issuecomment-720811031


   @openinx, please let me know if/when you'd like me to take a look at this PR.


----------------------------------------------------------------
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.

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


[GitHub] [iceberg] openinx commented on a change in pull request #1669: WIP add rewrite file operator after iceberg committer

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1669:
URL: https://github.com/apache/iceberg/pull/1669#discussion_r516383842



##########
File path: flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -179,37 +186,93 @@ public Builder writeParallelism(int newWriteParallelism) {
         try (TableLoader loader = tableLoader) {
           this.table = loader.loadTable();
         } catch (IOException e) {
-          throw new UncheckedIOException("Failed to load iceberg table from table loader: " + tableLoader, e);
+          throw new UncheckedIOException(
+              "Failed to load iceberg table from table loader: " + tableLoader,
+              e);
         }
       }
 
       IcebergStreamWriter<RowData> streamWriter = createStreamWriter(table, tableSchema);
       IcebergFilesCommitter filesCommitter = new IcebergFilesCommitter(tableLoader, overwrite);
 
-      this.writeParallelism = writeParallelism == null ? rowDataInput.getParallelism() : writeParallelism;
-
-      DataStream<Void> returnStream = rowDataInput
-          .transform(ICEBERG_STREAM_WRITER_NAME, TypeInformation.of(DataFile.class), streamWriter)
-          .setParallelism(writeParallelism)
-          .transform(ICEBERG_FILES_COMMITTER_NAME, Types.VOID, filesCommitter)
+      DataStream<?> returnStream = rowDataInput
+          .transform(ICEBERG_STREAM_WRITER_NAME,
+              TypeInformation.of(DataFile.class), streamWriter)
+          .setParallelism(rowDataInput.getParallelism())

Review comment:
       Why change this ? we will need to allow users to set their own writer parallelism, so I think we may need to revert this ? 

##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkTableProperties.java
##########
@@ -0,0 +1,44 @@
+/*
+ * 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.iceberg.flink;
+
+/**
+ * Table Properties when using Flink IcebergSink.
+ */
+public class FlinkTableProperties {
+
+  private FlinkTableProperties() {
+  }
+
+  public static final String CASE_SENSITIVE = "case-sensitive";

Review comment:
       Those properties are provided for flink only ? we'd better to have a `flink.` prefix in the keys. 

##########
File path: flink/src/main/java/org/apache/iceberg/flink/sink/rewrite/AbstractRewriteOperator.java
##########
@@ -0,0 +1,163 @@
+/*
+ * 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.iceberg.flink.sink.rewrite;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.BoundedOneInput;
+import org.apache.flink.streaming.api.operators.ChainingStrategy;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.util.Preconditions;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.flink.FlinkTableProperties;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.CloseableIterator;
+import org.apache.iceberg.relocated.com.google.common.collect.ListMultimap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Multimaps;
+import org.apache.iceberg.relocated.com.google.common.collect.Streams;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.StructLikeWrapper;
+import org.apache.iceberg.util.TableScanUtil;
+
+abstract class AbstractRewriteOperator<InT, OutT> extends AbstractStreamOperator<OutT>
+    implements OneInputStreamOperator<InT, OutT>, BoundedOneInput {

Review comment:
       We've abstracted the `BaseRewriteDataFilesAction` in PR https://github.com/apache/iceberg/pull/1624, I think it's good to reuse that part of code.  we could abstract it again when it's necessary.

##########
File path: flink/src/main/java/org/apache/iceberg/flink/sink/rewrite/RowDataRewriter.java
##########
@@ -0,0 +1,86 @@
+/*
+ * 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.iceberg.flink.sink.rewrite;
+
+import java.io.Serializable;
+import java.util.List;
+import org.apache.flink.table.data.RowData;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.encryption.EncryptionManager;
+import org.apache.iceberg.flink.sink.TaskWriterFactory;
+import org.apache.iceberg.flink.source.RowDataIterator;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.TaskWriter;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class RowDataRewriter implements Serializable {

Review comment:
       We will have a `Rewriter` once the  https://github.com/apache/iceberg/pull/1623/files#diff-81a2df27390859ae0b203aec87391f01a201ea07d2e7226a9e84e87f74dd3e5fR50  get merged,   so will need to consider sharing code between them. 




----------------------------------------------------------------
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.

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


[GitHub] [iceberg] openinx commented on pull request #1669: WIP add rewrite file operator after iceberg committer

Posted by GitBox <gi...@apache.org>.
openinx commented on pull request #1669:
URL: https://github.com/apache/iceberg/pull/1669#issuecomment-723837802


   @simonsssu Mind to grant the access to the document ? 


----------------------------------------------------------------
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.

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


[GitHub] [iceberg] zhangjun0x01 commented on pull request #1669: WIP add rewrite file operator after iceberg committer

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on pull request #1669:
URL: https://github.com/apache/iceberg/pull/1669#issuecomment-718305151


   In the case of a large amount of data, in order to ensure the performance of normal writing,
   1. Should we use asynchronous Rewrite? (use AsyncWaitOperator)
   2. Should you add a switch to control whether to  enable rewrite


----------------------------------------------------------------
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.

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


[GitHub] [iceberg] zhangjun0x01 commented on a change in pull request #1669: WIP add rewrite file operator after iceberg committer

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #1669:
URL: https://github.com/apache/iceberg/pull/1669#discussion_r513347072



##########
File path: flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -179,37 +186,93 @@ public Builder writeParallelism(int newWriteParallelism) {
         try (TableLoader loader = tableLoader) {
           this.table = loader.loadTable();
         } catch (IOException e) {
-          throw new UncheckedIOException("Failed to load iceberg table from table loader: " + tableLoader, e);
+          throw new UncheckedIOException(
+              "Failed to load iceberg table from table loader: " + tableLoader,
+              e);
         }
       }
 
       IcebergStreamWriter<RowData> streamWriter = createStreamWriter(table, tableSchema);
       IcebergFilesCommitter filesCommitter = new IcebergFilesCommitter(tableLoader, overwrite);
 
-      this.writeParallelism = writeParallelism == null ? rowDataInput.getParallelism() : writeParallelism;
-
-      DataStream<Void> returnStream = rowDataInput
-          .transform(ICEBERG_STREAM_WRITER_NAME, TypeInformation.of(DataFile.class), streamWriter)
-          .setParallelism(writeParallelism)
-          .transform(ICEBERG_FILES_COMMITTER_NAME, Types.VOID, filesCommitter)
+      DataStream<?> returnStream = rowDataInput
+          .transform(ICEBERG_STREAM_WRITER_NAME,
+              TypeInformation.of(DataFile.class), streamWriter)
+          .setParallelism(rowDataInput.getParallelism())
+          .transform(ICEBERG_FILES_COMMITTER_NAME,
+              TypeInformation.of(Long.class), filesCommitter)
           .setParallelism(1)

Review comment:
       I feel that the master is not the latest version when the patch is rebase




----------------------------------------------------------------
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.

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


[GitHub] [iceberg] JingsongLi commented on pull request #1669: WIP add rewrite file operator after iceberg committer

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on pull request #1669:
URL: https://github.com/apache/iceberg/pull/1669#issuecomment-720930842


   Thanks @simonsssu for your contribution.
   If you don't mind, can you write a design document?
   I think it is good to have a global design.
   
   I have some initial question:
   - When to do compaction?
   - Does compaction block checkpoint? If there are large number of files need to be compacted, that will lead to the timeout/failure of checkpoint.
   - How to do incremental compaction instead of compacting the whole table?
   - When to commit compaction? Does it support multiple streaming sinks?
   - in addition, can we reuse more codes in https://github.com/apache/iceberg/pull/1623 ? I think essentially they do the same job.


----------------------------------------------------------------
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.

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


[GitHub] [iceberg] openinx commented on pull request #1669: WIP add rewrite file operator after iceberg committer

Posted by GitBox <gi...@apache.org>.
openinx commented on pull request #1669:
URL: https://github.com/apache/iceberg/pull/1669#issuecomment-720827476


   Will take a look today, thanks.


----------------------------------------------------------------
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.

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


[GitHub] [iceberg] simonsssu commented on pull request #1669: WIP add rewrite file operator after iceberg committer

Posted by GitBox <gi...@apache.org>.
simonsssu commented on pull request #1669:
URL: https://github.com/apache/iceberg/pull/1669#issuecomment-724389141


   @openinx @JingsongLi Done 
   https://docs.google.com/document/d/1pK2aN9Nv1HjYaJFYTQUWsttp4ahwFPoZbvmPpZ0T32k/edit?usp=sharing


----------------------------------------------------------------
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.

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


[GitHub] [iceberg] zhangjun0x01 commented on a change in pull request #1669: WIP add rewrite file operator after iceberg committer

Posted by GitBox <gi...@apache.org>.
zhangjun0x01 commented on a change in pull request #1669:
URL: https://github.com/apache/iceberg/pull/1669#discussion_r513333204



##########
File path: flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -179,37 +186,93 @@ public Builder writeParallelism(int newWriteParallelism) {
         try (TableLoader loader = tableLoader) {
           this.table = loader.loadTable();
         } catch (IOException e) {
-          throw new UncheckedIOException("Failed to load iceberg table from table loader: " + tableLoader, e);
+          throw new UncheckedIOException(
+              "Failed to load iceberg table from table loader: " + tableLoader,
+              e);

Review comment:
       it seems no change




----------------------------------------------------------------
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.

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


[GitHub] [iceberg] simonsssu commented on a change in pull request #1669: WIP add rewrite file operator after iceberg committer

Posted by GitBox <gi...@apache.org>.
simonsssu commented on a change in pull request #1669:
URL: https://github.com/apache/iceberg/pull/1669#discussion_r516530184



##########
File path: flink/src/main/java/org/apache/iceberg/flink/sink/rewrite/RowDataRewriter.java
##########
@@ -0,0 +1,86 @@
+/*
+ * 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.iceberg.flink.sink.rewrite;
+
+import java.io.Serializable;
+import java.util.List;
+import org.apache.flink.table.data.RowData;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.encryption.EncryptionManager;
+import org.apache.iceberg.flink.sink.TaskWriterFactory;
+import org.apache.iceberg.flink.source.RowDataIterator;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.TaskWriter;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class RowDataRewriter implements Serializable {

Review comment:
       Yes, we can reuse most of logic here.




----------------------------------------------------------------
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.

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


[GitHub] [iceberg] simonsssu commented on pull request #1669: WIP add rewrite file operator after iceberg committer

Posted by GitBox <gi...@apache.org>.
simonsssu commented on pull request #1669:
URL: https://github.com/apache/iceberg/pull/1669#issuecomment-723812718


   https://docs.google.com/document/d/1pK2aN9Nv1HjYaJFYTQUWsttp4ahwFPoZbvmPpZ0T32k/edit#heading=h.asry5i6qtp9p
   @JingsongLi  Hi Jingsong, I have draft a simple doc. Sorry for late response.


----------------------------------------------------------------
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.

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


[GitHub] [iceberg] simonsssu commented on pull request #1669: WIP add rewrite file operator after iceberg committer

Posted by GitBox <gi...@apache.org>.
simonsssu commented on pull request #1669:
URL: https://github.com/apache/iceberg/pull/1669#issuecomment-721004658


   @JingsongLi Hi Jingsong, Sorry for late response.  I will write a design doc in google doc. 


----------------------------------------------------------------
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.

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