garyli1019 commented on a change in pull request #3046:
URL: https://github.com/apache/hudi/pull/3046#discussion_r647076300



##########
File path: 
hudi-flink/src/main/java/org/apache/hudi/sink/compact/CompactionCommitSource.java
##########
@@ -0,0 +1,152 @@
+package org.apache.hudi.sink.compact;

Review comment:
       missing apache license. 

##########
File path: 
hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/compact/FlinkScheduleCompactionActionExecutor.java
##########
@@ -149,4 +155,41 @@ public Long parsedToSeconds(String time) {
     }
     return timestamp;
   }
+
+  @Override
+  public Option<HoodieCompactionPlan> execute() {
+    if 
(!config.getWriteConcurrencyMode().supportsOptimisticConcurrencyControl()
+            && !config.getFailedWritesCleanPolicy().isLazy()) {
+      // if there are inflight writes, their instantTime must not be less than 
that of compaction instant time
+      
table.getActiveTimeline().getCommitsTimeline().filterPendingExcludingCompaction().firstInstant()

Review comment:
       should we move these validation checks to a commonplace? Looks like this 
was being used elsewhere too.

##########
File path: hudi-flink/src/test/java/org/apache/hudi/sink/StreamWriteITCase.java
##########
@@ -137,6 +145,60 @@ public void testWriteToHoodie() throws Exception {
     TestData.checkWrittenFullData(tempFile, EXPECTED);
   }
 
+  @Test
+  public void testHoodieFlinkCompactor() throws Exception {
+    // Create hoodie table and insert into data.
+    EnvironmentSettings settings = 
EnvironmentSettings.newInstance().inBatchMode().build();
+    TableEnvironment tableEnv = TableEnvironmentImpl.create(settings);
+    tableEnv.getConfig().getConfiguration()
+            
.setInteger(ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 1);
+    Map<String, String> options = new HashMap<>();
+    options.put(FlinkOptions.PATH.key(), tempFile.getAbsolutePath());
+    options.put(FlinkOptions.TABLE_TYPE.key(), "MERGE_ON_READ");
+    String hoodieTableDDL = TestConfigurations.getCreateHoodieTableDDL("t1", 
options);
+    tableEnv.executeSql(hoodieTableDDL);
+    String insertInto = "insert into t1 values\n"
+            + "('id1','Danny',23,TIMESTAMP '1970-01-01 00:00:01','par1'),\n"
+            + "('id2','Stephen',33,TIMESTAMP '1970-01-01 00:00:02','par1'),\n"
+            + "('id3','Julian',53,TIMESTAMP '1970-01-01 00:00:03','par2'),\n"
+            + "('id4','Fabian',31,TIMESTAMP '1970-01-01 00:00:04','par2'),\n"
+            + "('id5','Sophia',18,TIMESTAMP '1970-01-01 00:00:05','par3'),\n"
+            + "('id6','Emma',20,TIMESTAMP '1970-01-01 00:00:06','par3'),\n"
+            + "('id7','Bob',44,TIMESTAMP '1970-01-01 00:00:07','par4'),\n"
+            + "('id8','Han',56,TIMESTAMP '1970-01-01 00:00:08','par4')";
+    TableResult tableResult = tableEnv.executeSql(insertInto);
+    TimeUnit.SECONDS.sleep(5);
+    tableResult.await();
+
+    // Make configuration and setAvroSchema.
+    StreamExecutionEnvironment env = 
StreamExecutionEnvironment.getExecutionEnvironment();
+    FlinkCompactionConfig cfg = new FlinkCompactionConfig();
+    cfg.path = tempFile.getAbsolutePath();
+    cfg.hoodieTableName = "t1";
+    Configuration conf = FlinkCompactionConfig.toCompactionConfig(cfg);
+    conf.setString(FlinkOptions.TABLE_TYPE.key(), "MERGE_ON_READ");
+    conf.setString(FlinkOptions.PARTITION_PATH_FIELD.key(), "partition");
+
+    // set table schema.
+    CompactionUtil.setAvroSchema(conf);
+
+    env.addSource(new CompactionCommitSource(conf))
+        .name("compaction_source")
+        .uid("uid_compaction_source")
+        .keyBy(event -> event.getOperation().hashCode())

Review comment:
       I found this `keyBy` will unevenly distribute the task cause the total 
events number are relatively small. Maybe we could find a better partitioning 
strategy?

##########
File path: 
hudi-flink/src/main/java/org/apache/hudi/sink/compact/CompactionCommitSource.java
##########
@@ -0,0 +1,152 @@
+package org.apache.hudi.sink.compact;
+
+import org.apache.hudi.avro.model.HoodieCompactionPlan;
+import org.apache.hudi.client.HoodieFlinkWriteClient;
+import org.apache.hudi.common.model.CompactionOperation;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.table.timeline.HoodieTimeline;
+import org.apache.hudi.common.util.CompactionUtils;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.exception.HoodieIOException;
+import org.apache.hudi.table.HoodieFlinkTable;
+import org.apache.hudi.util.CompactionUtil;
+import org.apache.hudi.util.StreamerUtil;
+
+import org.apache.flink.api.common.functions.AbstractRichFunction;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.hadoop.fs.Path;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Objects;
+
+import static java.util.stream.Collectors.toList;
+
+/**
+ * flink hudi compaction source function.
+ * @author zhengyunhong.

Review comment:
       we usually don't add personal info to the codebase.

##########
File path: 
hudi-flink/src/main/java/org/apache/hudi/sink/compact/CompactionCommitSource.java
##########
@@ -0,0 +1,152 @@
+package org.apache.hudi.sink.compact;
+
+import org.apache.hudi.avro.model.HoodieCompactionPlan;
+import org.apache.hudi.client.HoodieFlinkWriteClient;
+import org.apache.hudi.common.model.CompactionOperation;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.table.timeline.HoodieTimeline;
+import org.apache.hudi.common.util.CompactionUtils;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.exception.HoodieIOException;
+import org.apache.hudi.table.HoodieFlinkTable;
+import org.apache.hudi.util.CompactionUtil;
+import org.apache.hudi.util.StreamerUtil;
+
+import org.apache.flink.api.common.functions.AbstractRichFunction;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.hadoop.fs.Path;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Objects;
+
+import static java.util.stream.Collectors.toList;
+
+/**
+ * flink hudi compaction source function.

Review comment:
       Can we add more descriptions?




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


Reply via email to