This is an automated email from the ASF dual-hosted git repository.

lirui pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


The following commit(s) were added to refs/heads/master by this push:
     new 1b1d6fc  [FLINK-23316][table-runtime][hive] Add test for Custom 
PartitionCommitPolicy
1b1d6fc is described below

commit 1b1d6fc2518b78e34d69c5f17fcf6b21ae0f2182
Author: Jack <[email protected]>
AuthorDate: Sun Sep 12 10:58:09 2021 +0800

    [FLINK-23316][table-runtime][hive] Add test for Custom PartitionCommitPolicy
    
    This closes #17245
---
 .../flink/connectors/hive/HiveTableSinkITCase.java | 115 +++++++++++++++++++++
 .../connectors/hive/TestCustomCommitPolicy.java    |  41 ++++++++
 .../table/filesystem/PartitionCommitPolicy.java    |   2 +-
 3 files changed, 157 insertions(+), 1 deletion(-)

diff --git 
a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveTableSinkITCase.java
 
b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveTableSinkITCase.java
index 4620186..0cb2335 100644
--- 
a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveTableSinkITCase.java
+++ 
b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveTableSinkITCase.java
@@ -18,8 +18,10 @@
 
 package org.apache.flink.connectors.hive;
 
+import org.apache.flink.api.common.restartstrategy.RestartStrategies;
 import org.apache.flink.api.common.typeinfo.Types;
 import org.apache.flink.api.java.typeutils.RowTypeInfo;
+import org.apache.flink.core.fs.Path;
 import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.util.FiniteTestSource;
@@ -37,6 +39,9 @@ import org.apache.flink.table.catalog.hive.HiveTestUtils;
 import org.apache.flink.types.Row;
 import org.apache.flink.util.CloseableIterator;
 import org.apache.flink.util.CollectionUtil;
+import org.apache.flink.util.ExceptionUtils;
+
+import org.apache.flink.shaded.guava30.com.google.common.collect.Lists;
 
 import org.junit.AfterClass;
 import org.junit.Assert;
@@ -54,11 +59,13 @@ import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 import java.util.function.Consumer;
 
 import static org.apache.flink.table.api.Expressions.$;
 import static 
org.apache.flink.table.filesystem.FileSystemConnectorOptions.PARTITION_TIME_EXTRACTOR_TIMESTAMP_PATTERN;
 import static 
org.apache.flink.table.filesystem.FileSystemConnectorOptions.SINK_PARTITION_COMMIT_DELAY;
+import static 
org.apache.flink.table.filesystem.FileSystemConnectorOptions.SINK_PARTITION_COMMIT_POLICY_CLASS;
 import static 
org.apache.flink.table.filesystem.FileSystemConnectorOptions.SINK_PARTITION_COMMIT_POLICY_KIND;
 import static 
org.apache.flink.table.filesystem.FileSystemConnectorOptions.SINK_PARTITION_COMMIT_SUCCESS_FILE_NAME;
 import static 
org.apache.flink.table.planner.utils.TableTestUtil.readFromResource;
@@ -421,6 +428,28 @@ public class HiveTableSinkITCase {
         }
     }
 
