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:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]