JingsongLi commented on code in PR #437:
URL: https://github.com/apache/flink-table-store/pull/437#discussion_r1048142161


##########
flink-table-store-connector/src/main/java/org/apache/flink/table/store/connector/TableStoreCompactJob.java:
##########
@@ -0,0 +1,147 @@
+/*
+ * 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.flink.table.store.connector;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.RuntimeExecutionMode;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.ExecutionOptions;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.core.execution.JobClient;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.streaming.api.datastream.DataStreamSource;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.store.CoreOptions;
+import org.apache.flink.table.store.connector.sink.CompactorSinkBuilder;
+import org.apache.flink.table.store.connector.source.CompactorSourceBuilder;
+import 
org.apache.flink.table.store.connector.utils.StreamExecutionEnvironmentUtils;
+import org.apache.flink.table.store.table.FileStoreTable;
+import org.apache.flink.table.store.table.FileStoreTableFactory;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/** Stand-alone compact job for table store. */
+public class TableStoreCompactJob {
+
+    public static void main(String[] args) throws Exception {
+        if (args.length < 1 || args.length > 4) {
+            printHelp();
+            System.exit(1);
+        }
+
+        Path location;
+        if (args.length <= 2) {
+            location = new Path(args[0]);

Review Comment:
   Maybe we can use `ParameterTool` or `MultipleParameterTool` to translate key 
values here.



##########
flink-table-store-connector/src/main/java/org/apache/flink/table/store/connector/TableStoreCompactJob.java:
##########
@@ -0,0 +1,147 @@
+/*
+ * 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.flink.table.store.connector;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.RuntimeExecutionMode;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.ExecutionOptions;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.core.execution.JobClient;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.streaming.api.datastream.DataStreamSource;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.store.CoreOptions;
+import org.apache.flink.table.store.connector.sink.CompactorSinkBuilder;
+import org.apache.flink.table.store.connector.source.CompactorSourceBuilder;
+import 
org.apache.flink.table.store.connector.utils.StreamExecutionEnvironmentUtils;
+import org.apache.flink.table.store.table.FileStoreTable;
+import org.apache.flink.table.store.table.FileStoreTableFactory;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/** Stand-alone compact job for table store. */
+public class TableStoreCompactJob {

Review Comment:
   Can we have a `Actions` class like in iceberg.
   Call different types of jobs through parameters, such as: `-o compact` (o: 
operation).



##########
flink-table-store-connector/src/main/java/org/apache/flink/table/store/connector/TableStoreCompactJob.java:
##########
@@ -0,0 +1,147 @@
+/*
+ * 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.flink.table.store.connector;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.RuntimeExecutionMode;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.ExecutionOptions;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.core.execution.JobClient;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.streaming.api.datastream.DataStreamSource;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.store.CoreOptions;
+import org.apache.flink.table.store.connector.sink.CompactorSinkBuilder;
+import org.apache.flink.table.store.connector.source.CompactorSourceBuilder;
+import 
org.apache.flink.table.store.connector.utils.StreamExecutionEnvironmentUtils;
+import org.apache.flink.table.store.table.FileStoreTable;
+import org.apache.flink.table.store.table.FileStoreTableFactory;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/** Stand-alone compact job for table store. */
+public class TableStoreCompactJob {
+
+    public static void main(String[] args) throws Exception {
+        if (args.length < 1 || args.length > 4) {
+            printHelp();
+            System.exit(1);
+        }
+
+        Path location;
+        if (args.length <= 2) {
+            location = new Path(args[0]);
+        } else {
+            location = new Path(new Path(args[0], args[1] + ".db"), args[2]);
+        }
+
+        String partitionsSpecString = null;
+        if (args.length == 2) {
+            partitionsSpecString = args[1];
+        } else if (args.length == 4) {
+            partitionsSpecString = args[3];
+        }
+
+        StreamExecutionEnvironment env = 
StreamExecutionEnvironment.getExecutionEnvironment();
+
+        runCompactJob(location, partitionsSpecString, env);
+    }
+
+    @VisibleForTesting
+    public static JobClient runCompactJob(
+            Path location, String partitionsSpecString, 
StreamExecutionEnvironment env)
+            throws Exception {
+        Configuration tableOptions = new Configuration();
+        tableOptions.set(CoreOptions.PATH, location.toString());
+        tableOptions.set(CoreOptions.WRITE_COMPACTION_SKIP, false);
+        FileStoreTable table = FileStoreTableFactory.create(tableOptions);
+
+        List<Map<String, String>> specifiedPartition = 
parsePartitionSpec(partitionsSpecString);
+
+        ReadableConfig conf = 
StreamExecutionEnvironmentUtils.getConfiguration(env);
+        boolean isStreaming =
+                conf.get(ExecutionOptions.RUNTIME_MODE) == 
RuntimeExecutionMode.STREAMING;
+
+        DataStreamSource<RowData> source =
+                new CompactorSourceBuilder(location.toString(), table)
+                        .withEnv(env)
+                        .withContinuousMode(isStreaming)
+                        .withPartitions(specifiedPartition)
+                        .build();
+        new CompactorSinkBuilder(table).withInput(source).build();
+        JobClient client = env.executeAsync("Compaction Job: " + location);

Review Comment:
   Just use `execute(String jobName)`? Let `execution.attached` to decide 
whether to wait the execution.



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

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

Reply via email to