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

lzljs3620320 pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink-table-store.git


The following commit(s) were added to refs/heads/master by this push:
     new 36c365d  [FLINK-26562] Introduce table.path option for FileStoreOptions
36c365d is described below

commit 36c365d48a6d5a7f009b1f15d9f27565f29bf843
Author: Jane Chan <[email protected]>
AuthorDate: Thu Mar 10 12:58:41 2022 +0800

    [FLINK-26562] Introduce table.path option for FileStoreOptions
    
    This closes #39
---
 .../apache/flink/table/store/connector/FileStoreITCase.java  |  6 +++---
 .../org/apache/flink/table/store/file/FileStoreOptions.java  | 12 ++++++++++--
 .../org/apache/flink/table/store/file/TestFileStore.java     |  4 ++--
 3 files changed, 15 insertions(+), 7 deletions(-)

diff --git 
a/flink-table-store-connector/src/test/java/org/apache/flink/table/store/connector/FileStoreITCase.java
 
b/flink-table-store-connector/src/test/java/org/apache/flink/table/store/connector/FileStoreITCase.java
index f671476..221e85c 100644
--- 
a/flink-table-store-connector/src/test/java/org/apache/flink/table/store/connector/FileStoreITCase.java
+++ 
b/flink-table-store-connector/src/test/java/org/apache/flink/table/store/connector/FileStoreITCase.java
@@ -64,7 +64,7 @@ import java.util.Map;
 
 import static org.apache.flink.table.store.file.FileStoreOptions.BUCKET;
 import static org.apache.flink.table.store.file.FileStoreOptions.FILE_FORMAT;
-import static org.apache.flink.table.store.file.FileStoreOptions.FILE_PATH;
+import static org.apache.flink.table.store.file.FileStoreOptions.TABLE_PATH;
 import static org.assertj.core.api.Assertions.assertThat;
 
 /** ITCase for {@link FileStoreSource} and {@link StoreSink}. */
@@ -213,10 +213,10 @@ public class FileStoreITCase extends AbstractTestBase {
         Configuration options = new Configuration();
         options.set(BUCKET, NUM_BUCKET);
         if (noFail) {
-            options.set(FILE_PATH, folder.toURI().toString());
+            options.set(TABLE_PATH, folder.toURI().toString());
         } else {
             FailingAtomicRenameFileSystem.get().reset(3, 100);
-            options.set(FILE_PATH, 
FailingAtomicRenameFileSystem.getFailingPath(folder.getPath()));
+            options.set(TABLE_PATH, 
FailingAtomicRenameFileSystem.getFailingPath(folder.getPath()));
         }
         options.set(FILE_FORMAT, "avro");
         return options;
diff --git 
a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/FileStoreOptions.java
 
b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/FileStoreOptions.java
index bf1b0e1..ddceee8 100644
--- 
a/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/FileStoreOptions.java
+++ 
b/flink-table-store-core/src/main/java/org/apache/flink/table/store/file/FileStoreOptions.java
@@ -18,6 +18,7 @@
 
 package org.apache.flink.table.store.file;
 
+import org.apache.flink.annotation.Internal;
 import org.apache.flink.configuration.ConfigOption;
 import org.apache.flink.configuration.ConfigOptions;
 import org.apache.flink.configuration.Configuration;
@@ -48,7 +49,14 @@ public class FileStoreOptions implements Serializable {
             ConfigOptions.key("file.path")
                     .stringType()
                     .noDefaultValue()
-                    .withDescription("The file path of the table store in the 
filesystem.");
+                    .withDescription("The root file path of the table store in 
the filesystem.");
+
+    @Internal
+    public static final ConfigOption<String> TABLE_PATH =
+            ConfigOptions.key("table.path")
+                    .stringType()
+                    .noDefaultValue()
+                    .withDescription("The table file path of the table store 
in the filesystem.");
 
     public static final ConfigOption<String> FILE_FORMAT =
             ConfigOptions.key("file.format")
@@ -122,7 +130,7 @@ public class FileStoreOptions implements Serializable {
     }
 
     public Path path() {
-        return new Path(options.get(FILE_PATH));
+        return new Path(options.get(TABLE_PATH));
     }
 
     public FileFormat fileFormat() {
diff --git 
a/flink-table-store-core/src/test/java/org/apache/flink/table/store/file/TestFileStore.java
 
b/flink-table-store-core/src/test/java/org/apache/flink/table/store/file/TestFileStore.java
index 0057435..58adb76 100644
--- 
a/flink-table-store-core/src/test/java/org/apache/flink/table/store/file/TestFileStore.java
+++ 
b/flink-table-store-core/src/test/java/org/apache/flink/table/store/file/TestFileStore.java
@@ -95,7 +95,7 @@ public class TestFileStore extends FileStoreImpl {
 
         conf.set(FileStoreOptions.FILE_FORMAT, format);
         conf.set(FileStoreOptions.MANIFEST_FORMAT, format);
-        conf.set(FileStoreOptions.FILE_PATH, root);
+        conf.set(FileStoreOptions.TABLE_PATH, root);
         conf.set(FileStoreOptions.BUCKET, numBuckets);
 
         return new TestFileStore(conf, partitionType, keyType, valueType, 
accumulator);
@@ -108,7 +108,7 @@ public class TestFileStore extends FileStoreImpl {
             RowType valueType,
             Accumulator accumulator) {
         super(conf, UUID.randomUUID().toString(), partitionType, keyType, 
valueType, accumulator);
-        this.root = conf.getString(FileStoreOptions.FILE_PATH);
+        this.root = conf.getString(FileStoreOptions.TABLE_PATH);
         this.keySerializer = new RowDataSerializer(keyType);
         this.valueSerializer = new RowDataSerializer(valueType);
     }

Reply via email to