wuchong commented on a change in pull request #14536:
URL: https://github.com/apache/flink/pull/14536#discussion_r557022428



##########
File path: 
flink-connectors/flink-connector-hbase-base/src/main/java/org/apache/flink/connector/hbase/options/HBaseOptions.java
##########
@@ -0,0 +1,263 @@
+/*
+ * 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.connector.hbase.options;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigOptions;
+import org.apache.flink.configuration.MemorySize;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.connector.hbase.util.HBaseConfigurationUtil;
+import org.apache.flink.connector.hbase.util.HBaseTableSchema;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.descriptors.DescriptorProperties;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
+
+import java.io.Serializable;
+import java.time.Duration;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Properties;
+
+import static 
org.apache.flink.table.descriptors.AbstractHBaseValidator.CONNECTOR_PROPERTIES;
+import static 
org.apache.flink.table.descriptors.AbstractHBaseValidator.CONNECTOR_ZK_NODE_PARENT;
+import static 
org.apache.flink.table.descriptors.AbstractHBaseValidator.CONNECTOR_ZK_QUORUM;
+import static org.apache.flink.table.factories.FactoryUtil.SINK_PARALLELISM;
+
+/** Common Options for HBase. */
+@Internal
+public class HBaseOptions implements Serializable {
+
+    public static final ConfigOption<String> TABLE_NAME =
+            ConfigOptions.key("table-name")
+                    .stringType()
+                    .noDefaultValue()
+                    .withDescription("The name of HBase table to connect.");
+
+    public static final ConfigOption<String> ZOOKEEPER_QUORUM =
+            ConfigOptions.key("zookeeper.quorum")
+                    .stringType()
+                    .noDefaultValue()
+                    .withDescription("The HBase Zookeeper quorum.");
+
+    public static final ConfigOption<String> ZOOKEEPER_ZNODE_PARENT =
+            ConfigOptions.key("zookeeper.znode.parent")
+                    .stringType()
+                    .defaultValue("/hbase")
+                    .withDescription("The root dir in Zookeeper for HBase 
cluster.");
+
+    public static final ConfigOption<String> NULL_STRING_LITERAL =
+            ConfigOptions.key("null-string-literal")
+                    .stringType()
+                    .defaultValue("null")
+                    .withDescription(
+                            "Representation for null values for string fields. 
HBase source and "
+                                    + "sink encodes/decodes empty bytes as 
null values for all types except string type.");
+
+    public static final ConfigOption<MemorySize> SINK_BUFFER_FLUSH_MAX_SIZE =
+            ConfigOptions.key("sink.buffer-flush.max-size")
+                    .memoryType()
+                    .defaultValue(MemorySize.parse("2mb"))
+                    .withDescription(
+                            "Writing option, maximum size in memory of 
buffered rows for each "
+                                    + "writing request. This can improve 
performance for writing data to HBase database, "
+                                    + "but may increase the latency. Can be 
set to '0' to disable it. ");
+
+    public static final ConfigOption<Integer> SINK_BUFFER_FLUSH_MAX_ROWS =
+            ConfigOptions.key("sink.buffer-flush.max-rows")
+                    .intType()
+                    .defaultValue(1000)
+                    .withDescription(
+                            "Writing option, maximum number of rows to buffer 
for each writing request. "
+                                    + "This can improve performance for 
writing data to HBase database, but may increase the latency. "
+                                    + "Can be set to '0' to disable it.");
+
+    public static final ConfigOption<Duration> SINK_BUFFER_FLUSH_INTERVAL =
+            ConfigOptions.key("sink.buffer-flush.interval")
+                    .durationType()
+                    .defaultValue(Duration.ofSeconds(1))
+                    .withDescription(
+                            "Writing option, the interval to flush any 
buffered rows. "
+                                    + "This can improve performance for 
writing data to HBase database, but may increase the latency. "
+                                    + "Can be set to '0' to disable it. Note, 
both 'sink.buffer-flush.max-size' and 'sink.buffer-flush.max-rows' "
+                                    + "can be set to '0' with the flush 
interval set allowing for complete async processing of buffered actions.");
+
+    // Prefix for HBase specific properties.
+    public static final String PROPERTIES_PREFIX = "properties.";
+
+    // 
--------------------------------------------------------------------------------------------
+    // Validation
+    // 
--------------------------------------------------------------------------------------------
+
+    /**
+     * Checks that the HBase table have row key defined. A row key is defined 
as an atomic type, and
+     * column families and qualifiers are defined as ROW type. There shouldn't 
be multiple atomic
+     * type columns in the schema. The PRIMARY KEY constraint is optional, if 
exist, the primary key
+     * constraint must be defined on the single row key field.
+     */
+    public static void validatePrimaryKey(TableSchema schema) {
+        HBaseTableSchema hbaseSchema = 
HBaseTableSchema.fromTableSchema(schema);
+        if (!hbaseSchema.getRowKeyName().isPresent()) {
+            throw new IllegalArgumentException(
+                    "HBase table requires to define a row key field. "
+                            + "A row key field is defined as an atomic type, "
+                            + "column families and qualifiers are defined as 
ROW type.");
+        }
+        schema.getPrimaryKey()
+                .ifPresent(
+                        k -> {
+                            if (k.getColumns().size() > 1) {
+                                throw new IllegalArgumentException(
+                                        "HBase table doesn't support a primary 
Key on multiple columns. "
+                                                + "The primary key of HBase 
table must be defined on row key field.");
+                            }
+                            if 
(!hbaseSchema.getRowKeyName().get().equals(k.getColumns().get(0))) {
+                                throw new IllegalArgumentException(
+                                        "Primary key of HBase table must be 
defined on the row key field. "
+                                                + "A row key field is defined 
as an atomic type, "
+                                                + "column families and 
qualifiers are defined as ROW type.");
+                            }
+                        });
+    }
+
+    /**
+     * Options for validate HBase source table , The reason for separate 
source and sink is to
+     * facilitate future expansion of parameters.
+     *
+     * @param tableOptions
+     */
+    public static void validateTableSourceOptions(ReadableConfig tableOptions) 
{
+        validateTableName(tableOptions);
+        validateZookeeperQuorum(tableOptions);
+    }
+
+    /**
+     * Options for validate HBase sink table.
+     *
+     * @param tableOptions
+     */
+    public static void validateTableSinkOptions(ReadableConfig tableOptions) {
+        validateTableName(tableOptions);
+        validateZookeeperQuorum(tableOptions);
+    }
+
+    private static void validateTableName(ReadableConfig tableOptions) {
+        Optional<String> tableName = tableOptions.getOptional(TABLE_NAME);
+        if (!tableName.isPresent()) {
+            throw new ValidationException("Option 'table-name' must be set.");
+        }
+    }
+
+    private static void validateZookeeperQuorum(ReadableConfig tableOptions) {
+        Optional<String> zookeeperQuorum = 
tableOptions.getOptional(ZOOKEEPER_QUORUM);
+        if (!zookeeperQuorum.isPresent()) {
+            throw new ValidationException("Option 'zookeeper.quorum' must be 
set.");
+        }
+    }
+
+    public static String getTableName(ReadableConfig tableOptions) {
+        return tableOptions.getOptional(TABLE_NAME).orElse(null);
+    }
+
+    private static String getZookeeperQuorum(ReadableConfig tableOptions) {
+        return tableOptions.getOptional(ZOOKEEPER_QUORUM).orElse(null);
+    }
+
+    private static String getZookeeperNodeParent(ReadableConfig tableOptions) {
+        return tableOptions.get(ZOOKEEPER_ZNODE_PARENT);
+    }
+
+    public static String getNullStringLiteral(ReadableConfig tableOptions) {
+        return tableOptions.get(NULL_STRING_LITERAL);
+    }
+
+    public static HBaseWriteOptions getHBaseWriteOptions(ReadableConfig 
tableOptions) {
+        HBaseWriteOptions.Builder builder = HBaseWriteOptions.builder();
+        builder.setBufferFlushIntervalMillis(
+                tableOptions.get(SINK_BUFFER_FLUSH_INTERVAL).toMillis());
+        
builder.setBufferFlushMaxRows(tableOptions.get(SINK_BUFFER_FLUSH_MAX_ROWS));
+        builder.setBufferFlushMaxSizeInBytes(
+                tableOptions.get(SINK_BUFFER_FLUSH_MAX_SIZE).getBytes());
+        
builder.setParallelism(tableOptions.getOptional(SINK_PARALLELISM).orElse(null));
+        return builder.build();
+    }
+
+    /**
+     * config HBase Configuration.
+     *
+     * @param tableOptions table option
+     * @param options properties option
+     * @return
+     */
+    public static Configuration getHBaseConfiguration(
+            ReadableConfig tableOptions, Map<String, String> options) {
+        // create default configuration from current runtime env 
(`hbase-site.xml` in classpath)
+        // first,
+        Configuration hbaseClientConf = 
HBaseConfigurationUtil.getHBaseConfiguration();
+        hbaseClientConf.set(HConstants.ZOOKEEPER_QUORUM, 
getZookeeperQuorum(tableOptions));
+        hbaseClientConf.set(
+                HConstants.ZOOKEEPER_ZNODE_PARENT, 
getZookeeperNodeParent(tableOptions));
+        // add HBase properties
+        final Properties properties = getHBaseClientProperties(options);
+        properties.forEach((k, v) -> hbaseClientConf.set(k.toString(), 
v.toString()));
+        return hbaseClientConf;
+    }
+
+    // get HBase table properties
+    private static Properties getHBaseClientProperties(Map<String, String> 
tableOptions) {
+        final Properties hbaseProperties = new Properties();
+
+        if (containsHBaseClientProperties(tableOptions)) {
+            tableOptions.keySet().stream()
+                    .filter(key -> key.startsWith(PROPERTIES_PREFIX))
+                    .forEach(
+                            key -> {
+                                final String value = tableOptions.get(key);
+                                final String subKey = 
key.substring((PROPERTIES_PREFIX).length());
+                                hbaseProperties.put(subKey, value);
+                            });
+        }
+        return hbaseProperties;
+    }
+
+    /** Returns wether the table options contains HBase client properties or 
not. 'properties'. */
+    private static boolean containsHBaseClientProperties(Map<String, String> 
tableOptions) {
+        return tableOptions.keySet().stream().anyMatch(k -> 
k.startsWith(PROPERTIES_PREFIX));
+    }
+
+    public static Configuration getHBaseConf(DescriptorProperties 
descriptorProperties) {

Review comment:
       This is only used for legacy connector, please put this still in the 
legacy `HBase1TableFactory`.  Then we can easily remove them entirely once we 
don't want support legacy connector. 

##########
File path: 
flink-connectors/flink-connector-hbase-base/src/main/java/org/apache/flink/connector/hbase/options/HBaseOptions.java
##########
@@ -0,0 +1,263 @@
+/*
+ * 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.connector.hbase.options;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigOptions;
+import org.apache.flink.configuration.MemorySize;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.connector.hbase.util.HBaseConfigurationUtil;
+import org.apache.flink.connector.hbase.util.HBaseTableSchema;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.descriptors.DescriptorProperties;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
+
+import java.io.Serializable;
+import java.time.Duration;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Properties;
+
+import static 
org.apache.flink.table.descriptors.AbstractHBaseValidator.CONNECTOR_PROPERTIES;
+import static 
org.apache.flink.table.descriptors.AbstractHBaseValidator.CONNECTOR_ZK_NODE_PARENT;
+import static 
org.apache.flink.table.descriptors.AbstractHBaseValidator.CONNECTOR_ZK_QUORUM;
+import static org.apache.flink.table.factories.FactoryUtil.SINK_PARALLELISM;
+
+/** Common Options for HBase. */
+@Internal
+public class HBaseOptions implements Serializable {
+
+    public static final ConfigOption<String> TABLE_NAME =
+            ConfigOptions.key("table-name")
+                    .stringType()
+                    .noDefaultValue()
+                    .withDescription("The name of HBase table to connect.");
+
+    public static final ConfigOption<String> ZOOKEEPER_QUORUM =
+            ConfigOptions.key("zookeeper.quorum")
+                    .stringType()
+                    .noDefaultValue()
+                    .withDescription("The HBase Zookeeper quorum.");
+
+    public static final ConfigOption<String> ZOOKEEPER_ZNODE_PARENT =
+            ConfigOptions.key("zookeeper.znode.parent")
+                    .stringType()
+                    .defaultValue("/hbase")
+                    .withDescription("The root dir in Zookeeper for HBase 
cluster.");
+
+    public static final ConfigOption<String> NULL_STRING_LITERAL =
+            ConfigOptions.key("null-string-literal")
+                    .stringType()
+                    .defaultValue("null")
+                    .withDescription(
+                            "Representation for null values for string fields. 
HBase source and "
+                                    + "sink encodes/decodes empty bytes as 
null values for all types except string type.");
+
+    public static final ConfigOption<MemorySize> SINK_BUFFER_FLUSH_MAX_SIZE =
+            ConfigOptions.key("sink.buffer-flush.max-size")
+                    .memoryType()
+                    .defaultValue(MemorySize.parse("2mb"))
+                    .withDescription(
+                            "Writing option, maximum size in memory of 
buffered rows for each "
+                                    + "writing request. This can improve 
performance for writing data to HBase database, "
+                                    + "but may increase the latency. Can be 
set to '0' to disable it. ");
+
+    public static final ConfigOption<Integer> SINK_BUFFER_FLUSH_MAX_ROWS =
+            ConfigOptions.key("sink.buffer-flush.max-rows")
+                    .intType()
+                    .defaultValue(1000)
+                    .withDescription(
+                            "Writing option, maximum number of rows to buffer 
for each writing request. "
+                                    + "This can improve performance for 
writing data to HBase database, but may increase the latency. "
+                                    + "Can be set to '0' to disable it.");
+
+    public static final ConfigOption<Duration> SINK_BUFFER_FLUSH_INTERVAL =
+            ConfigOptions.key("sink.buffer-flush.interval")
+                    .durationType()
+                    .defaultValue(Duration.ofSeconds(1))
+                    .withDescription(
+                            "Writing option, the interval to flush any 
buffered rows. "
+                                    + "This can improve performance for 
writing data to HBase database, but may increase the latency. "
+                                    + "Can be set to '0' to disable it. Note, 
both 'sink.buffer-flush.max-size' and 'sink.buffer-flush.max-rows' "
+                                    + "can be set to '0' with the flush 
interval set allowing for complete async processing of buffered actions.");
+
+    // Prefix for HBase specific properties.
+    public static final String PROPERTIES_PREFIX = "properties.";
+
+    // 
--------------------------------------------------------------------------------------------
+    // Validation
+    // 
--------------------------------------------------------------------------------------------
+
+    /**
+     * Checks that the HBase table have row key defined. A row key is defined 
as an atomic type, and
+     * column families and qualifiers are defined as ROW type. There shouldn't 
be multiple atomic
+     * type columns in the schema. The PRIMARY KEY constraint is optional, if 
exist, the primary key
+     * constraint must be defined on the single row key field.
+     */
+    public static void validatePrimaryKey(TableSchema schema) {
+        HBaseTableSchema hbaseSchema = 
HBaseTableSchema.fromTableSchema(schema);
+        if (!hbaseSchema.getRowKeyName().isPresent()) {
+            throw new IllegalArgumentException(
+                    "HBase table requires to define a row key field. "
+                            + "A row key field is defined as an atomic type, "
+                            + "column families and qualifiers are defined as 
ROW type.");
+        }
+        schema.getPrimaryKey()
+                .ifPresent(
+                        k -> {
+                            if (k.getColumns().size() > 1) {
+                                throw new IllegalArgumentException(
+                                        "HBase table doesn't support a primary 
Key on multiple columns. "
+                                                + "The primary key of HBase 
table must be defined on row key field.");
+                            }
+                            if 
(!hbaseSchema.getRowKeyName().get().equals(k.getColumns().get(0))) {
+                                throw new IllegalArgumentException(
+                                        "Primary key of HBase table must be 
defined on the row key field. "
+                                                + "A row key field is defined 
as an atomic type, "
+                                                + "column families and 
qualifiers are defined as ROW type.");
+                            }
+                        });
+    }
+
+    /**
+     * Options for validate HBase source table , The reason for separate 
source and sink is to
+     * facilitate future expansion of parameters.
+     *
+     * @param tableOptions
+     */
+    public static void validateTableSourceOptions(ReadableConfig tableOptions) 
{
+        validateTableName(tableOptions);
+        validateZookeeperQuorum(tableOptions);
+    }
+
+    /**
+     * Options for validate HBase sink table.
+     *
+     * @param tableOptions
+     */
+    public static void validateTableSinkOptions(ReadableConfig tableOptions) {
+        validateTableName(tableOptions);
+        validateZookeeperQuorum(tableOptions);
+    }
+
+    private static void validateTableName(ReadableConfig tableOptions) {
+        Optional<String> tableName = tableOptions.getOptional(TABLE_NAME);
+        if (!tableName.isPresent()) {
+            throw new ValidationException("Option 'table-name' must be set.");
+        }
+    }
+
+    private static void validateZookeeperQuorum(ReadableConfig tableOptions) {
+        Optional<String> zookeeperQuorum = 
tableOptions.getOptional(ZOOKEEPER_QUORUM);
+        if (!zookeeperQuorum.isPresent()) {
+            throw new ValidationException("Option 'zookeeper.quorum' must be 
set.");
+        }
+    }
+
+    public static String getTableName(ReadableConfig tableOptions) {
+        return tableOptions.getOptional(TABLE_NAME).orElse(null);
+    }
+
+    private static String getZookeeperQuorum(ReadableConfig tableOptions) {
+        return tableOptions.getOptional(ZOOKEEPER_QUORUM).orElse(null);

Review comment:
       Can be simplified to `tableOptions.get(ZOOKEEPER_QUORUM)`, it is a 
required option. We have checked it must be defined before we get the value in 
`helper.validateExcept(PROPERTIES_PREFIX)`. 
   
   The same to `TABLE_NAME`.
   
   Actually, I think they are so simple and don't need to wrap a method. 

##########
File path: 
flink-connectors/flink-connector-hbase-1.4/src/test/java/org/apache/flink/connector/hbase1/HBaseDynamicTableFactoryTest.java
##########
@@ -183,18 +181,6 @@ public void testTableSinkFactory() {
                 new DataType[] {DECIMAL(10, 3), TIMESTAMP(3), DATE(), TIME()},
                 hbaseSchema.getQualifierDataTypes("f4"));
 
-        Properties properties = new Properties();
-        properties.setProperty("hbase.security.authentication", "kerberos");
-        HBaseOptions expectedHBaseOptions =
-                HBaseOptions.builder()
-                        .setTableName("testHBastTable")
-                        .setZkQuorum("localhost:2181")
-                        .setZkNodeParent("/flink")
-                        .setHbaseProperties(properties)
-                        .build();
-        HBaseOptions actualHBaseOptions = hbaseSink.getHBaseOptions();
-        assertEquals(expectedHBaseOptions, actualHBaseOptions);

Review comment:
       We should add another way to test the configurations. For example : we 
can get the Hadoop Configruation and convert it into an Map via 
`Configruation.iterator()`, the we can use `assertEquals` to verify the 
configs. 
   
   We should also remember to verify table name separately, because it is not 
in the Hadoop Configruation now. 

##########
File path: 
flink-connectors/flink-connector-hbase-base/src/main/java/org/apache/flink/connector/hbase/options/HBaseOptions.java
##########
@@ -0,0 +1,263 @@
+/*
+ * 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.connector.hbase.options;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigOptions;
+import org.apache.flink.configuration.MemorySize;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.connector.hbase.util.HBaseConfigurationUtil;
+import org.apache.flink.connector.hbase.util.HBaseTableSchema;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.descriptors.DescriptorProperties;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
+
+import java.io.Serializable;
+import java.time.Duration;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Properties;
+
+import static 
org.apache.flink.table.descriptors.AbstractHBaseValidator.CONNECTOR_PROPERTIES;
+import static 
org.apache.flink.table.descriptors.AbstractHBaseValidator.CONNECTOR_ZK_NODE_PARENT;
+import static 
org.apache.flink.table.descriptors.AbstractHBaseValidator.CONNECTOR_ZK_QUORUM;
+import static org.apache.flink.table.factories.FactoryUtil.SINK_PARALLELISM;
+
+/** Common Options for HBase. */
+@Internal
+public class HBaseOptions implements Serializable {
+
+    public static final ConfigOption<String> TABLE_NAME =
+            ConfigOptions.key("table-name")
+                    .stringType()
+                    .noDefaultValue()
+                    .withDescription("The name of HBase table to connect.");
+
+    public static final ConfigOption<String> ZOOKEEPER_QUORUM =
+            ConfigOptions.key("zookeeper.quorum")
+                    .stringType()
+                    .noDefaultValue()
+                    .withDescription("The HBase Zookeeper quorum.");
+
+    public static final ConfigOption<String> ZOOKEEPER_ZNODE_PARENT =
+            ConfigOptions.key("zookeeper.znode.parent")
+                    .stringType()
+                    .defaultValue("/hbase")
+                    .withDescription("The root dir in Zookeeper for HBase 
cluster.");
+
+    public static final ConfigOption<String> NULL_STRING_LITERAL =
+            ConfigOptions.key("null-string-literal")
+                    .stringType()
+                    .defaultValue("null")
+                    .withDescription(
+                            "Representation for null values for string fields. 
HBase source and "
+                                    + "sink encodes/decodes empty bytes as 
null values for all types except string type.");
+
+    public static final ConfigOption<MemorySize> SINK_BUFFER_FLUSH_MAX_SIZE =
+            ConfigOptions.key("sink.buffer-flush.max-size")
+                    .memoryType()
+                    .defaultValue(MemorySize.parse("2mb"))
+                    .withDescription(
+                            "Writing option, maximum size in memory of 
buffered rows for each "
+                                    + "writing request. This can improve 
performance for writing data to HBase database, "
+                                    + "but may increase the latency. Can be 
set to '0' to disable it. ");
+
+    public static final ConfigOption<Integer> SINK_BUFFER_FLUSH_MAX_ROWS =
+            ConfigOptions.key("sink.buffer-flush.max-rows")
+                    .intType()
+                    .defaultValue(1000)
+                    .withDescription(
+                            "Writing option, maximum number of rows to buffer 
for each writing request. "
+                                    + "This can improve performance for 
writing data to HBase database, but may increase the latency. "
+                                    + "Can be set to '0' to disable it.");
+
+    public static final ConfigOption<Duration> SINK_BUFFER_FLUSH_INTERVAL =
+            ConfigOptions.key("sink.buffer-flush.interval")
+                    .durationType()
+                    .defaultValue(Duration.ofSeconds(1))
+                    .withDescription(
+                            "Writing option, the interval to flush any 
buffered rows. "
+                                    + "This can improve performance for 
writing data to HBase database, but may increase the latency. "
+                                    + "Can be set to '0' to disable it. Note, 
both 'sink.buffer-flush.max-size' and 'sink.buffer-flush.max-rows' "
+                                    + "can be set to '0' with the flush 
interval set allowing for complete async processing of buffered actions.");
+
+    // Prefix for HBase specific properties.
+    public static final String PROPERTIES_PREFIX = "properties.";
+
+    // 
--------------------------------------------------------------------------------------------
+    // Validation
+    // 
--------------------------------------------------------------------------------------------
+
+    /**
+     * Checks that the HBase table have row key defined. A row key is defined 
as an atomic type, and
+     * column families and qualifiers are defined as ROW type. There shouldn't 
be multiple atomic
+     * type columns in the schema. The PRIMARY KEY constraint is optional, if 
exist, the primary key
+     * constraint must be defined on the single row key field.
+     */
+    public static void validatePrimaryKey(TableSchema schema) {
+        HBaseTableSchema hbaseSchema = 
HBaseTableSchema.fromTableSchema(schema);
+        if (!hbaseSchema.getRowKeyName().isPresent()) {
+            throw new IllegalArgumentException(
+                    "HBase table requires to define a row key field. "
+                            + "A row key field is defined as an atomic type, "
+                            + "column families and qualifiers are defined as 
ROW type.");
+        }
+        schema.getPrimaryKey()
+                .ifPresent(
+                        k -> {
+                            if (k.getColumns().size() > 1) {
+                                throw new IllegalArgumentException(
+                                        "HBase table doesn't support a primary 
Key on multiple columns. "
+                                                + "The primary key of HBase 
table must be defined on row key field.");
+                            }
+                            if 
(!hbaseSchema.getRowKeyName().get().equals(k.getColumns().get(0))) {
+                                throw new IllegalArgumentException(
+                                        "Primary key of HBase table must be 
defined on the row key field. "
+                                                + "A row key field is defined 
as an atomic type, "
+                                                + "column families and 
qualifiers are defined as ROW type.");
+                            }
+                        });
+    }
+
+    /**
+     * Options for validate HBase source table , The reason for separate 
source and sink is to
+     * facilitate future expansion of parameters.
+     *
+     * @param tableOptions
+     */
+    public static void validateTableSourceOptions(ReadableConfig tableOptions) 
{
+        validateTableName(tableOptions);
+        validateZookeeperQuorum(tableOptions);
+    }
+
+    /**
+     * Options for validate HBase sink table.
+     *
+     * @param tableOptions
+     */
+    public static void validateTableSinkOptions(ReadableConfig tableOptions) {
+        validateTableName(tableOptions);
+        validateZookeeperQuorum(tableOptions);
+    }
+
+    private static void validateTableName(ReadableConfig tableOptions) {
+        Optional<String> tableName = tableOptions.getOptional(TABLE_NAME);
+        if (!tableName.isPresent()) {
+            throw new ValidationException("Option 'table-name' must be set.");
+        }
+    }
+
+    private static void validateZookeeperQuorum(ReadableConfig tableOptions) {
+        Optional<String> zookeeperQuorum = 
tableOptions.getOptional(ZOOKEEPER_QUORUM);
+        if (!zookeeperQuorum.isPresent()) {
+            throw new ValidationException("Option 'zookeeper.quorum' must be 
set.");
+        }
+    }
+
+    public static String getTableName(ReadableConfig tableOptions) {
+        return tableOptions.getOptional(TABLE_NAME).orElse(null);
+    }
+
+    private static String getZookeeperQuorum(ReadableConfig tableOptions) {
+        return tableOptions.getOptional(ZOOKEEPER_QUORUM).orElse(null);
+    }
+
+    private static String getZookeeperNodeParent(ReadableConfig tableOptions) {
+        return tableOptions.get(ZOOKEEPER_ZNODE_PARENT);
+    }
+
+    public static String getNullStringLiteral(ReadableConfig tableOptions) {
+        return tableOptions.get(NULL_STRING_LITERAL);
+    }
+
+    public static HBaseWriteOptions getHBaseWriteOptions(ReadableConfig 
tableOptions) {
+        HBaseWriteOptions.Builder builder = HBaseWriteOptions.builder();
+        builder.setBufferFlushIntervalMillis(
+                tableOptions.get(SINK_BUFFER_FLUSH_INTERVAL).toMillis());
+        
builder.setBufferFlushMaxRows(tableOptions.get(SINK_BUFFER_FLUSH_MAX_ROWS));
+        builder.setBufferFlushMaxSizeInBytes(
+                tableOptions.get(SINK_BUFFER_FLUSH_MAX_SIZE).getBytes());
+        
builder.setParallelism(tableOptions.getOptional(SINK_PARALLELISM).orElse(null));
+        return builder.build();
+    }
+
+    /**
+     * config HBase Configuration.
+     *
+     * @param tableOptions table option
+     * @param options properties option
+     * @return
+     */
+    public static Configuration getHBaseConfiguration(
+            ReadableConfig tableOptions, Map<String, String> options) {

Review comment:
       It's confusing to pass in two options parameter, actually they are 
identical. We can just pass in `options` Map, and build the `ReadableConfig` by 
`Configuration.fromMap(options)`.

##########
File path: 
flink-connectors/flink-connector-hbase-base/src/main/java/org/apache/flink/connector/hbase/options/HBaseOptions.java
##########
@@ -0,0 +1,263 @@
+/*
+ * 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.connector.hbase.options;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigOptions;
+import org.apache.flink.configuration.MemorySize;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.connector.hbase.util.HBaseConfigurationUtil;
+import org.apache.flink.connector.hbase.util.HBaseTableSchema;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.descriptors.DescriptorProperties;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
+
+import java.io.Serializable;
+import java.time.Duration;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Properties;
+
+import static 
org.apache.flink.table.descriptors.AbstractHBaseValidator.CONNECTOR_PROPERTIES;
+import static 
org.apache.flink.table.descriptors.AbstractHBaseValidator.CONNECTOR_ZK_NODE_PARENT;
+import static 
org.apache.flink.table.descriptors.AbstractHBaseValidator.CONNECTOR_ZK_QUORUM;
+import static org.apache.flink.table.factories.FactoryUtil.SINK_PARALLELISM;
+
+/** Common Options for HBase. */
+@Internal
+public class HBaseOptions implements Serializable {
+
+    public static final ConfigOption<String> TABLE_NAME =
+            ConfigOptions.key("table-name")
+                    .stringType()
+                    .noDefaultValue()
+                    .withDescription("The name of HBase table to connect.");
+
+    public static final ConfigOption<String> ZOOKEEPER_QUORUM =
+            ConfigOptions.key("zookeeper.quorum")
+                    .stringType()
+                    .noDefaultValue()
+                    .withDescription("The HBase Zookeeper quorum.");
+
+    public static final ConfigOption<String> ZOOKEEPER_ZNODE_PARENT =
+            ConfigOptions.key("zookeeper.znode.parent")
+                    .stringType()
+                    .defaultValue("/hbase")
+                    .withDescription("The root dir in Zookeeper for HBase 
cluster.");
+
+    public static final ConfigOption<String> NULL_STRING_LITERAL =
+            ConfigOptions.key("null-string-literal")
+                    .stringType()
+                    .defaultValue("null")
+                    .withDescription(
+                            "Representation for null values for string fields. 
HBase source and "
+                                    + "sink encodes/decodes empty bytes as 
null values for all types except string type.");
+
+    public static final ConfigOption<MemorySize> SINK_BUFFER_FLUSH_MAX_SIZE =
+            ConfigOptions.key("sink.buffer-flush.max-size")
+                    .memoryType()
+                    .defaultValue(MemorySize.parse("2mb"))
+                    .withDescription(
+                            "Writing option, maximum size in memory of 
buffered rows for each "
+                                    + "writing request. This can improve 
performance for writing data to HBase database, "
+                                    + "but may increase the latency. Can be 
set to '0' to disable it. ");
+
+    public static final ConfigOption<Integer> SINK_BUFFER_FLUSH_MAX_ROWS =
+            ConfigOptions.key("sink.buffer-flush.max-rows")
+                    .intType()
+                    .defaultValue(1000)
+                    .withDescription(
+                            "Writing option, maximum number of rows to buffer 
for each writing request. "
+                                    + "This can improve performance for 
writing data to HBase database, but may increase the latency. "
+                                    + "Can be set to '0' to disable it.");
+
+    public static final ConfigOption<Duration> SINK_BUFFER_FLUSH_INTERVAL =
+            ConfigOptions.key("sink.buffer-flush.interval")
+                    .durationType()
+                    .defaultValue(Duration.ofSeconds(1))
+                    .withDescription(
+                            "Writing option, the interval to flush any 
buffered rows. "
+                                    + "This can improve performance for 
writing data to HBase database, but may increase the latency. "
+                                    + "Can be set to '0' to disable it. Note, 
both 'sink.buffer-flush.max-size' and 'sink.buffer-flush.max-rows' "
+                                    + "can be set to '0' with the flush 
interval set allowing for complete async processing of buffered actions.");
+
+    // Prefix for HBase specific properties.
+    public static final String PROPERTIES_PREFIX = "properties.";
+
+    // 
--------------------------------------------------------------------------------------------
+    // Validation
+    // 
--------------------------------------------------------------------------------------------
+
+    /**
+     * Checks that the HBase table have row key defined. A row key is defined 
as an atomic type, and
+     * column families and qualifiers are defined as ROW type. There shouldn't 
be multiple atomic
+     * type columns in the schema. The PRIMARY KEY constraint is optional, if 
exist, the primary key
+     * constraint must be defined on the single row key field.
+     */
+    public static void validatePrimaryKey(TableSchema schema) {
+        HBaseTableSchema hbaseSchema = 
HBaseTableSchema.fromTableSchema(schema);
+        if (!hbaseSchema.getRowKeyName().isPresent()) {
+            throw new IllegalArgumentException(
+                    "HBase table requires to define a row key field. "
+                            + "A row key field is defined as an atomic type, "
+                            + "column families and qualifiers are defined as 
ROW type.");
+        }
+        schema.getPrimaryKey()
+                .ifPresent(
+                        k -> {
+                            if (k.getColumns().size() > 1) {
+                                throw new IllegalArgumentException(
+                                        "HBase table doesn't support a primary 
Key on multiple columns. "
+                                                + "The primary key of HBase 
table must be defined on row key field.");
+                            }
+                            if 
(!hbaseSchema.getRowKeyName().get().equals(k.getColumns().get(0))) {
+                                throw new IllegalArgumentException(
+                                        "Primary key of HBase table must be 
defined on the row key field. "
+                                                + "A row key field is defined 
as an atomic type, "
+                                                + "column families and 
qualifiers are defined as ROW type.");
+                            }
+                        });
+    }
+
+    /**
+     * Options for validate HBase source table , The reason for separate 
source and sink is to
+     * facilitate future expansion of parameters.
+     *
+     * @param tableOptions
+     */
+    public static void validateTableSourceOptions(ReadableConfig tableOptions) 
{
+        validateTableName(tableOptions);
+        validateZookeeperQuorum(tableOptions);
+    }
+
+    /**
+     * Options for validate HBase sink table.
+     *
+     * @param tableOptions
+     */
+    public static void validateTableSinkOptions(ReadableConfig tableOptions) {
+        validateTableName(tableOptions);
+        validateZookeeperQuorum(tableOptions);
+    }
+
+    private static void validateTableName(ReadableConfig tableOptions) {
+        Optional<String> tableName = tableOptions.getOptional(TABLE_NAME);
+        if (!tableName.isPresent()) {
+            throw new ValidationException("Option 'table-name' must be set.");
+        }
+    }
+
+    private static void validateZookeeperQuorum(ReadableConfig tableOptions) {
+        Optional<String> zookeeperQuorum = 
tableOptions.getOptional(ZOOKEEPER_QUORUM);
+        if (!zookeeperQuorum.isPresent()) {
+            throw new ValidationException("Option 'zookeeper.quorum' must be 
set.");
+        }
+    }
+
+    public static String getTableName(ReadableConfig tableOptions) {
+        return tableOptions.getOptional(TABLE_NAME).orElse(null);
+    }
+
+    private static String getZookeeperQuorum(ReadableConfig tableOptions) {
+        return tableOptions.getOptional(ZOOKEEPER_QUORUM).orElse(null);
+    }
+
+    private static String getZookeeperNodeParent(ReadableConfig tableOptions) {
+        return tableOptions.get(ZOOKEEPER_ZNODE_PARENT);
+    }
+
+    public static String getNullStringLiteral(ReadableConfig tableOptions) {
+        return tableOptions.get(NULL_STRING_LITERAL);
+    }
+
+    public static HBaseWriteOptions getHBaseWriteOptions(ReadableConfig 
tableOptions) {
+        HBaseWriteOptions.Builder builder = HBaseWriteOptions.builder();
+        builder.setBufferFlushIntervalMillis(
+                tableOptions.get(SINK_BUFFER_FLUSH_INTERVAL).toMillis());
+        
builder.setBufferFlushMaxRows(tableOptions.get(SINK_BUFFER_FLUSH_MAX_ROWS));
+        builder.setBufferFlushMaxSizeInBytes(
+                tableOptions.get(SINK_BUFFER_FLUSH_MAX_SIZE).getBytes());
+        
builder.setParallelism(tableOptions.getOptional(SINK_PARALLELISM).orElse(null));
+        return builder.build();
+    }
+
+    /**
+     * config HBase Configuration.
+     *
+     * @param tableOptions table option
+     * @param options properties option
+     * @return

Review comment:
       Remove empty return annotation. 

##########
File path: 
flink-connectors/flink-connector-hbase-base/src/main/java/org/apache/flink/connector/hbase/options/HBaseOptions.java
##########
@@ -0,0 +1,263 @@
+/*
+ * 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.connector.hbase.options;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigOptions;
+import org.apache.flink.configuration.MemorySize;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.connector.hbase.util.HBaseConfigurationUtil;
+import org.apache.flink.connector.hbase.util.HBaseTableSchema;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.descriptors.DescriptorProperties;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
+
+import java.io.Serializable;
+import java.time.Duration;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Properties;
+
+import static 
org.apache.flink.table.descriptors.AbstractHBaseValidator.CONNECTOR_PROPERTIES;
+import static 
org.apache.flink.table.descriptors.AbstractHBaseValidator.CONNECTOR_ZK_NODE_PARENT;
+import static 
org.apache.flink.table.descriptors.AbstractHBaseValidator.CONNECTOR_ZK_QUORUM;
+import static org.apache.flink.table.factories.FactoryUtil.SINK_PARALLELISM;
+
+/** Common Options for HBase. */
+@Internal
+public class HBaseOptions implements Serializable {

Review comment:
       Don't need to implements `Serializable`.

##########
File path: 
flink-connectors/flink-connector-hbase-base/src/main/java/org/apache/flink/connector/hbase/options/HBaseOptions.java
##########
@@ -0,0 +1,263 @@
+/*
+ * 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.connector.hbase.options;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigOptions;
+import org.apache.flink.configuration.MemorySize;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.connector.hbase.util.HBaseConfigurationUtil;
+import org.apache.flink.connector.hbase.util.HBaseTableSchema;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.descriptors.DescriptorProperties;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
+
+import java.io.Serializable;
+import java.time.Duration;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Properties;
+
+import static 
org.apache.flink.table.descriptors.AbstractHBaseValidator.CONNECTOR_PROPERTIES;
+import static 
org.apache.flink.table.descriptors.AbstractHBaseValidator.CONNECTOR_ZK_NODE_PARENT;
+import static 
org.apache.flink.table.descriptors.AbstractHBaseValidator.CONNECTOR_ZK_QUORUM;
+import static org.apache.flink.table.factories.FactoryUtil.SINK_PARALLELISM;
+
+/** Common Options for HBase. */
+@Internal
+public class HBaseOptions implements Serializable {
+
+    public static final ConfigOption<String> TABLE_NAME =
+            ConfigOptions.key("table-name")
+                    .stringType()
+                    .noDefaultValue()
+                    .withDescription("The name of HBase table to connect.");
+
+    public static final ConfigOption<String> ZOOKEEPER_QUORUM =
+            ConfigOptions.key("zookeeper.quorum")
+                    .stringType()
+                    .noDefaultValue()
+                    .withDescription("The HBase Zookeeper quorum.");
+
+    public static final ConfigOption<String> ZOOKEEPER_ZNODE_PARENT =
+            ConfigOptions.key("zookeeper.znode.parent")
+                    .stringType()
+                    .defaultValue("/hbase")
+                    .withDescription("The root dir in Zookeeper for HBase 
cluster.");
+
+    public static final ConfigOption<String> NULL_STRING_LITERAL =
+            ConfigOptions.key("null-string-literal")
+                    .stringType()
+                    .defaultValue("null")
+                    .withDescription(
+                            "Representation for null values for string fields. 
HBase source and "
+                                    + "sink encodes/decodes empty bytes as 
null values for all types except string type.");
+
+    public static final ConfigOption<MemorySize> SINK_BUFFER_FLUSH_MAX_SIZE =
+            ConfigOptions.key("sink.buffer-flush.max-size")
+                    .memoryType()
+                    .defaultValue(MemorySize.parse("2mb"))
+                    .withDescription(
+                            "Writing option, maximum size in memory of 
buffered rows for each "
+                                    + "writing request. This can improve 
performance for writing data to HBase database, "
+                                    + "but may increase the latency. Can be 
set to '0' to disable it. ");
+
+    public static final ConfigOption<Integer> SINK_BUFFER_FLUSH_MAX_ROWS =
+            ConfigOptions.key("sink.buffer-flush.max-rows")
+                    .intType()
+                    .defaultValue(1000)
+                    .withDescription(
+                            "Writing option, maximum number of rows to buffer 
for each writing request. "
+                                    + "This can improve performance for 
writing data to HBase database, but may increase the latency. "
+                                    + "Can be set to '0' to disable it.");
+
+    public static final ConfigOption<Duration> SINK_BUFFER_FLUSH_INTERVAL =
+            ConfigOptions.key("sink.buffer-flush.interval")
+                    .durationType()
+                    .defaultValue(Duration.ofSeconds(1))
+                    .withDescription(
+                            "Writing option, the interval to flush any 
buffered rows. "
+                                    + "This can improve performance for 
writing data to HBase database, but may increase the latency. "
+                                    + "Can be set to '0' to disable it. Note, 
both 'sink.buffer-flush.max-size' and 'sink.buffer-flush.max-rows' "
+                                    + "can be set to '0' with the flush 
interval set allowing for complete async processing of buffered actions.");
+
+    // Prefix for HBase specific properties.
+    public static final String PROPERTIES_PREFIX = "properties.";
+
+    // 
--------------------------------------------------------------------------------------------
+    // Validation
+    // 
--------------------------------------------------------------------------------------------
+
+    /**
+     * Checks that the HBase table have row key defined. A row key is defined 
as an atomic type, and
+     * column families and qualifiers are defined as ROW type. There shouldn't 
be multiple atomic
+     * type columns in the schema. The PRIMARY KEY constraint is optional, if 
exist, the primary key
+     * constraint must be defined on the single row key field.
+     */
+    public static void validatePrimaryKey(TableSchema schema) {
+        HBaseTableSchema hbaseSchema = 
HBaseTableSchema.fromTableSchema(schema);
+        if (!hbaseSchema.getRowKeyName().isPresent()) {
+            throw new IllegalArgumentException(
+                    "HBase table requires to define a row key field. "
+                            + "A row key field is defined as an atomic type, "
+                            + "column families and qualifiers are defined as 
ROW type.");
+        }
+        schema.getPrimaryKey()
+                .ifPresent(
+                        k -> {
+                            if (k.getColumns().size() > 1) {
+                                throw new IllegalArgumentException(
+                                        "HBase table doesn't support a primary 
Key on multiple columns. "
+                                                + "The primary key of HBase 
table must be defined on row key field.");
+                            }
+                            if 
(!hbaseSchema.getRowKeyName().get().equals(k.getColumns().get(0))) {
+                                throw new IllegalArgumentException(
+                                        "Primary key of HBase table must be 
defined on the row key field. "
+                                                + "A row key field is defined 
as an atomic type, "
+                                                + "column families and 
qualifiers are defined as ROW type.");
+                            }
+                        });
+    }
+
+    /**
+     * Options for validate HBase source table , The reason for separate 
source and sink is to
+     * facilitate future expansion of parameters.
+     *
+     * @param tableOptions
+     */
+    public static void validateTableSourceOptions(ReadableConfig tableOptions) 
{
+        validateTableName(tableOptions);
+        validateZookeeperQuorum(tableOptions);
+    }
+
+    /**
+     * Options for validate HBase sink table.
+     *
+     * @param tableOptions
+     */
+    public static void validateTableSinkOptions(ReadableConfig tableOptions) {

Review comment:
       ditto. 

##########
File path: 
flink-connectors/flink-connector-hbase-base/src/main/java/org/apache/flink/connector/hbase/options/HBaseOptions.java
##########
@@ -0,0 +1,263 @@
+/*
+ * 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.connector.hbase.options;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigOptions;
+import org.apache.flink.configuration.MemorySize;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.connector.hbase.util.HBaseConfigurationUtil;
+import org.apache.flink.connector.hbase.util.HBaseTableSchema;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.descriptors.DescriptorProperties;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
+
+import java.io.Serializable;
+import java.time.Duration;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Properties;
+
+import static 
org.apache.flink.table.descriptors.AbstractHBaseValidator.CONNECTOR_PROPERTIES;
+import static 
org.apache.flink.table.descriptors.AbstractHBaseValidator.CONNECTOR_ZK_NODE_PARENT;
+import static 
org.apache.flink.table.descriptors.AbstractHBaseValidator.CONNECTOR_ZK_QUORUM;
+import static org.apache.flink.table.factories.FactoryUtil.SINK_PARALLELISM;
+
+/** Common Options for HBase. */
+@Internal
+public class HBaseOptions implements Serializable {
+
+    public static final ConfigOption<String> TABLE_NAME =
+            ConfigOptions.key("table-name")
+                    .stringType()
+                    .noDefaultValue()
+                    .withDescription("The name of HBase table to connect.");
+
+    public static final ConfigOption<String> ZOOKEEPER_QUORUM =
+            ConfigOptions.key("zookeeper.quorum")
+                    .stringType()
+                    .noDefaultValue()
+                    .withDescription("The HBase Zookeeper quorum.");
+
+    public static final ConfigOption<String> ZOOKEEPER_ZNODE_PARENT =
+            ConfigOptions.key("zookeeper.znode.parent")
+                    .stringType()
+                    .defaultValue("/hbase")
+                    .withDescription("The root dir in Zookeeper for HBase 
cluster.");
+
+    public static final ConfigOption<String> NULL_STRING_LITERAL =
+            ConfigOptions.key("null-string-literal")
+                    .stringType()
+                    .defaultValue("null")
+                    .withDescription(
+                            "Representation for null values for string fields. 
HBase source and "
+                                    + "sink encodes/decodes empty bytes as 
null values for all types except string type.");
+
+    public static final ConfigOption<MemorySize> SINK_BUFFER_FLUSH_MAX_SIZE =
+            ConfigOptions.key("sink.buffer-flush.max-size")
+                    .memoryType()
+                    .defaultValue(MemorySize.parse("2mb"))
+                    .withDescription(
+                            "Writing option, maximum size in memory of 
buffered rows for each "
+                                    + "writing request. This can improve 
performance for writing data to HBase database, "
+                                    + "but may increase the latency. Can be 
set to '0' to disable it. ");
+
+    public static final ConfigOption<Integer> SINK_BUFFER_FLUSH_MAX_ROWS =
+            ConfigOptions.key("sink.buffer-flush.max-rows")
+                    .intType()
+                    .defaultValue(1000)
+                    .withDescription(
+                            "Writing option, maximum number of rows to buffer 
for each writing request. "
+                                    + "This can improve performance for 
writing data to HBase database, but may increase the latency. "
+                                    + "Can be set to '0' to disable it.");
+
+    public static final ConfigOption<Duration> SINK_BUFFER_FLUSH_INTERVAL =
+            ConfigOptions.key("sink.buffer-flush.interval")
+                    .durationType()
+                    .defaultValue(Duration.ofSeconds(1))
+                    .withDescription(
+                            "Writing option, the interval to flush any 
buffered rows. "
+                                    + "This can improve performance for 
writing data to HBase database, but may increase the latency. "
+                                    + "Can be set to '0' to disable it. Note, 
both 'sink.buffer-flush.max-size' and 'sink.buffer-flush.max-rows' "
+                                    + "can be set to '0' with the flush 
interval set allowing for complete async processing of buffered actions.");
+
+    // Prefix for HBase specific properties.
+    public static final String PROPERTIES_PREFIX = "properties.";
+
+    // 
--------------------------------------------------------------------------------------------
+    // Validation
+    // 
--------------------------------------------------------------------------------------------
+
+    /**
+     * Checks that the HBase table have row key defined. A row key is defined 
as an atomic type, and
+     * column families and qualifiers are defined as ROW type. There shouldn't 
be multiple atomic
+     * type columns in the schema. The PRIMARY KEY constraint is optional, if 
exist, the primary key
+     * constraint must be defined on the single row key field.
+     */
+    public static void validatePrimaryKey(TableSchema schema) {
+        HBaseTableSchema hbaseSchema = 
HBaseTableSchema.fromTableSchema(schema);
+        if (!hbaseSchema.getRowKeyName().isPresent()) {
+            throw new IllegalArgumentException(
+                    "HBase table requires to define a row key field. "
+                            + "A row key field is defined as an atomic type, "
+                            + "column families and qualifiers are defined as 
ROW type.");
+        }
+        schema.getPrimaryKey()
+                .ifPresent(
+                        k -> {
+                            if (k.getColumns().size() > 1) {
+                                throw new IllegalArgumentException(
+                                        "HBase table doesn't support a primary 
Key on multiple columns. "
+                                                + "The primary key of HBase 
table must be defined on row key field.");
+                            }
+                            if 
(!hbaseSchema.getRowKeyName().get().equals(k.getColumns().get(0))) {
+                                throw new IllegalArgumentException(
+                                        "Primary key of HBase table must be 
defined on the row key field. "
+                                                + "A row key field is defined 
as an atomic type, "
+                                                + "column families and 
qualifiers are defined as ROW type.");
+                            }
+                        });
+    }
+
+    /**
+     * Options for validate HBase source table , The reason for separate 
source and sink is to
+     * facilitate future expansion of parameters.
+     *
+     * @param tableOptions
+     */
+    public static void validateTableSourceOptions(ReadableConfig tableOptions) 
{

Review comment:
       We don't need this, this has been checked in 
`helper.validateExcept(PROPERTIES_PREFIX)`.




----------------------------------------------------------------
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:
us...@infra.apache.org


Reply via email to