+    @Test
+    public void testCustomPartitionCommitPolicyNotFound() {
+        String customCommitPolicyClassName = 
"NotExistPartitionCommitPolicyClass";
+
+        try {
+            
testStreamingWriteWithCustomPartitionCommitPolicy(customCommitPolicyClassName);
+            fail("ExecutionException expected");
+        } catch (Exception e) {
+            assertTrue(
+                    ExceptionUtils.findThrowableWithMessage(
+                                    e,
+                                    "Can not create new instance for custom 
class from "
+                                            + customCommitPolicyClassName)
+                            .isPresent());
+        }
+    }
+
+    @Test
+    public void testCustomPartitionCommitPolicy() throws Exception {
+        
testStreamingWriteWithCustomPartitionCommitPolicy(TestCustomCommitPolicy.class.getName());
+    }
+
     private static List<String> fetchRows(Iterator<Row> iter, int size) {
         List<String> strings = new ArrayList<>(size);
         for (int i = 0; i < size; i++) {
@@ -441,6 +470,92 @@ public class HiveTableSinkITCase {
         Assert.assertTrue(new File(new File(basePath, "e=11"), 
"_MY_SUCCESS").exists());
     }
 
+    private void testStreamingWriteWithCustomPartitionCommitPolicy(
+            String customPartitionCommitPolicyClassName) throws Exception {
+        StreamExecutionEnvironment env = 
StreamExecutionEnvironment.getExecutionEnvironment();
+        env.setParallelism(1);
+        env.enableCheckpointing(100);
+        // avoid the job to restart infinitely
+        env.setRestartStrategy(RestartStrategies.fixedDelayRestart(3, 1_000));
+
+        StreamTableEnvironment tEnv = 
HiveTestUtils.createTableEnvInStreamingMode(env);
+        tEnv.registerCatalog(hiveCatalog.getName(), hiveCatalog);
+        tEnv.useCatalog(hiveCatalog.getName());
+        tEnv.getConfig().setSqlDialect(SqlDialect.HIVE);
+
+        try {
+            tEnv.executeSql("create database db1");
+            tEnv.useDatabase("db1");
+
+            // prepare source
+            List<Row> data =
+                    Arrays.asList(
+                            Row.of(1, "a", "b", "2020-05-03", "7"),
+                            Row.of(2, "p", "q", "2020-05-03", "8"),
+                            Row.of(3, "x", "y", "2020-05-03", "9"),
+                            Row.of(4, "x", "y", "2020-05-03", "10"),
+                            Row.of(5, "x", "y", "2020-05-03", "11"));
+            DataStream<Row> stream =
+                    env.addSource(
+                            new FiniteTestSource<>(data),
+                            new RowTypeInfo(
+                                    Types.INT,
+                                    Types.STRING,
+                                    Types.STRING,
+                                    Types.STRING,
+                                    Types.STRING));
+            tEnv.createTemporaryView("my_table", stream, $("a"), $("b"), 
$("c"), $("d"), $("e"));
+
+            // DDL
+            tEnv.executeSql(
+                    "create external table sink_table (a int,b string,c string"
+                            + ") "
+                            + "partitioned by (d string,e string) "
+                            + " stored as textfile"
+                            + " TBLPROPERTIES ("
+                            + "'"
+                            + SINK_PARTITION_COMMIT_DELAY.key()
+                            + "'='1h',"
+                            + "'"
+                            + SINK_PARTITION_COMMIT_POLICY_KIND.key()
+                            + "'='metastore,custom',"
+                            + "'"
+                            + SINK_PARTITION_COMMIT_POLICY_CLASS.key()
+                            + "'='"
+                            + customPartitionCommitPolicyClassName
+                            + "'"
+                            + ")");
+
+            // hive dialect only works with hive tables at the moment, switch 
to default dialect
+            tEnv.getConfig().setSqlDialect(SqlDialect.DEFAULT);
+            tEnv.sqlQuery("select * from 
my_table").executeInsert("sink_table").await();
+
+            // check committed partitions for CustomizedCommitPolicy
+            Set<String> committedPaths =
+                    
TestCustomCommitPolicy.getCommittedPartitionPathsAndReset();
+            String base =
+                    URI.create(
+                                    hiveCatalog
+                                            
.getHiveTable(ObjectPath.fromString("db1.sink_table"))
+                                            .getSd()
+                                            .getLocation())
+                            .getPath();
+            List<String> partitionKVs = Lists.newArrayList("e=7", "e=8", 
"e=9", "e=10", "e=11");
+            partitionKVs.forEach(
+                    partitionKV -> {
+                        String partitionPath =
+                                new Path(new Path(base, "d=2020-05-03"), 
partitionKV).toString();
+                        Assert.assertTrue(
+                                "Partition(d=2020-05-03, "
+                                        + partitionKV
+                                        + ") is not committed successfully",
+                                committedPaths.contains(partitionPath));
+                    });
+        } finally {
+            tEnv.executeSql("drop database if exists db1 cascade");
+        }
+    }
+
     private void testStreamingWrite(
             boolean part, boolean useMr, String format, Consumer<String> 
pathConsumer)
             throws Exception {
diff --git 
a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/TestCustomCommitPolicy.java
 
b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/TestCustomCommitPolicy.java
new file mode 100644
index 0000000..5e96c29
--- /dev/null
+++ 
b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/TestCustomCommitPolicy.java
@@ -0,0 +1,41 @@
+/*
+ * 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.connectors.hive;
+
+import org.apache.flink.table.filesystem.PartitionCommitPolicy;
+
+import java.util.HashSet;
+import java.util.Set;
+
+/** A custom PartitionCommitPolicy for test. */
+public class TestCustomCommitPolicy implements PartitionCommitPolicy {
+
+    private static Set<String> committedPartitionPaths = new HashSet<>();
+
+    @Override
+    public void commit(PartitionCommitPolicy.Context context) throws Exception 
{
+        
TestCustomCommitPolicy.committedPartitionPaths.add(context.partitionPath().getPath());
+    }
+
+    static Set<String> getCommittedPartitionPathsAndReset() {
+        Set<String> paths = TestCustomCommitPolicy.committedPartitionPaths;
+        TestCustomCommitPolicy.committedPartitionPaths = new HashSet<>();
+        return paths;
+    }
+}
diff --git 
a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/PartitionCommitPolicy.java
 
b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/PartitionCommitPolicy.java
index 627a2f9..67c56df 100644
--- 
a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/PartitionCommitPolicy.java
+++ 
b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/filesystem/PartitionCommitPolicy.java
@@ -111,7 +111,7 @@ public interface PartitionCommitPolicy {
                                             | IllegalAccessException
                                             | InstantiationException e) {
                                         throw new RuntimeException(
-                                                "Can not new instance for 
custom class from "
+                                                "Can not create new instance 
for custom class from "
                                                         + customClass,
                                                 e);
                                     }

Reply via email to