pvary commented on code in PR #10179:
URL: https://github.com/apache/iceberg/pull/10179#discussion_r1698264337


##########
flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/committer/IcebergWriteAggregator.java:
##########
@@ -0,0 +1,130 @@
+/*
+ * 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.committer;
+
+import java.io.IOException;
+import java.util.Collection;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.core.io.SimpleVersionedSerialization;
+import org.apache.flink.streaming.api.connector.sink2.CommittableMessage;
+import org.apache.flink.streaming.api.connector.sink2.CommittableSummary;
+import org.apache.flink.streaming.api.connector.sink2.CommittableWithLineage;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.flink.sink.DeltaManifests;
+import org.apache.iceberg.flink.sink.DeltaManifestsSerializer;
+import org.apache.iceberg.flink.sink.FlinkManifestUtil;
+import org.apache.iceberg.flink.sink.ManifestOutputFileFactory;
+import org.apache.iceberg.io.WriteResult;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Operator which aggregates the individual {@link WriteResult} objects) to a 
single {@link
+ * IcebergCommittable} per checkpoint (storing the serialized {@link 
DeltaManifests}, jobId,
+ * operatorId, checkpointId)
+ */
+@Internal
+public class IcebergWriteAggregator
+    extends AbstractStreamOperator<CommittableMessage<IcebergCommittable>>
+    implements OneInputStreamOperator<
+        CommittableMessage<WriteResult>, 
CommittableMessage<IcebergCommittable>> {
+  private static final Logger LOG = 
LoggerFactory.getLogger(IcebergWriteAggregator.class);
+  private static final byte[] EMPTY_MANIFEST_DATA = new byte[0];
+  private final Collection<WriteResult> results;
+  private transient ManifestOutputFileFactory icebergManifestOutputFileFactory;
+  private transient Table table;
+  private final TableLoader tableLoader;
+  private static final int FORMAT_V2 = 2;
+  private static final Long DUMMY_SNAPSHOT_ID = 0L;
+
+  public IcebergWriteAggregator(TableLoader tableLoader) {

Review Comment:
   We can check, but if my memory serves me well, we can not call 
`FlinkManifestUtil.createOutputFileFactory` with a static table. The issue was 
with `StaticTableOperations.metadataFileLocation`:
   ```
     @Override
     public String metadataFileLocation(String fileName) {
       throw new UnsupportedOperationException("Cannot modify a static table");
     }
   ```



-- 
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-unsubscr...@iceberg.apache.org

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


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

Reply via email to