mxm commented on code in PR #12979:
URL: https://github.com/apache/iceberg/pull/12979#discussion_r2104030673


##########
flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/CommittableToTableChangeConverter.java:
##########
@@ -0,0 +1,181 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import java.util.List;
+import org.apache.flink.api.common.functions.OpenContext;
+import org.apache.flink.api.common.state.CheckpointListener;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.core.io.SimpleVersionedSerialization;
+import org.apache.flink.runtime.state.FunctionInitializationContext;
+import org.apache.flink.runtime.state.FunctionSnapshotContext;
+import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
+import org.apache.flink.streaming.api.connector.sink2.CommittableMessage;
+import org.apache.flink.streaming.api.connector.sink2.CommittableWithLineage;
+import org.apache.flink.streaming.api.functions.ProcessFunction;
+import org.apache.flink.util.Collector;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.flink.maintenance.operator.TableChange;
+import org.apache.iceberg.io.WriteResult;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class CommittableToTableChangeConverter
+    extends ProcessFunction<CommittableMessage<IcebergCommittable>, 
TableChange>
+    implements CheckpointedFunction, CheckpointListener {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(CommittableToTableChangeConverter.class);
+
+  private final TableLoader tableLoader;
+  private transient Table table;
+  private transient ListState<ManifestFile> manifestFilesToRemoveState;
+  private transient List<ManifestFile> manifestFilesToRemoveList;
+  private transient long lastCompletedCheckpointId = -1L;
+  private transient String flinkJobId;
+
+  public CommittableToTableChangeConverter(TableLoader tableLoader) {
+    Preconditions.checkNotNull(tableLoader, "TableLoader should not be null");
+    this.tableLoader = tableLoader;
+  }
+
+  @Override
+  public void initializeState(FunctionInitializationContext context) throws 
Exception {
+    this.manifestFilesToRemoveList = Lists.newArrayList();
+    this.manifestFilesToRemoveState =
+        context
+            .getOperatorStateStore()
+            .getListState(new ListStateDescriptor<>("manifests-to-remove", 
ManifestFile.class));
+    if (context.isRestored()) {
+      manifestFilesToRemoveList = 
Lists.newArrayList(manifestFilesToRemoveState.get());
+    }
+  }
+
+  @Override
+  public void open(OpenContext openContext) throws Exception {
+    super.open(openContext);
+    this.flinkJobId = getRuntimeContext().getJobId().toString();
+    if (!tableLoader.isOpen()) {
+      tableLoader.open();
+    }
+    this.table = tableLoader.loadTable();
+  }
+
+  @Override
+  public void snapshotState(FunctionSnapshotContext context) throws Exception {
+    manifestFilesToRemoveState.update(manifestFilesToRemoveList);
+  }
+
+  @Override
+  public void processElement(
+      CommittableMessage<IcebergCommittable> value,
+      ProcessFunction<CommittableMessage<IcebergCommittable>, 
TableChange>.Context ctx,
+      Collector<TableChange> out)
+      throws Exception {
+    if (value instanceof CommittableWithLineage) {
+      CommittableWithLineage<IcebergCommittable> committable =
+          (CommittableWithLineage<IcebergCommittable>) value;
+      TableChange tableChange = 
convertToTableChange(committable.getCommittable());
+      out.collect(tableChange);
+    }

Review Comment:
   Should we log a warning here in case of other value types?



##########
flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSinkCompaction.java:
##########
@@ -0,0 +1,141 @@
+/*
+ * 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;
+
+import java.util.List;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.EnvironmentSettings;
+import org.apache.flink.table.api.Expressions;
+import org.apache.flink.table.api.TableEnvironment;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.iceberg.Parameter;
+import org.apache.iceberg.Parameters;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.flink.source.BoundedTableFactory;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.TestTemplate;
+
+public class TestFlinkTableSinkCompaction extends CatalogTestBase {
+
+  private static final String TABLE_NAME = "test_table";
+  private TableEnvironment tEnv;
+  private Table icebergTable;
+  private static final String TABLE_PROPERTIES =
+      
"'flink-maintenance.lock.type'='jdbc','flink-maintenance.lock.jdbc.uri'='jdbc:sqlite:file::memory:?ic','flink-maintenance.lock.jdbc.init-lock-table'='true','flink-maintenance.rewrite.rewrite-all'='true','flink-maintenance.rewrite.schedule.data-file-size'='1','flink-maintenance.lock-check-delay-seconds'='60'";
+
+  @Parameter(index = 2)
+  private boolean userSqlHint = true;
+
+  @Parameters(name = "catalogName={0}, baseNamespace={1}, userSqlHint={2}")
+  public static List<Object[]> parameters() {
+    List<Object[]> parameters = Lists.newArrayList();
+
+    for (Boolean userSqlHint : new Boolean[] {true, false}) {
+      String catalogName = "testhadoop_basenamespace";
+      Namespace baseNamespace = Namespace.of("l0", "l1");
+      parameters.add(new Object[] {catalogName, baseNamespace, userSqlHint});
+    }
+    return parameters;
+  }
+
+  @Override
+  protected TableEnvironment getTableEnv() {
+    if (tEnv == null) {
+      synchronized (this) {

Review Comment:
   Is this lock required?



##########
flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/LockConfig.java:
##########
@@ -0,0 +1,46 @@
+/*
+ * 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.maintenance.api;
+
+public class LockConfig {
+
+  private LockConfig() {}
+
+  public static final String CONFIG_PREFIX = 
TableMaintenanceConfig.CONFIG_PREFIX + "lock.";
+  public static final String LOCK_TYPE = "type";
+  public static final String LOCK_ID = "lock-id";
+
+  // JDBC
+  public static final String JDBC = "jdbc";
+  public static final String JDBC_URI = JDBC + ".uri";
+  public static final String JDBC_INIT_LOCK_TABLE = JDBC + ".init-lock-table";
+
+  // zk
+  public static final String ZK = "zookeeper";

Review Comment:
   NIT: Putting the different Lock options in separate inner classes would 
provide more structure.



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