danny0405 commented on a change in pull request #4153:
URL: https://github.com/apache/hudi/pull/4153#discussion_r762382771



##########
File path: 
hudi-flink/src/test/java/org/apache/hudi/table/catalog/TestHoodieCatalog.java
##########
@@ -151,57 +174,76 @@ public void testCreateDatabaseWithOptions() {
 
     assertThrows(
         CatalogException.class,
-        () -> catalog.createDatabase("db1", new CatalogDatabaseImpl(options, 
null), true),
-        "Hudi catalog doesn't support to create database with options."
-    );
+        () -> catalog.createDatabase("db1", new CatalogDatabaseImpl(options, 
null), true));
   }
 
   @Test
-  public void testTableRelatedMethod() throws Exception {
-    Map<String, String> expectedOptions = new HashMap<>();
-    expectedOptions.put(FlinkOptions.TABLE_TYPE.key(), 
FlinkOptions.TABLE_TYPE_MERGE_ON_READ);
-    expectedOptions.put(FlinkOptions.INDEX_GLOBAL_ENABLED.key(), "false");
-    expectedOptions.put(FlinkOptions.PRE_COMBINE.key(), "true");
-
-    ResolvedCatalogTable expectedCatalogTable = new ResolvedCatalogTable(
-        CatalogTable.of(
-            Schema.newBuilder().fromResolvedSchema(createTableSchema).build(),
-            "test",
-            Arrays.asList("partition"),
-            expectedOptions),
-        createTableSchema
-    );
-
+  public void testCreateTable() throws Exception {
     ObjectPath tablePath = new ObjectPath(TEST_DEFAULT_DATABASE, "tb1");
     // test create table
-    catalog.createTable(tablePath, expectedCatalogTable, true);
+    catalog.createTable(tablePath, EXPECTED_CATALOG_TABLE, true);
 

Review comment:
       add a test case: create table with existing name.

##########
File path: 
hudi-flink/src/test/java/org/apache/hudi/table/catalog/TestHoodieCatalog.java
##########
@@ -151,57 +174,76 @@ public void testCreateDatabaseWithOptions() {
 
     assertThrows(
         CatalogException.class,
-        () -> catalog.createDatabase("db1", new CatalogDatabaseImpl(options, 
null), true),
-        "Hudi catalog doesn't support to create database with options."
-    );
+        () -> catalog.createDatabase("db1", new CatalogDatabaseImpl(options, 
null), true));
   }
 
   @Test
-  public void testTableRelatedMethod() throws Exception {
-    Map<String, String> expectedOptions = new HashMap<>();
-    expectedOptions.put(FlinkOptions.TABLE_TYPE.key(), 
FlinkOptions.TABLE_TYPE_MERGE_ON_READ);
-    expectedOptions.put(FlinkOptions.INDEX_GLOBAL_ENABLED.key(), "false");
-    expectedOptions.put(FlinkOptions.PRE_COMBINE.key(), "true");
-
-    ResolvedCatalogTable expectedCatalogTable = new ResolvedCatalogTable(
-        CatalogTable.of(
-            Schema.newBuilder().fromResolvedSchema(createTableSchema).build(),
-            "test",
-            Arrays.asList("partition"),
-            expectedOptions),
-        createTableSchema
-    );
-
+  public void testCreateTable() throws Exception {
     ObjectPath tablePath = new ObjectPath(TEST_DEFAULT_DATABASE, "tb1");
     // test create table
-    catalog.createTable(tablePath, expectedCatalogTable, true);
+    catalog.createTable(tablePath, EXPECTED_CATALOG_TABLE, true);
 
     // test table exist
     assertTrue(catalog.tableExists(tablePath));
+  }
+
+  @Test
+  public void testListTable() throws Exception {
+    ObjectPath tablePath1 = new ObjectPath(TEST_DEFAULT_DATABASE, "tb1");
+    ObjectPath tablePath2 = new ObjectPath(TEST_DEFAULT_DATABASE, "tb2");
+
+    // create table
+    catalog.createTable(tablePath1, EXPECTED_CATALOG_TABLE, true);
+    catalog.createTable(tablePath2, EXPECTED_CATALOG_TABLE, true);
 
     // test list table
-    
assertTrue(catalog.listTables(TEST_DEFAULT_DATABASE).contains(tablePath.getObjectName()));
+    List<String> tables = catalog.listTables(TEST_DEFAULT_DATABASE);
+    assertTrue(tables.contains(tablePath1.getObjectName()));
+    assertTrue(tables.contains(tablePath2.getObjectName()));

Review comment:
       add a test case: list table with non-exist database.

##########
File path: 
hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieCatalog.java
##########
@@ -0,0 +1,508 @@
+/*
+ * 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.hudi.table.catalog;
+
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.TableSchemaResolver;
+import org.apache.hudi.configuration.FlinkOptions;
+import org.apache.hudi.util.AvroSchemaConverter;
+import org.apache.hudi.util.StreamerUtil;
+
+import org.apache.avro.Schema;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.table.catalog.AbstractCatalog;
+import org.apache.flink.table.catalog.CatalogBaseTable;
+import org.apache.flink.table.catalog.CatalogDatabase;
+import org.apache.flink.table.catalog.CatalogDatabaseImpl;
+import org.apache.flink.table.catalog.CatalogFunction;
+import org.apache.flink.table.catalog.CatalogPartition;
+import org.apache.flink.table.catalog.CatalogPartitionSpec;
+import org.apache.flink.table.catalog.CatalogTable;
+import org.apache.flink.table.catalog.CatalogView;
+import org.apache.flink.table.catalog.ObjectPath;
+import org.apache.flink.table.catalog.ResolvedCatalogTable;
+import org.apache.flink.table.catalog.ResolvedSchema;
+import org.apache.flink.table.catalog.exceptions.CatalogException;
+import org.apache.flink.table.catalog.exceptions.DatabaseAlreadyExistException;
+import org.apache.flink.table.catalog.exceptions.DatabaseNotEmptyException;
+import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException;
+import org.apache.flink.table.catalog.exceptions.FunctionAlreadyExistException;
+import org.apache.flink.table.catalog.exceptions.FunctionNotExistException;
+import 
org.apache.flink.table.catalog.exceptions.PartitionAlreadyExistsException;
+import org.apache.flink.table.catalog.exceptions.PartitionNotExistException;
+import org.apache.flink.table.catalog.exceptions.PartitionSpecInvalidException;
+import org.apache.flink.table.catalog.exceptions.TableAlreadyExistException;
+import org.apache.flink.table.catalog.exceptions.TableNotExistException;
+import org.apache.flink.table.catalog.exceptions.TableNotPartitionedException;
+import org.apache.flink.table.catalog.exceptions.TablePartitionedException;
+import org.apache.flink.table.catalog.stats.CatalogColumnStatistics;
+import org.apache.flink.table.catalog.stats.CatalogTableStatistics;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.util.CollectionUtil;
+import org.apache.flink.util.StringUtils;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.hudi.table.catalog.CatalogOptions.CATALOG_PATH;
+import static org.apache.hudi.table.catalog.CatalogOptions.DEFAULT_DATABASE;
+
+/**
+ * Catalog that can set up common options for underneath table.
+ */
+public class HoodieCatalog extends AbstractCatalog {
+  private static final Logger LOG = 
LoggerFactory.getLogger(HoodieCatalog.class);
+
+  private final org.apache.hadoop.conf.Configuration hadoopConf;
+  private final String catalogPathStr;
+  private final Map<String, String> tableCommonOptions;
+
+  private Path catalogPath;
+  private FileSystem fs;
+
+  public HoodieCatalog(String name, Configuration options) {
+    super(name, options.get(DEFAULT_DATABASE));
+    this.catalogPathStr = options.get(CATALOG_PATH);
+    this.hadoopConf = StreamerUtil.getHadoopConf();
+    this.tableCommonOptions = CatalogOptions.tableCommonOptions(options);
+  }
+
+  @Override
+  public void open() throws CatalogException {
+    fs = FSUtils.getFs(catalogPathStr, hadoopConf);
+    catalogPath = new Path(catalogPathStr);
+    try {
+      if (!fs.exists(catalogPath)) {
+        throw new CatalogException(String.format("Catalog %s path %s is not 
exists.", getName(), catalogPathStr));
+      }

Review comment:
       `exists` => `exist`.

##########
File path: 
hudi-flink/src/main/java/org/apache/hudi/table/catalog/HoodieCatalog.java
##########
@@ -103,10 +103,10 @@ public void open() throws CatalogException {
     catalogPath = new Path(catalogPathStr);
     try {
       if (!fs.exists(catalogPath)) {
-        throw new CatalogException(String.format("Catalog %s path %s is not 
exists.", getName(), catalogPathStr));
+        throw new CatalogException(String.format("Catalog %s path %s does not 
exists.", getName(), catalogPathStr));
       }
     } catch (IOException e) {
-      throw new CatalogException(String.format("Check catalog path %s exists 
occur error.", catalogPathStr), e);
+      throw new CatalogException(String.format("Check catalog path %s exists 
occur exception.", catalogPathStr), e);
     }

Review comment:
       If you want to unify the error message format, i would propose this 
style:
   xxxing xxx error(or exception), for e.g, we can refactor this error message 
to:
   `Checking whether catalog path %s exists exception."` 




-- 
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: commits-unsubscr...@hudi.apache.org

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


Reply via email to