[FLINK-32999][connectors/hbase] Remove HBase connector code from main repo (#23342)

* [FLINK-32999][connectors/hbase] Remove HBase connector code from main repo


---------

Co-authored-by: Sergey Nuyanzin <sergey.nuyanzin@aiven.io>
diff --git a/docs/content.zh/docs/connectors/table/hbase.md b/docs/content.zh/docs/connectors/table/hbase.md
deleted file mode 100644
index a3f8046..0000000
--- a/docs/content.zh/docs/connectors/table/hbase.md
+++ /dev/null
@@ -1,394 +0,0 @@
----
-title: HBase
-weight: 9
-type: docs
-aliases:
-  - /zh/dev/table/connectors/hbase.html
----
-<!--
-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.
--->
-
-# HBase SQL 连接器
-
-{{< label "Scan Source: Bounded" >}}
-{{< label "Lookup Source: Sync Mode" >}}
-{{< label "Sink: Batch" >}}
-{{< label "Sink: Streaming Upsert Mode" >}}
-
-HBase 连接器支持读取和写入 HBase 集群。本文档介绍如何使用 HBase 连接器基于 HBase 进行 SQL 查询。
-
-HBase 连接器在 upsert 模式下运行,可以使用 DDL 中定义的主键与外部系统交换更新操作消息。但是主键只能基于 HBase 的 rowkey 字段定义。如果没有声明主键,HBase 连接器默认取 rowkey 作为主键。
-
-依赖
-------------
-
-{{< sql_download_table "hbase" >}}
-
-HBase 连接器不是二进制发行版的一部分,请查阅[这里]({{< ref "docs/dev/configuration/overview" >}})了解如何在集群运行中引用 HBase 连接器。
-
-如何使用 HBase 表
-----------------
-
-所有 HBase 表的列簇必须定义为 ROW 类型,字段名对应列簇名(column family),嵌套的字段名对应列限定符名(column qualifier)。用户只需在表结构中声明查询中使用的的列簇和列限定符。除了 ROW 类型的列,剩下的原子数据类型字段(比如,STRING, BIGINT)将被识别为 HBase 的 rowkey,一张表中只能声明一个 rowkey。rowkey 字段的名字可以是任意的,如果是保留关键字,需要用反引号。
-
-```sql
--- 在 Flink SQL 中注册 HBase 表 "mytable"
-CREATE TABLE hTable (
- rowkey INT,
- family1 ROW<q1 INT>,
- family2 ROW<q2 STRING, q3 BIGINT>,
- family3 ROW<q4 DOUBLE, q5 BOOLEAN, q6 STRING>,
- PRIMARY KEY (rowkey) NOT ENFORCED
-) WITH (
- 'connector' = 'hbase-1.4',
- 'table-name' = 'mytable',
- 'zookeeper.quorum' = 'localhost:2181'
-);
-
--- 用 ROW(...) 构造函数构造列簇,并往 HBase 表写数据。
--- 假设 "T" 的表结构是 [rowkey, f1q1, f2q2, f2q3, f3q4, f3q5, f3q6]
-INSERT INTO hTable
-SELECT rowkey, ROW(f1q1), ROW(f2q2, f2q3), ROW(f3q4, f3q5, f3q6) FROM T;
-
--- 从 HBase 表扫描数据
-SELECT rowkey, family1, family3.q4, family3.q6 FROM hTable;
-
--- temporal join HBase 表,将 HBase 表作为维表
-SELECT * FROM myTopic
-LEFT JOIN hTable FOR SYSTEM_TIME AS OF myTopic.proctime
-ON myTopic.key = hTable.rowkey;
-```
-
-连接器参数
-----------------
-
-<table class="table table-bordered">
-    <thead>
-      <tr>
-        <th class="text-left" style="width: 25%">参数</th>
-        <th class="text-center" style="width: 10%">是否必选</th>
-        <th class="text-center" style="width: 10%">默认值</th>
-        <th class="text-center" style="width: 10%">数据类型</th>
-        <th class="text-center" style="width: 45%">描述</th>
-      </tr>
-    </thead>
-    <tbody>
-    <tr>
-      <td><h5>connector</h5></td>
-      <td>必选</td>
-      <td style="word-wrap: break-word;">(none)</td>
-      <td>String</td>
-      <td>指定使用的连接器, 支持的值如下 :
-        <ul>
-            <li><code>hbase-1.4</code>: 连接 HBase 1.4.x 集群</li>
-            <li><code>hbase-2.2</code>: 连接 HBase 2.2.x 集群</li>
-        </ul>
-      </td>
-    </tr>
-    <tr>
-      <td><h5>table-name</h5></td>
-      <td>必选</td>
-      <td style="word-wrap: break-word;">(none)</td>
-      <td>String</td>
-      <td>连接的 HBase 表名。默认该表在 "default" 命名空间下,指定命名空间下的表需要使用 "namespace:table"。</td>
-    </tr>
-    <tr>
-      <td><h5>zookeeper.quorum</h5></td>
-      <td>必选</td>
-      <td style="word-wrap: break-word;">(none)</td>
-      <td>String</td>
-      <td>HBase Zookeeper quorum 信息。</td>
-    </tr>
-    <tr>
-      <td><h5>zookeeper.znode.parent</h5></td>
-      <td>可选</td>
-      <td style="word-wrap: break-word;">/hbase</td>
-      <td>String</td>
-      <td>HBase 集群的 Zookeeper 根目录。</td>
-    </tr>
-    <tr>
-      <td><h5>null-string-literal</h5></td>
-      <td>可选</td>
-      <td style="word-wrap: break-word;">null</td>
-      <td>String</td>
-      <td>当字符串值为 <code>null</code> 时的存储形式,默认存成 "null" 字符串。HBase 的 source 和 sink 的编解码将所有数据类型(除字符串外)将 <code>null</code> 值以空字节来存储。</td>
-    </tr>
-    <tr>
-      <td><h5>sink.buffer-flush.max-size</h5></td>
-      <td>可选</td>
-      <td style="word-wrap: break-word;">2mb</td>
-      <td>MemorySize</td>
-      <td>写入的参数选项。每次写入请求缓存行的最大大小。它能提升写入 HBase 数据库的性能,但是也可能增加延迟。设置为 "0" 关闭此选项。
-      </td>
-    </tr>
-    <tr>
-      <td><h5>sink.buffer-flush.max-rows</h5></td>
-      <td>可选</td>
-      <td style="word-wrap: break-word;">1000</td>
-      <td>Integer</td>
-      <td>写入的参数选项。 每次写入请求缓存的最大行数。它能提升写入 HBase 数据库的性能,但是也可能增加延迟。设置为 "0" 关闭此选项。
-      </td>
-    </tr>
-    <tr>
-      <td><h5>sink.buffer-flush.interval</h5></td>
-      <td>可选</td>
-      <td style="word-wrap: break-word;">1s</td>
-      <td>Duration</td>
-      <td>写入的参数选项。刷写缓存行的间隔。它能提升写入 HBase 数据库的性能,但是也可能增加延迟。设置为 "0" 关闭此选项。注意:"sink.buffer-flush.max-size" 和 "sink.buffer-flush.max-rows" 同时设置为 "0",刷写选项整个异步处理缓存行为。
-      </td>
-    </tr>
-    <tr>
-      <td><h5>sink.parallelism</h5></td>
-      <td>可选</td>
-      <td style="word-wrap: break-word;">(none)</td>
-      <td>Integer</td>
-      <td>为 HBase sink operator 定义并行度。默认情况下,并行度由框架决定,和链在一起的上游 operator 一样。</td>
-    </tr>
-    <tr>
-      <td><h5>lookup.async</h5></td>
-      <td>可选</td>
-      <td style="word-wrap: break-word;">false</td>
-      <td>Boolean</td>
-      <td>是否启用异步查找。如果为真,查找将是异步的。注意:异步方式只支持 hbase-2.2 连接器</td>
-    </tr>
-    <tr>
-      <td><h5>lookup.cache</h5></td>
-      <td>可选</td>
-      <td style="word-wrap: break-word;">NONE</td>
-      <td><p>枚举类型</p>可选值: NONE, PARTIAL</td>
-      <td>维表的缓存策略。 目前支持 NONE(不缓存)和 PARTIAL(只在外部数据库中查找数据时缓存)。</td>
-    </tr>
-    <tr>
-      <td><h5>lookup.partial-cache.max-rows</h5></td>
-      <td>可选</td>
-      <td style="word-wrap: break-word;">(none)</td>
-      <td>Long</td>
-      <td>查找缓存的最大行数,超过这个值,最旧的行将过期。使用该配置时 "lookup.cache" 必须设置为 "PARTIAL”。</td>
-    </tr>
-    <tr>
-      <td><h5>lookup.partial-cache.expire-after-write</h5></td>
-      <td>可选</td>
-      <td style="word-wrap: break-word;">(none)</td>
-      <td>Duration</td>
-      <td>在记录写入缓存后该记录的最大保留时间。
-      使用该配置时 "lookup.cache" 必须设置为 "PARTIAL”。</td>
-    </tr>
-    <tr>
-      <td><h5>lookup.partial-cache.expire-after-access</h5></td>
-      <td>可选</td>
-      <td style="word-wrap: break-word;">(none)</td>
-      <td>Duration</td>
-      <td>在缓存中的记录被访问后该记录的最大保留时间。
-      使用该配置时 "lookup.cache" 必须设置为 "PARTIAL”。</td>
-    </tr>
-    <tr>
-      <td><h5>lookup.partial-cache.cache-missing-key</h5></td>
-      <td>可选</td>
-      <td style="word-wrap: break-word;">true</td>
-      <td>Boolean</td>
-      <td>是否缓存维表中不存在的键,默认为true。
-        使用该配置时 "lookup.cache" 必须设置为 "PARTIAL”。</td>
-    </tr>
-    <tr>
-      <td><h5>lookup.max-retries</h5></td>
-      <td>可选</td>
-      <td style="word-wrap: break-word;">3</td>
-      <td>Integer</td>
-      <td>查找数据库失败时的最大重试次数。</td>
-    </tr>
-    <tr>
-      <td><h5>properties.*</h5></td>
-      <td>可选</td>
-      <td style="word-wrap: break-word;">(无)</td>
-      <td>String</td>
-      <td>
-         可以设置任意 HBase 的配置项。后缀名必须匹配在 <a href="http://hbase.apache.org/2.3/book.html#hbase_default_configurations">HBase 配置文档</a> 中定义的配置键。Flink 将移除 "properties." 配置键前缀并将变换后的配置键和值传入底层的 HBase 客户端。
-         例如您可以设置 <code>'properties.hbase.security.authentication' = 'kerberos'</code> 等kerberos认证参数。
-      </td>
-    </tr>
-    </tbody>
-</table>
-
-### 已弃用的配置
-这些弃用配置已经被上述的新配置代替,而且最终会被弃用。请优先考虑使用新配置。
-<table>
-    <thead>
-      <tr>
-        <th class="text-left" style="width: 25%">Option</th>
-        <th class="text-left" style="width: 8%">Required</th>
-        <th class="text-left" style="width: 8%">Forwarded</th>
-        <th class="text-left" style="width: 7%">Default</th>
-        <th class="text-left" style="width: 10%">Type</th>
-        <th class="text-left" style="width: 42%">Description</th>
-      </tr>
-    </thead>
-    <tbody>
-        <tr>
-          <td><h5>lookup.cache.max-rows</h5></td>
-          <td>optional</td>
-          <td>yes</td>
-          <td style="word-wrap: break-word;">(none)</td>
-          <td>Integer</td>
-          <td>请配置 "lookup.cache" = "PARTIAL" 并使用 "lookup.partial-cache.max-rows" 代替</td>
-        </tr>
-        <tr>
-          <td><h5>lookup.cache.ttl</h5></td>
-          <td>optional</td>
-          <td>yes</td>
-          <td style="word-wrap: break-word;">(none)</td>
-          <td>Duration</td>
-          <td>请配置 "lookup.cache" = "PARTIAL" 并使用 "lookup.partial-cache.expire-after-write" 代替</td>
-        </tr>
-    </tbody>
-<table>
-
-
-
-数据类型映射表
-----------------
-
-HBase 以字节数组存储所有数据。在读和写过程中要序列化和反序列化数据。
-
-Flink 的 HBase 连接器利用 HBase(Hadoop) 的工具类 `org.apache.hadoop.hbase.util.Bytes` 进行字节数组和 Flink 数据类型转换。
-
-Flink 的 HBase 连接器将所有数据类型(除字符串外)`null` 值编码成空字节。对于字符串类型,`null` 值的字面值由`null-string-literal`选项值决定。
-
-数据类型映射表如下:
-
-<table class="table table-bordered">
-    <thead>
-      <tr>
-        <th class="text-left">Flink 数据类型</th>
-        <th class="text-left">HBase 转换</th>
-      </tr>
-    </thead>
-    <tbody>
-    <tr>
-      <td><code>CHAR / VARCHAR / STRING</code></td>
-      <td>
-{{< highlight "java" >}}
-byte[] toBytes(String s)
-String toString(byte[] b)
-{{< /highlight >}}
-      </td>
-    </tr>
-    <tr>
-      <td><code>BOOLEAN</code></td>
-      <td>
-{{< highlight "java" >}}
-byte[] toBytes(boolean b)
-boolean toBoolean(byte[] b)
-{{< /highlight >}}
-      </td>
-    </tr>
-    <tr>
-      <td><code>BINARY / VARBINARY</code></td>
-      <td>返回 <code>byte[]</code>。</td>
-    </tr>
-    <tr>
-      <td><code>DECIMAL</code></td>
-      <td>
-{{< highlight "java" >}}
-byte[] toBytes(BigDecimal v)
-BigDecimal toBigDecimal(byte[] b)
-{{< /highlight >}}
-      </td>
-    </tr>
-    <tr>
-      <td><code>TINYINT</code></td>
-      <td>
-{{< highlight "java" >}}
-new byte[] { val }
-bytes[0] // returns first and only byte from bytes
-{{< /highlight >}}
-      </td>
-    </tr>
-    <tr>
-      <td><code>SMALLINT</code></td>
-      <td>
-{{< highlight "java" >}}
-byte[] toBytes(short val)
-short toShort(byte[] bytes)
-{{< /highlight >}}
-      </td>
-    </tr>
-    <tr>
-      <td><code>INT</code></td>
-      <td>
-{{< highlight "java" >}}
-byte[] toBytes(int val)
-int toInt(byte[] bytes)
-{{< /highlight >}}
-      </td>
-    </tr>
-    <tr>
-      <td><code>BIGINT</code></td>
-      <td>
-{{< highlight "java" >}}
-byte[] toBytes(long val)
-long toLong(byte[] bytes)
-{{< /highlight >}}
-      </td>
-    </tr>
-    <tr>
-      <td><code>FLOAT</code></td>
-      <td>
-{{< highlight "java" >}}
-byte[] toBytes(float val)
-float toFloat(byte[] bytes)
-{{< /highlight >}}
-      </td>
-    </tr>
-    <tr>
-      <td><code>DOUBLE</code></td>
-      <td>
-{{< highlight "java" >}}
-byte[] toBytes(double val)
-double toDouble(byte[] bytes)
-{{< /highlight >}}
-      </td>
-    </tr>
-    <tr>
-      <td><code>DATE</code></td>
-      <td>从 1970-01-01 00:00:00 UTC 开始的天数,int 值。</td>
-    </tr>
-    <tr>
-      <td><code>TIME</code></td>
-      <td>从 1970-01-01 00:00:00 UTC 开始天的毫秒数,int 值。</td>
-    </tr>
-    <tr>
-      <td><code>TIMESTAMP</code></td>
-      <td>从 1970-01-01 00:00:00 UTC 开始的毫秒数,long 值。</td>
-    </tr>
-    <tr>
-      <td><code>ARRAY</code></td>
-      <td>不支持</td>
-    </tr>
-    <tr>
-      <td><code>MAP / MULTISET</code></td>
-      <td>不支持</td>
-    </tr>
-    <tr>
-      <td><code>ROW</code></td>
-      <td>不支持</td>
-    </tr>
-    </tbody>
-</table>
-
-{{< top >}}
diff --git a/docs/content/docs/connectors/table/hbase.md b/docs/content/docs/connectors/table/hbase.md
deleted file mode 100644
index 4c2b37b..0000000
--- a/docs/content/docs/connectors/table/hbase.md
+++ /dev/null
@@ -1,421 +0,0 @@
----
-title: HBase
-weight: 9
-type: docs
-aliases:
-  - /dev/table/connectors/hbase.html
----
-<!--
-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.
--->
-
-# HBase SQL Connector
-
-{{< label "Scan Source: Bounded" >}}
-{{< label "Lookup Source: Sync Mode" >}}
-{{< label "Sink: Batch" >}}
-{{< label "Sink: Streaming Upsert Mode" >}}
-
-The HBase connector allows for reading from and writing to an HBase cluster. This document describes how to setup the HBase Connector to run SQL queries against HBase.
-
-HBase always works in upsert mode for exchange changelog messages with the external system using a primary key defined on the DDL. The primary key must be defined on the HBase rowkey field (rowkey field must be declared). If the PRIMARY KEY clause is not declared, the HBase connector will take rowkey as the primary key by default.
-
-Dependencies
-------------
-
-{{< sql_download_table "hbase" >}}
-
-The HBase connector is not part of the binary distribution.
-See how to link with it for cluster execution [here]({{< ref "docs/dev/configuration/overview" >}}).
-
-
-How to use HBase table
-----------------
-
-All the column families in HBase table must be declared as ROW type, the field name maps to the column family name, and the nested field names map to the column qualifier names. There is no need to declare all the families and qualifiers in the schema, users can declare what’s used in the query. Except the ROW type fields, the single atomic type field (e.g. STRING, BIGINT) will be recognized as HBase rowkey. The rowkey field can be arbitrary name, but should be quoted using backticks if it is a reserved keyword.
-
-```sql
--- register the HBase table 'mytable' in Flink SQL
-CREATE TABLE hTable (
- rowkey INT,
- family1 ROW<q1 INT>,
- family2 ROW<q2 STRING, q3 BIGINT>,
- family3 ROW<q4 DOUBLE, q5 BOOLEAN, q6 STRING>,
- PRIMARY KEY (rowkey) NOT ENFORCED
-) WITH (
- 'connector' = 'hbase-1.4',
- 'table-name' = 'mytable',
- 'zookeeper.quorum' = 'localhost:2181'
-);
-
--- use ROW(...) construction function construct column families and write data into the HBase table.
--- assuming the schema of "T" is [rowkey, f1q1, f2q2, f2q3, f3q4, f3q5, f3q6]
-INSERT INTO hTable
-SELECT rowkey, ROW(f1q1), ROW(f2q2, f2q3), ROW(f3q4, f3q5, f3q6) FROM T;
-
--- scan data from the HBase table
-SELECT rowkey, family1, family3.q4, family3.q6 FROM hTable;
-
--- temporal join the HBase table as a dimension table
-SELECT * FROM myTopic
-LEFT JOIN hTable FOR SYSTEM_TIME AS OF myTopic.proctime
-ON myTopic.key = hTable.rowkey;
-```
-
-Connector Options
-----------------
-
-<table class="table table-bordered">
-    <thead>
-      <tr>
-        <th class="text-left" style="width: 25%">Option</th>
-        <th class="text-center" style="width: 8%">Required</th>
-        <th class="text-center" style="width: 8%">Forwarded</th>
-        <th class="text-center" style="width: 7%">Default</th>
-        <th class="text-center" style="width: 10%">Type</th>
-        <th class="text-center" style="width: 42%">Description</th>
-      </tr>
-    </thead>
-    <tbody>
-    <tr>
-      <td><h5>connector</h5></td>
-      <td>required</td>
-      <td>no</td>
-      <td style="word-wrap: break-word;">(none)</td>
-      <td>String</td>
-      <td>Specify what connector to use, valid values are:
-        <ul>
-            <li><code>hbase-1.4</code>: connect to HBase 1.4.x cluster</li>
-            <li><code>hbase-2.2</code>: connect to HBase 2.2.x cluster</li>
-        </ul>
-      </td>
-    </tr>
-    <tr>
-      <td><h5>table-name</h5></td>
-      <td>required</td>
-      <td>yes</td>
-      <td style="word-wrap: break-word;">(none)</td>
-      <td>String</td>
-      <td>The name of HBase table to connect. By default, the table is in 'default' namespace. To assign the table a specified namespace you need to use 'namespace:table'.</td>
-    </tr>
-    <tr>
-      <td><h5>zookeeper.quorum</h5></td>
-      <td>required</td>
-      <td>yes</td>
-      <td style="word-wrap: break-word;">(none)</td>
-      <td>String</td>
-      <td>The HBase Zookeeper quorum.</td>
-    </tr>
-    <tr>
-      <td><h5>zookeeper.znode.parent</h5></td>
-      <td>optional</td>
-      <td>yes</td>
-      <td style="word-wrap: break-word;">/hbase</td>
-      <td>String</td>
-      <td>The root dir in Zookeeper for HBase cluster.</td>
-    </tr>
-    <tr>
-      <td><h5>null-string-literal</h5></td>
-      <td>optional</td>
-      <td>yes</td>
-      <td style="word-wrap: break-word;">null</td>
-      <td>String</td>
-      <td>Representation for null values for string fields. HBase source and sink encodes/decodes empty bytes as null values for all types except string type.</td>
-    </tr>
-    <tr>
-      <td><h5>sink.buffer-flush.max-size</h5></td>
-      <td>optional</td>
-      <td>yes</td>
-      <td style="word-wrap: break-word;">2mb</td>
-      <td>MemorySize</td>
-      <td>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 <code>'0'</code> to disable it.
-      </td>
-    </tr>
-    <tr>
-      <td><h5>sink.buffer-flush.max-rows</h5></td>
-      <td>optional</td>
-      <td>yes</td>
-      <td style="word-wrap: break-word;">1000</td>
-      <td>Integer</td>
-      <td>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 <code>'0'</code> to disable it.
-      </td>
-    </tr>
-    <tr>
-      <td><h5>sink.buffer-flush.interval</h5></td>
-      <td>optional</td>
-      <td>yes</td>
-      <td style="word-wrap: break-word;">1s</td>
-      <td>Duration</td>
-      <td>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 <code>'0'</code> to disable it. Note, both <code>'sink.buffer-flush.max-size'</code> and <code>'sink.buffer-flush.max-rows'</code>
-      can be set to <code>'0'</code> with the flush interval set allowing for complete async processing of buffered actions.
-      </td>
-    </tr>
-    <tr>
-      <td><h5>sink.parallelism</h5></td>
-      <td>optional</td>
-      <td>no</td>
-      <td style="word-wrap: break-word;">(none)</td>
-      <td>Integer</td>
-      <td>Defines the parallelism of the HBase sink operator. By default, the parallelism is determined by the framework using the same parallelism of the upstream chained operator.</td>
-    </tr>
-    <tr>
-      <td><h5>lookup.async</h5></td>
-      <td>optional</td>
-      <td>no</td>
-      <td style="word-wrap: break-word;">false</td>
-      <td>Boolean</td>
-      <td>Whether async lookup are enabled. If true, the lookup will be async. Note, async only supports hbase-2.2 connector.</td>
-    </tr>
-    <tr>
-      <td><h5>lookup.cache</h5></td>
-      <td>optional</td>
-      <td>yes</td>
-      <td style="word-wrap: break-word;">NONE</td>
-      <td><p>Enum</p>Possible values: NONE, PARTIAL</td>
-      <td>The cache strategy for the lookup table. Currently supports NONE (no caching) and PARTIAL (caching entries on lookup operation in external database).</td>
-    </tr>
-    <tr>
-      <td><h5>lookup.partial-cache.max-rows</h5></td>
-      <td>optional</td>
-      <td>yes</td>
-      <td style="word-wrap: break-word;">(none)</td>
-      <td>Long</td>
-      <td>The max number of rows of lookup cache, over this value, the oldest rows will be expired. 
-        "lookup.cache" must be set to "PARTIAL" to use this option.</td>
-    </tr>
-    <tr>
-      <td><h5>lookup.partial-cache.expire-after-write</h5></td>
-      <td>optional</td>
-      <td>yes</td>
-      <td style="word-wrap: break-word;">(none)</td>
-      <td>Duration</td>
-      <td>The max time to live for each rows in lookup cache after writing into the cache
-        "lookup.cache" must be set to "PARTIAL" to use this option. </td>
-    </tr>
-    <tr>
-      <td><h5>lookup.partial-cache.expire-after-access</h5></td>
-      <td>optional</td>
-      <td>yes</td>
-      <td style="word-wrap: break-word;">(none)</td>
-      <td>Duration</td>
-      <td>The max time to live for each rows in lookup cache after accessing the entry in the cache.
-      "lookup.cache" must be set to "PARTIAL" to use this option. </td>
-    </tr>
-    <tr>
-      <td><h5>lookup.partial-cache.cache-missing-key</h5></td>
-      <td>optional</td>
-      <td>yes</td>
-      <td style="word-wrap: break-word;">true</td>
-      <td>Boolean</td>
-      <td>Whether to store an empty value into the cache if the lookup key doesn't match any rows in the table. 
-        "lookup.cache" must be set to "PARTIAL" to use this option.</td>
-    </tr>
-    <tr>
-      <td><h5>lookup.max-retries</h5></td>
-      <td>optional</td>
-      <td>yes</td>
-      <td style="word-wrap: break-word;">3</td>
-      <td>Integer</td>
-      <td>The max retry times if lookup database failed.</td>
-    </tr>
-    <tr>
-      <td><h5>properties.*</h5></td>
-      <td>optional</td>
-      <td>no</td>
-      <td style="word-wrap: break-word;">(none)</td>
-      <td>String</td>
-      <td>
-         This can set and pass arbitrary HBase configurations. Suffix names must match the configuration key defined in <a href="http://hbase.apache.org/2.3/book.html#hbase_default_configurations">HBase Configuration documentation</a>. Flink will remove the "properties." key prefix and pass the transformed key and values to the underlying HBaseClient. For example, you can add a kerberos authentication parameter <code>'properties.hbase.security.authentication' = 'kerberos'</code>.
-      </td>
-    </tr>
-    </tbody>
-</table>
-
-### Deprecated Options
-These deprecated options has been replaced by new options listed above and will be removed eventually. Please consider using new options first.
-<table>
-    <thead>
-      <tr>
-        <th class="text-left" style="width: 25%">Option</th>
-        <th class="text-left" style="width: 8%">Required</th>
-        <th class="text-left" style="width: 8%">Forwarded</th>
-        <th class="text-left" style="width: 7%">Default</th>
-        <th class="text-left" style="width: 10%">Type</th>
-        <th class="text-left" style="width: 42%">Description</th>
-      </tr>
-    </thead>
-    <tbody>
-        <tr>
-          <td><h5>lookup.cache.max-rows</h5></td>
-          <td>optional</td>
-          <td>yes</td>
-          <td style="word-wrap: break-word;">(none)</td>
-          <td>Integer</td>
-          <td>Please set "lookup.cache" = "PARTIAL" and use "lookup.partial-cache.max-rows" instead.</td>
-        </tr>
-        <tr>
-          <td><h5>lookup.cache.ttl</h5></td>
-          <td>optional</td>
-          <td>yes</td>
-          <td style="word-wrap: break-word;">(none)</td>
-          <td>Duration</td>
-          <td>Please set "lookup.cache" = "PARTIAL" and use "lookup.partial-cache.expire-after-write" instead.</td>
-        </tr>
-    </tbody>
-<table>
-
-
-
-Data Type Mapping
-----------------
-
-HBase stores all data as byte arrays. The data needs to be serialized and deserialized during read and write operation
-
-When serializing and de-serializing, Flink HBase connector uses utility class `org.apache.hadoop.hbase.util.Bytes` provided by HBase (Hadoop) to convert Flink Data Types to and from byte arrays.
-
-Flink HBase connector encodes `null` values to empty bytes, and decode empty bytes to `null` values for all data types except string type. For string type, the null literal is determined by `null-string-literal` option.
-
-The data type mappings are as follows:
-
-<table class="table table-bordered">
-    <thead>
-      <tr>
-        <th class="text-left">Flink SQL type</th>
-        <th class="text-left">HBase conversion</th>
-      </tr>
-    </thead>
-    <tbody>
-    <tr>
-      <td><code>CHAR / VARCHAR / STRING</code></td>
-      <td>
-{{< highlight "java" >}}
-byte[] toBytes(String s)
-String toString(byte[] b)
-{{< /highlight >}}
-      </td>
-    </tr>
-    <tr>
-      <td><code>BOOLEAN</code></td>
-      <td>
-{{< highlight "java" >}}
-byte[] toBytes(boolean b)
-boolean toBoolean(byte[] b)
-{{< /highlight >}}
-      </td>
-    </tr>
-    <tr>
-      <td><code>BINARY / VARBINARY</code></td>
-      <td>Returns <code>byte[]</code> as is.</td>
-    </tr>
-    <tr>
-      <td><code>DECIMAL</code></td>
-      <td>
-{{< highlight "java" >}}
-byte[] toBytes(BigDecimal v)
-BigDecimal toBigDecimal(byte[] b)
-{{< /highlight >}}
-      </td>
-    </tr>
-    <tr>
-      <td><code>TINYINT</code></td>
-      <td>
-{{< highlight "java" >}}
-new byte[] { val }
-bytes[0] // returns first and only byte from bytes
-{{< /highlight >}}
-      </td>
-    </tr>
-    <tr>
-      <td><code>SMALLINT</code></td>
-      <td>
-{{< highlight "java" >}}
-byte[] toBytes(short val)
-short toShort(byte[] bytes)
-{{< /highlight >}}
-      </td>
-    </tr>
-    <tr>
-      <td><code>INT</code></td>
-      <td>
-{{< highlight "java" >}}
-byte[] toBytes(int val)
-int toInt(byte[] bytes)
-{{< /highlight >}}
-      </td>
-    </tr>
-    <tr>
-      <td><code>BIGINT</code></td>
-      <td>
-{{< highlight "java" >}}
-byte[] toBytes(long val)
-long toLong(byte[] bytes)
-{{< /highlight >}}
-      </td>
-    </tr>
-    <tr>
-      <td><code>FLOAT</code></td>
-      <td>
-{{< highlight "java" >}}
-byte[] toBytes(float val)
-float toFloat(byte[] bytes)
-{{< /highlight >}}
-      </td>
-    </tr>
-    <tr>
-      <td><code>DOUBLE</code></td>
-      <td>
-{{< highlight "java" >}}
-byte[] toBytes(double val)
-double toDouble(byte[] bytes)
-{{< /highlight >}}
-      </td>
-    </tr>
-    <tr>
-      <td><code>DATE</code></td>
-      <td>Stores the number of days since epoch as int value.</td>
-    </tr>
-    <tr>
-      <td><code>TIME</code></td>
-      <td>Stores the number of milliseconds of the day as int value.</td>
-    </tr>
-    <tr>
-      <td><code>TIMESTAMP</code></td>
-      <td>Stores the milliseconds since epoch as long value.</td>
-    </tr>
-    <tr>
-      <td><code>ARRAY</code></td>
-      <td>Not supported</td>
-    </tr>
-    <tr>
-      <td><code>MAP / MULTISET</code></td>
-      <td>Not supported</td>
-    </tr>
-    <tr>
-      <td><code>ROW</code></td>
-      <td>Not supported</td>
-    </tr>
-    </tbody>
-</table>
-
-{{< top >}}
diff --git a/docs/setup_docs.sh b/docs/setup_docs.sh
index 2e11ed8..11302bb 100755
--- a/docs/setup_docs.sh
+++ b/docs/setup_docs.sh
@@ -53,6 +53,7 @@
 integrate_connector_docs mongodb v1.0
 integrate_connector_docs opensearch v1.0
 integrate_connector_docs kafka v3.0
+integrate_connector_docs hbase v3.0
 
 cd ..
 rm -rf tmp
diff --git a/flink-architecture-tests/flink-architecture-tests-production/archunit-violations/e5126cae-f3fe-48aa-b6fb-60ae6cc3fcd5 b/flink-architecture-tests/flink-architecture-tests-production/archunit-violations/e5126cae-f3fe-48aa-b6fb-60ae6cc3fcd5
index 1046514..1fea781 100644
--- a/flink-architecture-tests/flink-architecture-tests-production/archunit-violations/e5126cae-f3fe-48aa-b6fb-60ae6cc3fcd5
+++ b/flink-architecture-tests/flink-architecture-tests-production/archunit-violations/e5126cae-f3fe-48aa-b6fb-60ae6cc3fcd5
@@ -7,7 +7,7 @@
 Method <org.apache.flink.runtime.blob.BlobInputStream.read([B, int, int)> calls method <org.apache.flink.runtime.blob.BlobKey.getHash()> in (BlobInputStream.java:163)
 Method <org.apache.flink.runtime.blob.BlobOutputStream.receiveAndCheckPutResponse(java.io.InputStream, java.security.MessageDigest, org.apache.flink.runtime.blob.BlobKey$BlobType)> calls method <org.apache.flink.runtime.blob.BlobKey.getHash()> in (BlobOutputStream.java:155)
 Method <org.apache.flink.runtime.blob.BlobUtils.checkAndDeleteCorruptedBlobs(java.nio.file.Path, org.slf4j.Logger)> calls method <org.apache.flink.runtime.blob.BlobKey.getHash()> in (BlobUtils.java:514)
-Method <org.apache.flink.runtime.blob.FileSystemBlobStore.get(java.lang.String, java.io.File, org.apache.flink.runtime.blob.BlobKey)> calls method <org.apache.flink.runtime.blob.BlobKey.getHash()> in (FileSystemBlobStore.java:133)
+Method <org.apache.flink.runtime.blob.FileSystemBlobStore.get(java.lang.String, java.io.File, org.apache.flink.runtime.blob.BlobKey)> calls method <org.apache.flink.runtime.blob.BlobKey.getHash()> in (FileSystemBlobStore.java:129)
 Method <org.apache.flink.runtime.executiongraph.Execution.finishPartitionsAndUpdateConsumers()> calls method <org.apache.flink.runtime.executiongraph.ExecutionVertex.finishPartitionsIfNeeded()> in (Execution.java:985)
 Method <org.apache.flink.runtime.highavailability.nonha.embedded.EmbeddedHaServicesWithLeadershipControl.grantDispatcherLeadership()> calls method <org.apache.flink.runtime.highavailability.nonha.embedded.EmbeddedLeaderService.grantLeadership()> in (EmbeddedHaServicesWithLeadershipControl.java:83)
 Method <org.apache.flink.runtime.highavailability.nonha.embedded.EmbeddedHaServicesWithLeadershipControl.grantJobMasterLeadership(org.apache.flink.api.common.JobID)> calls method <org.apache.flink.runtime.highavailability.nonha.embedded.EmbeddedLeaderService.grantLeadership()> in (EmbeddedHaServicesWithLeadershipControl.java:95)
@@ -16,12 +16,12 @@
 Method <org.apache.flink.runtime.highavailability.nonha.embedded.EmbeddedHaServicesWithLeadershipControl.revokeJobMasterLeadership(org.apache.flink.api.common.JobID)> calls method <org.apache.flink.runtime.highavailability.nonha.embedded.EmbeddedLeaderService.revokeLeadership()> in (EmbeddedHaServicesWithLeadershipControl.java:89)
 Method <org.apache.flink.runtime.highavailability.nonha.embedded.EmbeddedHaServicesWithLeadershipControl.revokeResourceManagerLeadership()> calls method <org.apache.flink.runtime.highavailability.nonha.embedded.EmbeddedLeaderService.revokeLeadership()> in (EmbeddedHaServicesWithLeadershipControl.java:102)
 Method <org.apache.flink.runtime.io.network.partition.ResourceManagerPartitionTrackerImpl.listDataSets()> calls method <org.apache.flink.runtime.io.network.partition.DataSetMetaInfo.withNumRegisteredPartitions(int, int)> in (ResourceManagerPartitionTrackerImpl.java:286)
-Method <org.apache.flink.runtime.operators.coordination.RecreateOnResetOperatorCoordinator$DeferrableCoordinator.closeAsync(long)> calls method <org.apache.flink.runtime.operators.coordination.RecreateOnResetOperatorCoordinator$QuiesceableContext.quiesce()> in (RecreateOnResetOperatorCoordinator.java:354)
+Method <org.apache.flink.runtime.operators.coordination.RecreateOnResetOperatorCoordinator$DeferrableCoordinator.closeAsync(long)> calls method <org.apache.flink.runtime.operators.coordination.RecreateOnResetOperatorCoordinator$QuiesceableContext.quiesce()> in (RecreateOnResetOperatorCoordinator.java:361)
 Method <org.apache.flink.runtime.taskexecutor.TaskManagerConfiguration.fromConfiguration(org.apache.flink.configuration.Configuration, org.apache.flink.runtime.taskexecutor.TaskExecutorResourceSpec, java.lang.String, java.io.File)> calls method <org.apache.flink.runtime.taskexecutor.TaskExecutorResourceUtils.generateDefaultSlotResourceProfile(org.apache.flink.runtime.taskexecutor.TaskExecutorResourceSpec, int)> in (TaskManagerConfiguration.java:244)
 Method <org.apache.flink.runtime.taskexecutor.TaskManagerConfiguration.fromConfiguration(org.apache.flink.configuration.Configuration, org.apache.flink.runtime.taskexecutor.TaskExecutorResourceSpec, java.lang.String, java.io.File)> calls method <org.apache.flink.runtime.taskexecutor.TaskExecutorResourceUtils.generateTotalAvailableResourceProfile(org.apache.flink.runtime.taskexecutor.TaskExecutorResourceSpec)> in (TaskManagerConfiguration.java:246)
-Method <org.apache.flink.runtime.taskexecutor.TaskManagerServices.createTaskSlotTable(int, org.apache.flink.runtime.taskexecutor.TaskExecutorResourceSpec, long, int, java.util.concurrent.Executor)> calls method <org.apache.flink.runtime.taskexecutor.TaskExecutorResourceUtils.generateDefaultSlotResourceProfile(org.apache.flink.runtime.taskexecutor.TaskExecutorResourceSpec, int)> in (TaskManagerServices.java:429)
-Method <org.apache.flink.runtime.taskexecutor.TaskManagerServices.createTaskSlotTable(int, org.apache.flink.runtime.taskexecutor.TaskExecutorResourceSpec, long, int, java.util.concurrent.Executor)> calls method <org.apache.flink.runtime.taskexecutor.TaskExecutorResourceUtils.generateTotalAvailableResourceProfile(org.apache.flink.runtime.taskexecutor.TaskExecutorResourceSpec)> in (TaskManagerServices.java:427)
+Method <org.apache.flink.runtime.taskexecutor.TaskManagerServices.createTaskSlotTable(int, org.apache.flink.runtime.taskexecutor.TaskExecutorResourceSpec, long, int, java.util.concurrent.Executor)> calls method <org.apache.flink.runtime.taskexecutor.TaskExecutorResourceUtils.generateDefaultSlotResourceProfile(org.apache.flink.runtime.taskexecutor.TaskExecutorResourceSpec, int)> in (TaskManagerServices.java:433)
+Method <org.apache.flink.runtime.taskexecutor.TaskManagerServices.createTaskSlotTable(int, org.apache.flink.runtime.taskexecutor.TaskExecutorResourceSpec, long, int, java.util.concurrent.Executor)> calls method <org.apache.flink.runtime.taskexecutor.TaskExecutorResourceUtils.generateTotalAvailableResourceProfile(org.apache.flink.runtime.taskexecutor.TaskExecutorResourceSpec)> in (TaskManagerServices.java:431)
 Method <org.apache.flink.streaming.api.operators.SourceOperator$1$1.asClassLoader()> calls method <org.apache.flink.streaming.api.operators.SourceOperator.getRuntimeContext()> in (SourceOperator.java:295)
 Method <org.apache.flink.streaming.api.operators.SourceOperator$1$1.registerReleaseHookIfAbsent(java.lang.String, java.lang.Runnable)> calls method <org.apache.flink.streaming.api.operators.SourceOperator.getRuntimeContext()> in (SourceOperator.java:301)
 Method <org.apache.flink.streaming.runtime.tasks.SourceOperatorStreamTask.init()> calls method <org.apache.flink.streaming.api.operators.SourceOperator.getSourceReader()> in (SourceOperatorStreamTask.java:95)
-Method <org.apache.flink.streaming.runtime.tasks.mailbox.MailboxExecutorImpl.isIdle()> calls method <org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor.isDefaultActionAvailable()> in (MailboxExecutorImpl.java:63)
+Method <org.apache.flink.streaming.runtime.tasks.mailbox.MailboxExecutorImpl.isIdle()> calls method <org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor.isDefaultActionAvailable()> in (MailboxExecutorImpl.java:63)
\ No newline at end of file
diff --git a/flink-architecture-tests/flink-architecture-tests-production/archunit-violations/f7a4e6fa-e7de-48c9-a61e-c13e83f0c72e b/flink-architecture-tests/flink-architecture-tests-production/archunit-violations/f7a4e6fa-e7de-48c9-a61e-c13e83f0c72e
index f7d7563..173d6ed 100644
--- a/flink-architecture-tests/flink-architecture-tests-production/archunit-violations/f7a4e6fa-e7de-48c9-a61e-c13e83f0c72e
+++ b/flink-architecture-tests/flink-architecture-tests-production/archunit-violations/f7a4e6fa-e7de-48c9-a61e-c13e83f0c72e
@@ -268,16 +268,15 @@
 Field <org.apache.flink.connector.file.table.stream.compact.CompactBucketWriter$Factory.factory> has generic type <org.apache.flink.util.function.SupplierWithException<org.apache.flink.streaming.api.functions.sink.filesystem.BucketWriter<T, java.lang.String>, java.io.IOException>> with type argument depending on <org.apache.flink.streaming.api.functions.sink.filesystem.BucketWriter> in (CompactBucketWriter.java:0)
 Field <org.apache.flink.connector.file.table.stream.compact.CompactBucketWriter.bucketWriter> has type <org.apache.flink.streaming.api.functions.sink.filesystem.BucketWriter> in (CompactBucketWriter.java:0)
 Field <org.apache.flink.connector.file.table.stream.compact.CompactBucketWriter.writer> has type <org.apache.flink.streaming.api.functions.sink.filesystem.InProgressFileWriter> in (CompactBucketWriter.java:0)
-Field <org.apache.flink.connector.hbase.util.HBaseSerde.reusedFamilyRows> has type <[Lorg.apache.flink.table.data.GenericRowData;> in (HBaseSerde.java:0)
 Field <org.apache.flink.connector.print.table.PrintTableSinkFactory$RowDataPrintFunction.writer> has type <org.apache.flink.api.common.functions.util.PrintSinkOutputWriter> in (PrintTableSinkFactory.java:0)
 Method <org.apache.flink.connector.base.sink.throwable.FatalExceptionClassifier.withRootCauseOfType(java.lang.Class, java.util.function.Function)> calls method <org.apache.flink.util.ExceptionUtils.findThrowable(java.lang.Throwable, java.lang.Class)> in (FatalExceptionClassifier.java:60)
 Method <org.apache.flink.connector.base.source.hybrid.HybridSource$HybridSourceBuilder.addSource(org.apache.flink.connector.base.source.hybrid.HybridSource$SourceFactory, org.apache.flink.api.connector.source.Boundedness)> calls method <org.apache.flink.api.java.ClosureCleaner.clean(java.lang.Object, org.apache.flink.api.common.ExecutionConfig$ClosureCleanerLevel, boolean)> in (HybridSource.java:246)
 Method <org.apache.flink.connector.base.source.hybrid.HybridSource$HybridSourceBuilder.addSource(org.apache.flink.connector.base.source.hybrid.HybridSource$SourceFactory, org.apache.flink.api.connector.source.Boundedness)> calls method <org.apache.flink.util.Preconditions.checkArgument(boolean, java.lang.Object)> in (HybridSource.java:242)
 Method <org.apache.flink.connector.base.source.hybrid.HybridSourceReader.addSplits(java.util.List)> calls method <org.apache.flink.util.Preconditions.checkState(boolean, java.lang.String, [Ljava.lang.Object;)> in (HybridSourceReader.java:153)
 Method <org.apache.flink.connector.base.source.hybrid.HybridSourceReader.setCurrentReader(int)> calls method <org.apache.flink.util.Preconditions.checkArgument(boolean)> in (HybridSourceReader.java:206)
-Method <org.apache.flink.connector.base.source.hybrid.HybridSourceSplitEnumerator.checkAndSignalNoMoreSplits(org.apache.flink.api.connector.source.SplitEnumeratorContext, int, int, int)> calls method <org.apache.flink.api.connector.source.SupportsIntermediateNoMoreSplits.signalIntermediateNoMoreSplits(int)> in (HybridSourceSplitEnumerator.java:440)
-Method <org.apache.flink.connector.base.source.hybrid.HybridSourceSplitEnumerator.checkAndSignalNoMoreSplits(org.apache.flink.api.connector.source.SplitEnumeratorContext, int, int, int)> calls method <org.apache.flink.util.Preconditions.checkState(boolean, java.lang.String, [Ljava.lang.Object;)> in (HybridSourceSplitEnumerator.java:432)
-Method <org.apache.flink.connector.base.source.hybrid.HybridSourceSplitEnumerator.checkAndSignalNoMoreSplits(org.apache.flink.api.connector.source.SplitEnumeratorContext, int, int, int)> checks instanceof <org.apache.flink.api.connector.source.SupportsIntermediateNoMoreSplits> in (HybridSourceSplitEnumerator.java:432)
+Method <org.apache.flink.connector.base.source.hybrid.HybridSourceSplitEnumerator.checkAndSignalNoMoreSplits(org.apache.flink.api.connector.source.SplitEnumeratorContext, int, int, int)> calls method <org.apache.flink.api.connector.source.SupportsIntermediateNoMoreSplits.signalIntermediateNoMoreSplits(int)> in (HybridSourceSplitEnumerator.java:446)
+Method <org.apache.flink.connector.base.source.hybrid.HybridSourceSplitEnumerator.checkAndSignalNoMoreSplits(org.apache.flink.api.connector.source.SplitEnumeratorContext, int, int, int)> calls method <org.apache.flink.util.Preconditions.checkState(boolean, java.lang.String, [Ljava.lang.Object;)> in (HybridSourceSplitEnumerator.java:438)
+Method <org.apache.flink.connector.base.source.hybrid.HybridSourceSplitEnumerator.checkAndSignalNoMoreSplits(org.apache.flink.api.connector.source.SplitEnumeratorContext, int, int, int)> checks instanceof <org.apache.flink.api.connector.source.SupportsIntermediateNoMoreSplits> in (HybridSourceSplitEnumerator.java:438)
 Method <org.apache.flink.connector.base.source.hybrid.HybridSourceSplitEnumerator.handleSplitRequest(int, java.lang.String)> calls method <org.apache.flink.util.Preconditions.checkState(boolean)> in (HybridSourceSplitEnumerator.java:110)
 Method <org.apache.flink.connector.base.source.hybrid.SwitchedSources.put(int, org.apache.flink.api.connector.source.Source)> calls method <org.apache.flink.util.Preconditions.checkNotNull(java.lang.Object)> in (SwitchedSources.java:48)
 Method <org.apache.flink.connector.base.source.hybrid.SwitchedSources.sourceOf(int)> calls method <org.apache.flink.util.Preconditions.checkNotNull(java.lang.Object, java.lang.String, [Ljava.lang.Object;)> in (SwitchedSources.java:38)
@@ -797,64 +796,6 @@
 Method <org.apache.flink.connector.file.table.stream.compact.CompactOperator.processElement(org.apache.flink.streaming.runtime.streamrecord.StreamRecord)> has generic parameter type <org.apache.flink.streaming.runtime.streamrecord.StreamRecord<org.apache.flink.connector.file.table.stream.compact.CompactMessages$CoordinatorOutput>> with type argument depending on <org.apache.flink.streaming.runtime.streamrecord.StreamRecord> in (CompactOperator.java:0)
 Method <org.apache.flink.connector.file.table.stream.compact.CompactOperator.processElement(org.apache.flink.streaming.runtime.streamrecord.StreamRecord)> has parameter of type <org.apache.flink.streaming.runtime.streamrecord.StreamRecord> in (CompactOperator.java:0)
 Method <org.apache.flink.connector.file.table.utils.CompactFileUtils.doSingleFileMove(org.apache.flink.core.fs.FileSystem, org.apache.flink.core.fs.Path, org.apache.flink.core.fs.Path)> calls method <org.apache.flink.util.IOUtils.copyBytes(java.io.InputStream, java.io.OutputStream, boolean)> in (CompactFileUtils.java:117)
-Method <org.apache.flink.connector.hbase.sink.HBaseSinkFunction.open(org.apache.flink.configuration.Configuration)> calls constructor <org.apache.flink.util.concurrent.ExecutorThreadFactory.<init>(java.lang.String)> in (HBaseSinkFunction.java:127)
-Method <org.apache.flink.connector.hbase.source.AbstractHBaseDynamicTableSource.getCache()> is annotated with <org.apache.flink.annotation.VisibleForTesting> in (AbstractHBaseDynamicTableSource.java:0)
-Method <org.apache.flink.connector.hbase.source.AbstractHBaseDynamicTableSource.getHBaseTableSchema()> is annotated with <org.apache.flink.annotation.VisibleForTesting> in (AbstractHBaseDynamicTableSource.java:0)
-Method <org.apache.flink.connector.hbase.source.AbstractHBaseDynamicTableSource.getLookupRuntimeProvider(org.apache.flink.table.connector.source.LookupTableSource$LookupContext)> calls method <org.apache.flink.util.Preconditions.checkArgument(boolean, java.lang.Object)> in (AbstractHBaseDynamicTableSource.java:79)
-Method <org.apache.flink.connector.hbase.source.AbstractHBaseDynamicTableSource.getLookupRuntimeProvider(org.apache.flink.table.connector.source.LookupTableSource$LookupContext)> calls method <org.apache.flink.util.Preconditions.checkArgument(boolean, java.lang.Object)> in (AbstractHBaseDynamicTableSource.java:82)
-Method <org.apache.flink.connector.hbase.source.AbstractHBaseDynamicTableSource.getLookupRuntimeProvider(org.apache.flink.table.connector.source.LookupTableSource$LookupContext)> calls method <org.apache.flink.util.Preconditions.checkArgument(boolean, java.lang.Object)> in (AbstractHBaseDynamicTableSource.java:85)
-Method <org.apache.flink.connector.hbase.source.AbstractHBaseDynamicTableSource.getMaxRetryTimes()> is annotated with <org.apache.flink.annotation.VisibleForTesting> in (AbstractHBaseDynamicTableSource.java:0)
-Method <org.apache.flink.connector.hbase.source.HBaseRowDataLookupFunction.getHTableName()> is annotated with <org.apache.flink.annotation.VisibleForTesting> in (HBaseRowDataLookupFunction.java:0)
-Method <org.apache.flink.connector.hbase.util.HBaseConfigurationUtil.deserializeWritable(org.apache.hadoop.io.Writable, [B)> calls method <org.apache.flink.util.Preconditions.checkArgument(boolean)> in (HBaseConfigurationUtil.java:173)
-Method <org.apache.flink.connector.hbase.util.HBaseConfigurationUtil.deserializeWritable(org.apache.hadoop.io.Writable, [B)> calls method <org.apache.flink.util.Preconditions.checkArgument(boolean)> in (HBaseConfigurationUtil.java:174)
-Method <org.apache.flink.connector.hbase.util.HBaseConfigurationUtil.serializeWritable(org.apache.hadoop.io.Writable)> calls method <org.apache.flink.util.Preconditions.checkArgument(boolean)> in (HBaseConfigurationUtil.java:155)
-Method <org.apache.flink.connector.hbase.util.HBaseSerde.convertToRow(org.apache.hadoop.hbase.client.Result, org.apache.flink.table.data.GenericRowData, [Lorg.apache.flink.table.data.GenericRowData;)> has parameter of type <[Lorg.apache.flink.table.data.GenericRowData;> in (HBaseSerde.java:0)
-Method <org.apache.flink.connector.hbase.util.HBaseSerde.createDeleteMutation(org.apache.flink.table.data.RowData)> calls method <org.apache.flink.util.Preconditions.checkArgument(boolean, java.lang.Object)> in (HBaseSerde.java:163)
-Method <org.apache.flink.connector.hbase.util.HBaseSerde.createFieldDecoder(org.apache.flink.table.types.logical.LogicalType)> calls method <org.apache.flink.table.types.logical.utils.LogicalTypeChecks.getPrecision(org.apache.flink.table.types.logical.LogicalType)> in (HBaseSerde.java:480)
-Method <org.apache.flink.connector.hbase.util.HBaseSerde.createFieldDecoder(org.apache.flink.table.types.logical.LogicalType)> calls method <org.apache.flink.table.types.logical.utils.LogicalTypeChecks.getPrecision(org.apache.flink.table.types.logical.LogicalType)> in (HBaseSerde.java:498)
-Method <org.apache.flink.connector.hbase.util.HBaseSerde.createFieldEncoder(org.apache.flink.table.types.logical.LogicalType)> calls method <org.apache.flink.table.types.logical.utils.LogicalTypeChecks.getPrecision(org.apache.flink.table.types.logical.LogicalType)> in (HBaseSerde.java:369)
-Method <org.apache.flink.connector.hbase.util.HBaseSerde.createFieldEncoder(org.apache.flink.table.types.logical.LogicalType)> calls method <org.apache.flink.table.types.logical.utils.LogicalTypeChecks.getPrecision(org.apache.flink.table.types.logical.LogicalType)> in (HBaseSerde.java:387)
-Method <org.apache.flink.connector.hbase.util.HBaseSerde.createGet(java.lang.Object)> calls method <org.apache.flink.util.Preconditions.checkArgument(boolean, java.lang.Object)> in (HBaseSerde.java:210)
-Method <org.apache.flink.connector.hbase.util.HBaseSerde.createPutMutation(org.apache.flink.table.data.RowData)> calls method <org.apache.flink.util.Preconditions.checkArgument(boolean, java.lang.Object)> in (HBaseSerde.java:131)
-Method <org.apache.flink.connector.hbase.util.HBaseStronglyIncreasingTsGenerator.getCurrentSystemTimeNano()> is annotated with <org.apache.flink.annotation.VisibleForTesting> in (HBaseStronglyIncreasingTsGenerator.java:0)
-Method <org.apache.flink.connector.hbase.util.HBaseTableSchema.addColumn(java.lang.String, java.lang.String, java.lang.Class)> calls method <org.apache.flink.table.types.utils.TypeConversions.fromLegacyInfoToDataType(org.apache.flink.api.common.typeinfo.TypeInformation)> in (HBaseTableSchema.java:68)
-Method <org.apache.flink.connector.hbase.util.HBaseTableSchema.addColumn(java.lang.String, java.lang.String, java.lang.Class)> calls method <org.apache.flink.util.Preconditions.checkNotNull(java.lang.Object, java.lang.String)> in (HBaseTableSchema.java:67)
-Method <org.apache.flink.connector.hbase.util.HBaseTableSchema.addColumn(java.lang.String, java.lang.String, org.apache.flink.table.types.DataType)> calls method <org.apache.flink.util.Preconditions.checkNotNull(java.lang.Object, java.lang.String)> in (HBaseTableSchema.java:73)
-Method <org.apache.flink.connector.hbase.util.HBaseTableSchema.addColumn(java.lang.String, java.lang.String, org.apache.flink.table.types.DataType)> calls method <org.apache.flink.util.Preconditions.checkNotNull(java.lang.Object, java.lang.String)> in (HBaseTableSchema.java:74)
-Method <org.apache.flink.connector.hbase.util.HBaseTableSchema.addColumn(java.lang.String, java.lang.String, org.apache.flink.table.types.DataType)> calls method <org.apache.flink.util.Preconditions.checkNotNull(java.lang.Object, java.lang.String)> in (HBaseTableSchema.java:75)
-Method <org.apache.flink.connector.hbase.util.HBaseTableSchema.convertToDataType()> calls method <org.apache.flink.shaded.guava31.com.google.common.collect.Streams.zip(java.util.stream.Stream, java.util.stream.Stream, java.util.function.BiFunction)> in (HBaseTableSchema.java:283)
-Method <org.apache.flink.connector.hbase.util.HBaseTableSchema.convertToDataType()> calls method <org.apache.flink.shaded.guava31.com.google.common.collect.Streams.zip(java.util.stream.Stream, java.util.stream.Stream, java.util.function.BiFunction)> in (HBaseTableSchema.java:298)
-Method <org.apache.flink.connector.hbase.util.HBaseTableSchema.fromDataType(org.apache.flink.table.types.DataType)> calls method <org.apache.flink.table.types.utils.TypeConversions.fromLogicalToDataType(org.apache.flink.table.types.logical.LogicalType)> in (HBaseTableSchema.java:335)
-Method <org.apache.flink.connector.hbase.util.HBaseTableSchema.fromDataType(org.apache.flink.table.types.DataType)> calls method <org.apache.flink.table.types.utils.TypeConversions.fromLogicalToDataType(org.apache.flink.table.types.logical.LogicalType)> in (HBaseTableSchema.java:338)
-Method <org.apache.flink.connector.hbase.util.HBaseTableSchema.getQualifierDataTypes(java.lang.String)> has return type <[Lorg.apache.flink.table.types.DataType;> in (HBaseTableSchema.java:0)
-Method <org.apache.flink.connector.hbase.util.HBaseTableSchema.getRowDataType([Ljava.lang.String;, [Lorg.apache.flink.table.types.DataType;)> has parameter of type <[Lorg.apache.flink.table.types.DataType;> in (HBaseTableSchema.java:0)
-Method <org.apache.flink.connector.hbase.util.HBaseTableSchema.getRowKeyTypeInfo()> calls method <org.apache.flink.table.types.utils.TypeConversions.fromDataTypeToLegacyInfo(org.apache.flink.table.types.DataType)> in (HBaseTableSchema.java:241)
-Method <org.apache.flink.connector.hbase.util.HBaseTableSchema.setRowKey(java.lang.String, java.lang.Class)> calls method <org.apache.flink.table.types.utils.TypeConversions.fromLegacyInfoToDataType(org.apache.flink.api.common.typeinfo.TypeInformation)> in (HBaseTableSchema.java:102)
-Method <org.apache.flink.connector.hbase.util.HBaseTableSchema.setRowKey(java.lang.String, java.lang.Class)> calls method <org.apache.flink.util.Preconditions.checkNotNull(java.lang.Object, java.lang.String)> in (HBaseTableSchema.java:101)
-Method <org.apache.flink.connector.hbase.util.HBaseTableSchema.setRowKey(java.lang.String, org.apache.flink.table.types.DataType)> calls method <org.apache.flink.util.Preconditions.checkNotNull(java.lang.Object, java.lang.String)> in (HBaseTableSchema.java:107)
-Method <org.apache.flink.connector.hbase.util.HBaseTableSchema.setRowKey(java.lang.String, org.apache.flink.table.types.DataType)> calls method <org.apache.flink.util.Preconditions.checkNotNull(java.lang.Object, java.lang.String)> in (HBaseTableSchema.java:108)
-Method <org.apache.flink.connector.hbase.util.HBaseTypeUtils.isSupportedType(org.apache.flink.table.types.logical.LogicalType)> calls method <org.apache.flink.table.types.logical.utils.LogicalTypeChecks.getPrecision(org.apache.flink.table.types.logical.LogicalType)> in (HBaseTypeUtils.java:190)
-Method <org.apache.flink.connector.hbase.util.HBaseTypeUtils.isSupportedType(org.apache.flink.table.types.logical.LogicalType)> calls method <org.apache.flink.table.types.logical.utils.LogicalTypeChecks.getPrecision(org.apache.flink.table.types.logical.LogicalType)> in (HBaseTypeUtils.java:201)
-Method <org.apache.flink.connector.hbase1.sink.HBaseDynamicTableSink.getConfiguration()> is annotated with <org.apache.flink.annotation.VisibleForTesting> in (HBaseDynamicTableSink.java:0)
-Method <org.apache.flink.connector.hbase1.sink.HBaseDynamicTableSink.getHBaseTableSchema()> is annotated with <org.apache.flink.annotation.VisibleForTesting> in (HBaseDynamicTableSink.java:0)
-Method <org.apache.flink.connector.hbase1.sink.HBaseDynamicTableSink.getTableName()> is annotated with <org.apache.flink.annotation.VisibleForTesting> in (HBaseDynamicTableSink.java:0)
-Method <org.apache.flink.connector.hbase1.sink.HBaseDynamicTableSink.getWriteOptions()> is annotated with <org.apache.flink.annotation.VisibleForTesting> in (HBaseDynamicTableSink.java:0)
-Method <org.apache.flink.connector.hbase1.source.AbstractTableInputFormat.createInputSplits(int)> has return type <[Lorg.apache.flink.core.io.InputSplit;> in (AbstractTableInputFormat.java:0)
-Method <org.apache.flink.connector.hbase1.source.AbstractTableInputFormat.getConnection()> is annotated with <org.apache.flink.annotation.VisibleForTesting> in (AbstractTableInputFormat.java:0)
-Method <org.apache.flink.connector.hbase1.source.AbstractTableInputFormat.getInputSplitAssigner([Lorg.apache.flink.core.io.InputSplit;)> has parameter of type <[Lorg.apache.flink.core.io.InputSplit;> in (AbstractTableInputFormat.java:0)
-Method <org.apache.flink.connector.hbase2.sink.HBaseDynamicTableSink.getConfiguration()> is annotated with <org.apache.flink.annotation.VisibleForTesting> in (HBaseDynamicTableSink.java:0)
-Method <org.apache.flink.connector.hbase2.sink.HBaseDynamicTableSink.getHBaseTableSchema()> is annotated with <org.apache.flink.annotation.VisibleForTesting> in (HBaseDynamicTableSink.java:0)
-Method <org.apache.flink.connector.hbase2.sink.HBaseDynamicTableSink.getTableName()> is annotated with <org.apache.flink.annotation.VisibleForTesting> in (HBaseDynamicTableSink.java:0)
-Method <org.apache.flink.connector.hbase2.sink.HBaseDynamicTableSink.getWriteOptions()> is annotated with <org.apache.flink.annotation.VisibleForTesting> in (HBaseDynamicTableSink.java:0)
-Method <org.apache.flink.connector.hbase2.source.AbstractTableInputFormat.close()> calls method <org.apache.flink.util.IOUtils.closeQuietly(java.lang.AutoCloseable)> in (AbstractTableInputFormat.java:201)
-Method <org.apache.flink.connector.hbase2.source.AbstractTableInputFormat.createInputSplits(int)> has return type <[Lorg.apache.flink.core.io.InputSplit;> in (AbstractTableInputFormat.java:0)
-Method <org.apache.flink.connector.hbase2.source.AbstractTableInputFormat.getConnection()> is annotated with <org.apache.flink.annotation.VisibleForTesting> in (AbstractTableInputFormat.java:0)
-Method <org.apache.flink.connector.hbase2.source.AbstractTableInputFormat.getInputSplitAssigner([Lorg.apache.flink.core.io.InputSplit;)> has parameter of type <[Lorg.apache.flink.core.io.InputSplit;> in (AbstractTableInputFormat.java:0)
-Method <org.apache.flink.connector.hbase2.source.HBaseDynamicTableSource.getLookupRuntimeProvider(org.apache.flink.table.connector.source.LookupTableSource$LookupContext)> calls method <org.apache.flink.util.Preconditions.checkArgument(boolean, java.lang.Object)> in (HBaseDynamicTableSource.java:63)
-Method <org.apache.flink.connector.hbase2.source.HBaseDynamicTableSource.getLookupRuntimeProvider(org.apache.flink.table.connector.source.LookupTableSource$LookupContext)> calls method <org.apache.flink.util.Preconditions.checkArgument(boolean, java.lang.Object)> in (HBaseDynamicTableSource.java:66)
-Method <org.apache.flink.connector.hbase2.source.HBaseDynamicTableSource.getLookupRuntimeProvider(org.apache.flink.table.connector.source.LookupTableSource$LookupContext)> calls method <org.apache.flink.util.Preconditions.checkArgument(boolean, java.lang.Object)> in (HBaseDynamicTableSource.java:69)
-Method <org.apache.flink.connector.hbase2.source.HBaseRowDataAsyncLookupFunction.getHTableName()> is annotated with <org.apache.flink.annotation.VisibleForTesting> in (HBaseRowDataAsyncLookupFunction.java:0)
-Method <org.apache.flink.connector.hbase2.source.HBaseRowDataAsyncLookupFunction.open(org.apache.flink.table.functions.FunctionContext)> calls constructor <org.apache.flink.util.concurrent.ExecutorThreadFactory.<init>(java.lang.String, java.lang.Thread$UncaughtExceptionHandler)> in (HBaseRowDataAsyncLookupFunction.java:96)
 Method <org.apache.flink.connector.print.table.PrintTableSinkFactory$RowDataPrintFunction.invoke(org.apache.flink.table.data.RowData, org.apache.flink.streaming.api.functions.sink.SinkFunction$Context)> calls method <org.apache.flink.api.common.functions.util.PrintSinkOutputWriter.write(java.lang.Object)> in (PrintTableSinkFactory.java:187)
 Method <org.apache.flink.connector.print.table.PrintTableSinkFactory$RowDataPrintFunction.open(org.apache.flink.configuration.Configuration)> calls method <org.apache.flink.api.common.functions.util.PrintSinkOutputWriter.open(int, int)> in (PrintTableSinkFactory.java:180)
 Method <org.apache.flink.connector.print.table.PrintTableSinkFactory$RowDataPrintFunction.open(org.apache.flink.configuration.Configuration)> calls method <org.apache.flink.streaming.api.operators.StreamingRuntimeContext.getIndexOfThisSubtask()> in (PrintTableSinkFactory.java:180)
diff --git a/flink-architecture-tests/flink-architecture-tests-production/pom.xml b/flink-architecture-tests/flink-architecture-tests-production/pom.xml
index 7e4fde8..fc0bd3c 100644
--- a/flink-architecture-tests/flink-architecture-tests-production/pom.xml
+++ b/flink-architecture-tests/flink-architecture-tests-production/pom.xml
@@ -136,21 +136,6 @@
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-connector-hbase-1.4</artifactId>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-connector-hbase-2.2</artifactId>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-connector-hbase-base</artifactId>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
 			<artifactId>flink-connector-hive_${scala.binary.version}</artifactId>
 		</dependency>
 	</dependencies>
diff --git a/flink-architecture-tests/pom.xml b/flink-architecture-tests/pom.xml
index e21adf5..34ffe55 100644
--- a/flink-architecture-tests/pom.xml
+++ b/flink-architecture-tests/pom.xml
@@ -148,27 +148,6 @@
 
 			<dependency>
 				<groupId>org.apache.flink</groupId>
-				<artifactId>flink-connector-hbase-1.4</artifactId>
-				<version>${project.version}</version>
-				<scope>test</scope>
-			</dependency>
-
-			<dependency>
-				<groupId>org.apache.flink</groupId>
-				<artifactId>flink-connector-hbase-2.2</artifactId>
-				<version>${project.version}</version>
-				<scope>test</scope>
-			</dependency>
-
-			<dependency>
-				<groupId>org.apache.flink</groupId>
-				<artifactId>flink-connector-hbase-base</artifactId>
-				<version>${project.version}</version>
-				<scope>test</scope>
-			</dependency>
-
-			<dependency>
-				<groupId>org.apache.flink</groupId>
 				<artifactId>flink-connector-hive_${scala.binary.version}</artifactId>
 				<version>${project.version}</version>
 				<scope>test</scope>
diff --git a/flink-connectors/flink-connector-hbase-1.4/archunit-violations/3ba92d3a-6609-4295-92ed-f2fe207ee2b3 b/flink-connectors/flink-connector-hbase-1.4/archunit-violations/3ba92d3a-6609-4295-92ed-f2fe207ee2b3
deleted file mode 100644
index e69de29..0000000
--- a/flink-connectors/flink-connector-hbase-1.4/archunit-violations/3ba92d3a-6609-4295-92ed-f2fe207ee2b3
+++ /dev/null
diff --git a/flink-connectors/flink-connector-hbase-1.4/archunit-violations/ffbddcc3-857a-4af7-a6b5-fcf71e2cc191 b/flink-connectors/flink-connector-hbase-1.4/archunit-violations/ffbddcc3-857a-4af7-a6b5-fcf71e2cc191
deleted file mode 100644
index e69de29..0000000
--- a/flink-connectors/flink-connector-hbase-1.4/archunit-violations/ffbddcc3-857a-4af7-a6b5-fcf71e2cc191
+++ /dev/null
diff --git a/flink-connectors/flink-connector-hbase-1.4/archunit-violations/stored.rules b/flink-connectors/flink-connector-hbase-1.4/archunit-violations/stored.rules
deleted file mode 100644
index 0dcd726..0000000
--- a/flink-connectors/flink-connector-hbase-1.4/archunit-violations/stored.rules
+++ /dev/null
@@ -1,4 +0,0 @@
-#
-#Tue Feb 22 12:17:39 CET 2022
-Tests\ inheriting\ from\ AbstractTestBase\ should\ have\ name\ ending\ with\ ITCase=ffbddcc3-857a-4af7-a6b5-fcf71e2cc191
-ITCASE\ tests\ should\ use\ a\ MiniCluster\ resource\ or\ extension=3ba92d3a-6609-4295-92ed-f2fe207ee2b3
diff --git a/flink-connectors/flink-connector-hbase-1.4/pom.xml b/flink-connectors/flink-connector-hbase-1.4/pom.xml
deleted file mode 100644
index fbe105d..0000000
--- a/flink-connectors/flink-connector-hbase-1.4/pom.xml
+++ /dev/null
@@ -1,413 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-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.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
-
-	<modelVersion>4.0.0</modelVersion>
-
-	<parent>
-		<groupId>org.apache.flink</groupId>
-		<artifactId>flink-connectors</artifactId>
-		<version>1.19-SNAPSHOT</version>
-	</parent>
-
-	<artifactId>flink-connector-hbase-1.4</artifactId>
-	<name>Flink : Connectors : HBase 1.4</name>
-	<packaging>jar</packaging>
-
-	<properties>
-		<hbase.version>1.4.3</hbase.version>
-	</properties>
-
-	<dependencies>
-
-		<!-- Core -->
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-core</artifactId>
-			<version>${project.version}</version>
-			<scope>provided</scope>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-java</artifactId>
-			<version>${project.version}</version>
-			<scope>provided</scope>
-		</dependency>
-
-		<!-- Flink HBase -->
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-connector-hbase-base</artifactId>
-			<version>${project.version}</version>
-		</dependency>
-
-		<!-- Table ecosystem -->
-
-		<!-- Projects depending on this project won't depend on flink-table-*. -->
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-table-api-java-bridge</artifactId>
-			<version>${project.version}</version>
-			<scope>provided</scope>
-			<optional>true</optional>
-		</dependency>
-
-		<!-- Tests -->
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-connector-hbase-base</artifactId>
-			<version>${project.version}</version>
-			<type>test-jar</type>
-			<scope>test</scope>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-hadoop-compatibility_${scala.binary.version}</artifactId>
-			<version>${project.version}</version>
-			<scope>test</scope>
-			<exclusions>
-				<exclusion>
-					<groupId>org.apache.flink</groupId>
-					<artifactId>flink-shaded-include-yarn_${scala.binary.version}</artifactId>
-				</exclusion>
-			</exclusions>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-test-utils</artifactId>
-			<version>${project.version}</version>
-			<scope>test</scope>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-table-common</artifactId>
-			<version>${project.version}</version>
-			<type>test-jar</type>
-			<scope>test</scope>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-table-planner_${scala.binary.version}</artifactId>
-			<version>${project.version}</version>
-			<scope>test</scope>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-table-api-scala-bridge_${scala.binary.version}</artifactId>
-			<version>${project.version}</version>
-			<scope>test</scope>
-		</dependency>
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-table-planner_${scala.binary.version}</artifactId>
-			<version>${project.version}</version>
-			<type>test-jar</type>
-			<scope>test</scope>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.hbase</groupId>
-			<artifactId>hbase-server</artifactId>
-			<version>${hbase.version}</version>
-			<type>test-jar</type>
-			<scope>test</scope>
-			<exclusions>
-				<exclusion>
-					<groupId>log4j</groupId>
-					<artifactId>log4j</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.slf4j</groupId>
-					<artifactId>slf4j-log4j12</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.apache.hadoop</groupId>
-					<artifactId>hadoop-auth</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>com.google.guava</groupId>
-					<artifactId>guava</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>ch.qos.reload4j</groupId>
-					<artifactId>reload4j</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.slf4j</groupId>
-					<artifactId>slf4j-reload4j</artifactId>
-				</exclusion>
-			</exclusions>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.hbase</groupId>
-			<artifactId>hbase-server</artifactId>
-			<version>${hbase.version}</version>
-			<scope>test</scope>
-			<exclusions>
-				<!-- Remove unneeded dependency, which is conflicting with our jetty-util version. -->
-				<exclusion>
-					<groupId>org.mortbay.jetty</groupId>
-					<artifactId>jetty-util</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.mortbay.jetty</groupId>
-					<artifactId>jetty</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.mortbay.jetty</groupId>
-					<artifactId>jetty-sslengine</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.mortbay.jetty</groupId>
-					<artifactId>jsp-2.1</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.mortbay.jetty</groupId>
-					<artifactId>jsp-api-2.1</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.mortbay.jetty</groupId>
-					<artifactId>servlet-api-2.5</artifactId>
-				</exclusion>
-				<!-- Bug in hbase annotations, can be removed when fixed. See FLINK-2153. -->
-				<exclusion>
-					<groupId>org.apache.hbase</groupId>
-					<artifactId>hbase-annotations</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>com.sun.jersey</groupId>
-					<artifactId>jersey-core</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>com.sun.jersey</groupId>
-					<artifactId>jersey-server</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>tomcat</groupId>
-					<artifactId>jasper-compiler</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>tomcat</groupId>
-					<artifactId>jasper-runtime</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.jruby.jcodings</groupId>
-					<artifactId>jcodings</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.jruby.joni</groupId>
-					<artifactId>joni</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.jamon</groupId>
-					<artifactId>jamon-runtime</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>log4j</groupId>
-					<artifactId>log4j</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.slf4j</groupId>
-					<artifactId>slf4j-log4j12</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.apache.hadoop</groupId>
-					<artifactId>hadoop-auth</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>com.google.guava</groupId>
-					<artifactId>guava</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>ch.qos.reload4j</groupId>
-					<artifactId>reload4j</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.slf4j</groupId>
-					<artifactId>slf4j-reload4j</artifactId>
-				</exclusion>
-			</exclusions>
-		</dependency>
-
-		<dependency>
-			<!-- Bump hbase netty dependency -->
-			<groupId>io.netty</groupId>
-			<artifactId>netty-all</artifactId>
-			<version>4.1.46.Final</version>
-		</dependency>
-
-		<!--
-			We declare this test-classified dependency here to move it into our
-			"test" scope. Otherwise it would end up in proudction jars.
-			This can probably be fixed once we rely on higher HBase versions which
-			have fixed this problem.
-		-->
-		<dependency>
-			<groupId>org.apache.hbase</groupId>
-			<artifactId>hbase-common</artifactId>
-			<version>${hbase.version}</version>
-			<classifier>tests</classifier>
-			<scope>test</scope>
-			<exclusions>
-				<exclusion>
-					<groupId>log4j</groupId>
-					<artifactId>log4j</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>com.google.guava</groupId>
-					<artifactId>guava</artifactId>
-				</exclusion>
-			</exclusions>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.hadoop</groupId>
-			<artifactId>hadoop-minicluster</artifactId>
-			<version>${flink.hadoop.version}</version>
-			<scope>test</scope>
-			<exclusions>
-				<exclusion>
-					<groupId>log4j</groupId>
-					<artifactId>log4j</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.slf4j</groupId>
-					<artifactId>slf4j-log4j12</artifactId>
-				</exclusion>
-				<exclusion>
-					<!-- This dependency is no longer shipped with the JDK since Java 9.-->
-					<groupId>jdk.tools</groupId>
-					<artifactId>jdk.tools</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>ch.qos.reload4j</groupId>
-					<artifactId>reload4j</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.slf4j</groupId>
-					<artifactId>slf4j-reload4j</artifactId>
-				</exclusion>
-			</exclusions>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.hbase</groupId>
-			<artifactId>hbase-hadoop-compat</artifactId>
-			<version>${hbase.version}</version>
-			<scope>test</scope>
-			<type>test-jar</type>
-			<exclusions>
-				<exclusion>
-					<groupId>log4j</groupId>
-					<artifactId>log4j</artifactId>
-				</exclusion>
-			</exclusions>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.hadoop</groupId>
-			<artifactId>hadoop-hdfs</artifactId>
-			<version>${flink.hadoop.version}</version>
-			<type>test-jar</type>
-			<scope>test</scope>
-			<exclusions>
-				<exclusion>
-					<groupId>log4j</groupId>
-					<artifactId>log4j</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>ch.qos.reload4j</groupId>
-					<artifactId>reload4j</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.slf4j</groupId>
-					<artifactId>slf4j-reload4j</artifactId>
-				</exclusion>
-			</exclusions>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.hbase</groupId>
-			<artifactId>hbase-hadoop2-compat</artifactId>
-			<version>${hbase.version}</version>
-			<scope>test</scope>
-			<type>test-jar</type>
-			<exclusions>
-				<exclusion>
-					<groupId>log4j</groupId>
-					<artifactId>log4j</artifactId>
-				</exclusion>
-			</exclusions>
-		</dependency>
-
-		<!-- ArchUit test dependencies -->
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-architecture-tests-test</artifactId>
-			<scope>test</scope>
-		</dependency>
-	</dependencies>
-
-	<dependencyManagement>
-		<dependencies>
-			<dependency>
-				<!-- HBase only works with Zookeper 3.4 -->
-				<groupId>org.apache.zookeeper</groupId>
-				<artifactId>zookeeper</artifactId>
-				<version>3.4.14</version>
-			</dependency>
-		</dependencies>
-	</dependencyManagement>
-
-	<profiles>
-		<profile>
-			<id>java11</id>
-			<activation>
-				<jdk>[11,)</jdk>
-			</activation>
-
-			<build>
-				<plugins>
-					<plugin>
-						<groupId>org.apache.maven.plugins</groupId>
-						<artifactId>maven-surefire-plugin</artifactId>
-						<configuration>
-							<!-- hbase currently does not support Java 11, see HBASE-21110 -->
-							<skip>true</skip>
-						</configuration>
-					</plugin>
-				</plugins>
-			</build>
-		</profile>
-	</profiles>
-
-</project>
diff --git a/flink-connectors/flink-connector-hbase-1.4/src/main/java/org/apache/flink/connector/hbase1/HBase1DynamicTableFactory.java b/flink-connectors/flink-connector-hbase-1.4/src/main/java/org/apache/flink/connector/hbase1/HBase1DynamicTableFactory.java
deleted file mode 100644
index fbc793c..0000000
--- a/flink-connectors/flink-connector-hbase-1.4/src/main/java/org/apache/flink/connector/hbase1/HBase1DynamicTableFactory.java
+++ /dev/null
@@ -1,181 +0,0 @@
-/*
- * 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.hbase1;
-
-import org.apache.flink.annotation.Internal;
-import org.apache.flink.configuration.ConfigOption;
-import org.apache.flink.configuration.ReadableConfig;
-import org.apache.flink.connector.hbase.options.HBaseWriteOptions;
-import org.apache.flink.connector.hbase.util.HBaseTableSchema;
-import org.apache.flink.connector.hbase1.sink.HBaseDynamicTableSink;
-import org.apache.flink.connector.hbase1.source.HBaseDynamicTableSource;
-import org.apache.flink.table.connector.sink.DynamicTableSink;
-import org.apache.flink.table.connector.source.DynamicTableSource;
-import org.apache.flink.table.connector.source.lookup.LookupOptions;
-import org.apache.flink.table.connector.source.lookup.cache.DefaultLookupCache;
-import org.apache.flink.table.connector.source.lookup.cache.LookupCache;
-import org.apache.flink.table.factories.DynamicTableSinkFactory;
-import org.apache.flink.table.factories.DynamicTableSourceFactory;
-import org.apache.flink.table.factories.FactoryUtil.TableFactoryHelper;
-
-import org.apache.hadoop.conf.Configuration;
-
-import java.time.Duration;
-import java.util.HashSet;
-import java.util.Set;
-import java.util.stream.Collectors;
-import java.util.stream.Stream;
-
-import static org.apache.flink.connector.hbase.table.HBaseConnectorOptions.LOOKUP_ASYNC;
-import static org.apache.flink.connector.hbase.table.HBaseConnectorOptions.LOOKUP_CACHE_MAX_ROWS;
-import static org.apache.flink.connector.hbase.table.HBaseConnectorOptions.LOOKUP_CACHE_TTL;
-import static org.apache.flink.connector.hbase.table.HBaseConnectorOptions.LOOKUP_MAX_RETRIES;
-import static org.apache.flink.connector.hbase.table.HBaseConnectorOptions.NULL_STRING_LITERAL;
-import static org.apache.flink.connector.hbase.table.HBaseConnectorOptions.SINK_BUFFER_FLUSH_INTERVAL;
-import static org.apache.flink.connector.hbase.table.HBaseConnectorOptions.SINK_BUFFER_FLUSH_MAX_ROWS;
-import static org.apache.flink.connector.hbase.table.HBaseConnectorOptions.SINK_BUFFER_FLUSH_MAX_SIZE;
-import static org.apache.flink.connector.hbase.table.HBaseConnectorOptions.SINK_PARALLELISM;
-import static org.apache.flink.connector.hbase.table.HBaseConnectorOptions.TABLE_NAME;
-import static org.apache.flink.connector.hbase.table.HBaseConnectorOptions.ZOOKEEPER_QUORUM;
-import static org.apache.flink.connector.hbase.table.HBaseConnectorOptions.ZOOKEEPER_ZNODE_PARENT;
-import static org.apache.flink.connector.hbase.table.HBaseConnectorOptionsUtil.PROPERTIES_PREFIX;
-import static org.apache.flink.connector.hbase.table.HBaseConnectorOptionsUtil.getHBaseConfiguration;
-import static org.apache.flink.connector.hbase.table.HBaseConnectorOptionsUtil.getHBaseWriteOptions;
-import static org.apache.flink.connector.hbase.table.HBaseConnectorOptionsUtil.validatePrimaryKey;
-import static org.apache.flink.table.factories.FactoryUtil.createTableFactoryHelper;
-
-/** HBase connector factory. */
-@Internal
-public class HBase1DynamicTableFactory
-        implements DynamicTableSourceFactory, DynamicTableSinkFactory {
-
-    private static final String IDENTIFIER = "hbase-1.4";
-
-    @Override
-    public DynamicTableSource createDynamicTableSource(Context context) {
-        TableFactoryHelper helper = createTableFactoryHelper(this, context);
-        helper.validateExcept(PROPERTIES_PREFIX);
-
-        final ReadableConfig tableOptions = helper.getOptions();
-
-        validatePrimaryKey(context.getPhysicalRowDataType(), context.getPrimaryKeyIndexes());
-
-        String tableName = tableOptions.get(TABLE_NAME);
-        Configuration hbaseClientConf = getHBaseConfiguration(tableOptions);
-        String nullStringLiteral = tableOptions.get(NULL_STRING_LITERAL);
-        HBaseTableSchema hbaseSchema =
-                HBaseTableSchema.fromDataType(context.getPhysicalRowDataType());
-        LookupCache cache = null;
-
-        // Backward compatible to legacy caching options
-        if (tableOptions.get(LOOKUP_CACHE_MAX_ROWS) > 0
-                && tableOptions.get(LOOKUP_CACHE_TTL).compareTo(Duration.ZERO) > 0) {
-            cache =
-                    DefaultLookupCache.newBuilder()
-                            .maximumSize(tableOptions.get(LOOKUP_CACHE_MAX_ROWS))
-                            .expireAfterWrite(tableOptions.get(LOOKUP_CACHE_TTL))
-                            .build();
-        }
-
-        if (tableOptions
-                .get(LookupOptions.CACHE_TYPE)
-                .equals(LookupOptions.LookupCacheType.PARTIAL)) {
-            cache = DefaultLookupCache.fromConfig(tableOptions);
-        }
-
-        return new HBaseDynamicTableSource(
-                hbaseClientConf,
-                tableName,
-                hbaseSchema,
-                nullStringLiteral,
-                tableOptions.get(LookupOptions.MAX_RETRIES),
-                cache);
-    }
-
-    @Override
-    public DynamicTableSink createDynamicTableSink(Context context) {
-        TableFactoryHelper helper = createTableFactoryHelper(this, context);
-        helper.validateExcept(PROPERTIES_PREFIX);
-
-        final ReadableConfig tableOptions = helper.getOptions();
-
-        validatePrimaryKey(context.getPhysicalRowDataType(), context.getPrimaryKeyIndexes());
-
-        String tableName = tableOptions.get(TABLE_NAME);
-        Configuration hbaseConf = getHBaseConfiguration(tableOptions);
-        HBaseWriteOptions hBaseWriteOptions = getHBaseWriteOptions(tableOptions);
-        String nullStringLiteral = tableOptions.get(NULL_STRING_LITERAL);
-        HBaseTableSchema hbaseSchema =
-                HBaseTableSchema.fromDataType(context.getPhysicalRowDataType());
-
-        return new HBaseDynamicTableSink(
-                tableName, hbaseSchema, hbaseConf, hBaseWriteOptions, nullStringLiteral);
-    }
-
-    @Override
-    public String factoryIdentifier() {
-        return IDENTIFIER;
-    }
-
-    @Override
-    public Set<ConfigOption<?>> requiredOptions() {
-        Set<ConfigOption<?>> set = new HashSet<>();
-        set.add(TABLE_NAME);
-        set.add(ZOOKEEPER_QUORUM);
-        return set;
-    }
-
-    @Override
-    public Set<ConfigOption<?>> optionalOptions() {
-        Set<ConfigOption<?>> set = new HashSet<>();
-        set.add(ZOOKEEPER_ZNODE_PARENT);
-        set.add(NULL_STRING_LITERAL);
-        set.add(SINK_BUFFER_FLUSH_MAX_SIZE);
-        set.add(SINK_BUFFER_FLUSH_MAX_ROWS);
-        set.add(SINK_BUFFER_FLUSH_INTERVAL);
-        set.add(SINK_PARALLELISM);
-        set.add(LOOKUP_ASYNC);
-        set.add(LOOKUP_CACHE_MAX_ROWS);
-        set.add(LOOKUP_CACHE_TTL);
-        set.add(LOOKUP_MAX_RETRIES);
-        set.add(LookupOptions.CACHE_TYPE);
-        set.add(LookupOptions.MAX_RETRIES);
-        set.add(LookupOptions.PARTIAL_CACHE_EXPIRE_AFTER_ACCESS);
-        set.add(LookupOptions.PARTIAL_CACHE_EXPIRE_AFTER_WRITE);
-        set.add(LookupOptions.PARTIAL_CACHE_CACHE_MISSING_KEY);
-        set.add(LookupOptions.PARTIAL_CACHE_MAX_ROWS);
-        return set;
-    }
-
-    @Override
-    public Set<ConfigOption<?>> forwardOptions() {
-        return Stream.of(
-                        TABLE_NAME,
-                        ZOOKEEPER_ZNODE_PARENT,
-                        ZOOKEEPER_QUORUM,
-                        NULL_STRING_LITERAL,
-                        SINK_BUFFER_FLUSH_MAX_SIZE,
-                        SINK_BUFFER_FLUSH_MAX_ROWS,
-                        SINK_BUFFER_FLUSH_INTERVAL,
-                        LOOKUP_CACHE_MAX_ROWS,
-                        LOOKUP_CACHE_TTL,
-                        LOOKUP_MAX_RETRIES)
-                .collect(Collectors.toSet());
-    }
-}
diff --git a/flink-connectors/flink-connector-hbase-1.4/src/main/java/org/apache/flink/connector/hbase1/sink/HBaseDynamicTableSink.java b/flink-connectors/flink-connector-hbase-1.4/src/main/java/org/apache/flink/connector/hbase1/sink/HBaseDynamicTableSink.java
deleted file mode 100644
index 2b9e87c..0000000
--- a/flink-connectors/flink-connector-hbase-1.4/src/main/java/org/apache/flink/connector/hbase1/sink/HBaseDynamicTableSink.java
+++ /dev/null
@@ -1,115 +0,0 @@
-/*
- * 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.hbase1.sink;
-
-import org.apache.flink.annotation.Internal;
-import org.apache.flink.annotation.VisibleForTesting;
-import org.apache.flink.connector.hbase.options.HBaseWriteOptions;
-import org.apache.flink.connector.hbase.sink.HBaseSinkFunction;
-import org.apache.flink.connector.hbase.sink.RowDataToMutationConverter;
-import org.apache.flink.connector.hbase.util.HBaseTableSchema;
-import org.apache.flink.table.connector.ChangelogMode;
-import org.apache.flink.table.connector.sink.DynamicTableSink;
-import org.apache.flink.table.connector.sink.SinkFunctionProvider;
-import org.apache.flink.table.data.RowData;
-import org.apache.flink.types.RowKind;
-
-import org.apache.hadoop.conf.Configuration;
-
-/** HBase table sink implementation. */
-@Internal
-public class HBaseDynamicTableSink implements DynamicTableSink {
-
-    private final HBaseTableSchema hbaseTableSchema;
-    private final String nullStringLiteral;
-    private final Configuration hbaseConf;
-    private final HBaseWriteOptions writeOptions;
-    private final String tableName;
-
-    public HBaseDynamicTableSink(
-            String tableName,
-            HBaseTableSchema hbaseTableSchema,
-            Configuration hbaseConf,
-            HBaseWriteOptions writeOptions,
-            String nullStringLiteral) {
-        this.hbaseTableSchema = hbaseTableSchema;
-        this.nullStringLiteral = nullStringLiteral;
-        this.hbaseConf = hbaseConf;
-        this.writeOptions = writeOptions;
-        this.tableName = tableName;
-    }
-
-    @Override
-    public SinkRuntimeProvider getSinkRuntimeProvider(Context context) {
-        HBaseSinkFunction<RowData> sinkFunction =
-                new HBaseSinkFunction<>(
-                        tableName,
-                        hbaseConf,
-                        new RowDataToMutationConverter(hbaseTableSchema, nullStringLiteral),
-                        writeOptions.getBufferFlushMaxSizeInBytes(),
-                        writeOptions.getBufferFlushMaxRows(),
-                        writeOptions.getBufferFlushIntervalMillis());
-        return SinkFunctionProvider.of(sinkFunction, writeOptions.getParallelism());
-    }
-
-    @Override
-    public ChangelogMode getChangelogMode(ChangelogMode requestedMode) {
-        // UPSERT mode
-        ChangelogMode.Builder builder = ChangelogMode.newBuilder();
-        for (RowKind kind : requestedMode.getContainedKinds()) {
-            if (kind != RowKind.UPDATE_BEFORE) {
-                builder.addContainedKind(kind);
-            }
-        }
-        return builder.build();
-    }
-
-    @Override
-    public DynamicTableSink copy() {
-        return new HBaseDynamicTableSink(
-                tableName, hbaseTableSchema, hbaseConf, writeOptions, nullStringLiteral);
-    }
-
-    @Override
-    public String asSummaryString() {
-        return "HBase";
-    }
-
-    // -------------------------------------------------------------------------------------------
-
-    @VisibleForTesting
-    public HBaseTableSchema getHBaseTableSchema() {
-        return this.hbaseTableSchema;
-    }
-
-    @VisibleForTesting
-    public HBaseWriteOptions getWriteOptions() {
-        return writeOptions;
-    }
-
-    @VisibleForTesting
-    public Configuration getConfiguration() {
-        return this.hbaseConf;
-    }
-
-    @VisibleForTesting
-    public String getTableName() {
-        return this.tableName;
-    }
-}
diff --git a/flink-connectors/flink-connector-hbase-1.4/src/main/java/org/apache/flink/connector/hbase1/source/AbstractTableInputFormat.java b/flink-connectors/flink-connector-hbase-1.4/src/main/java/org/apache/flink/connector/hbase1/source/AbstractTableInputFormat.java
deleted file mode 100644
index 6b60adc..0000000
--- a/flink-connectors/flink-connector-hbase-1.4/src/main/java/org/apache/flink/connector/hbase1/source/AbstractTableInputFormat.java
+++ /dev/null
@@ -1,320 +0,0 @@
-/*
- * 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.hbase1.source;
-
-import org.apache.flink.annotation.Internal;
-import org.apache.flink.annotation.VisibleForTesting;
-import org.apache.flink.api.common.io.InputFormat;
-import org.apache.flink.api.common.io.LocatableInputSplitAssigner;
-import org.apache.flink.api.common.io.RichInputFormat;
-import org.apache.flink.api.common.io.statistics.BaseStatistics;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.connector.hbase.source.TableInputSplit;
-import org.apache.flink.connector.hbase.util.HBaseConfigurationUtil;
-import org.apache.flink.core.io.InputSplitAssigner;
-
-import org.apache.hadoop.hbase.client.Connection;
-import org.apache.hadoop.hbase.client.HTable;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.ResultScanner;
-import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.Pair;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-
-/** Abstract {@link InputFormat} to read data from HBase tables. */
-@Internal
-public abstract class AbstractTableInputFormat<T> extends RichInputFormat<T, TableInputSplit> {
-
-    protected static final Logger LOG = LoggerFactory.getLogger(AbstractTableInputFormat.class);
-    private static final long serialVersionUID = 1L;
-
-    // helper variable to decide whether the input is exhausted or not
-    protected boolean endReached = false;
-
-    protected transient Connection connection = null;
-    protected transient HTable table = null;
-    protected transient Scan scan = null;
-
-    /** HBase iterator wrapper. */
-    protected ResultScanner resultScanner = null;
-
-    protected byte[] currentRow;
-    protected long scannedRows;
-
-    // Configuration is not serializable
-    protected byte[] serializedConfig;
-
-    public AbstractTableInputFormat(org.apache.hadoop.conf.Configuration hConf) {
-        serializedConfig = HBaseConfigurationUtil.serializeConfiguration(hConf);
-    }
-
-    /**
-     * Creates a {@link Scan} object and opens the {@link HTable} connection to initialize the HBase
-     * table.
-     *
-     * @throws IOException Thrown, if the connection could not be opened due to an I/O problem.
-     */
-    protected abstract void initTable() throws IOException;
-
-    /**
-     * Returns an instance of Scan that retrieves the required subset of records from the HBase
-     * table.
-     *
-     * @return The appropriate instance of Scan for this use case.
-     */
-    protected abstract Scan getScanner();
-
-    /**
-     * What table is to be read.
-     *
-     * <p>Per instance of a TableInputFormat derivative only a single table name is possible.
-     *
-     * @return The name of the table
-     */
-    protected abstract String getTableName();
-
-    /**
-     * HBase returns an instance of {@link Result}.
-     *
-     * <p>This method maps the returned {@link Result} instance into the output type {@link T}.
-     *
-     * @param r The Result instance from HBase that needs to be converted
-     * @return The appropriate instance of {@link T} that contains the data of Result.
-     */
-    protected abstract T mapResultToOutType(Result r);
-
-    @Override
-    public void configure(Configuration parameters) {}
-
-    protected org.apache.hadoop.conf.Configuration getHadoopConfiguration() {
-        return HBaseConfigurationUtil.deserializeConfiguration(
-                serializedConfig, HBaseConfigurationUtil.getHBaseConfiguration());
-    }
-
-    /**
-     * Creates a {@link Scan} object and opens the {@link HTable} connection. The connection is
-     * opened in this method and closed in {@link #close()}.
-     *
-     * @param split The split to be opened.
-     * @throws IOException Thrown, if the spit could not be opened due to an I/O problem.
-     */
-    @Override
-    public void open(TableInputSplit split) throws IOException {
-        initTable();
-
-        if (split == null) {
-            throw new IOException("Input split is null!");
-        }
-
-        logSplitInfo("opening", split);
-
-        // set scan range
-        currentRow = split.getStartRow();
-        scan.setStartRow(currentRow);
-        scan.setStopRow(split.getEndRow());
-
-        resultScanner = table.getScanner(scan);
-        endReached = false;
-        scannedRows = 0;
-    }
-
-    public T nextRecord(T reuse) throws IOException {
-        if (resultScanner == null) {
-            throw new IOException("No table result scanner provided!");
-        }
-        Result res;
-        try {
-            res = resultScanner.next();
-        } catch (Exception e) {
-            resultScanner.close();
-            // workaround for timeout on scan
-            LOG.warn(
-                    "Error after scan of " + scannedRows + " rows. Retry with a new scanner...", e);
-            scan.withStartRow(currentRow, false);
-            resultScanner = table.getScanner(scan);
-            res = resultScanner.next();
-        }
-
-        if (res != null) {
-            scannedRows++;
-            currentRow = res.getRow();
-            return mapResultToOutType(res);
-        }
-
-        endReached = true;
-        return null;
-    }
-
-    private void logSplitInfo(String action, TableInputSplit split) {
-        int splitId = split.getSplitNumber();
-        String splitStart = Bytes.toString(split.getStartRow());
-        String splitEnd = Bytes.toString(split.getEndRow());
-        String splitStartKey = splitStart.isEmpty() ? "-" : splitStart;
-        String splitStopKey = splitEnd.isEmpty() ? "-" : splitEnd;
-        String[] hostnames = split.getHostnames();
-        LOG.info(
-                "{} split (this={})[{}|{}|{}|{}]",
-                action,
-                this,
-                splitId,
-                hostnames,
-                splitStartKey,
-                splitStopKey);
-    }
-
-    @Override
-    public boolean reachedEnd() throws IOException {
-        return endReached;
-    }
-
-    @Override
-    public void close() throws IOException {
-        LOG.info("Closing split (scanned {} rows)", scannedRows);
-        currentRow = null;
-        try {
-            if (resultScanner != null) {
-                resultScanner.close();
-            }
-            closeTable();
-        } finally {
-            resultScanner = null;
-        }
-    }
-
-    public void closeTable() {
-        if (table != null) {
-            try {
-                table.close();
-            } catch (IOException e) {
-                LOG.warn("Exception occurs while closing HBase Table.", e);
-            }
-            table = null;
-        }
-        if (connection != null) {
-            try {
-                connection.close();
-            } catch (IOException e) {
-                LOG.warn("Exception occurs while closing HBase Connection.", e);
-            }
-            connection = null;
-        }
-    }
-
-    @Override
-    public TableInputSplit[] createInputSplits(final int minNumSplits) throws IOException {
-        try {
-            initTable();
-
-            // Get the starting and ending row keys for every region in the currently open table
-            final Pair<byte[][], byte[][]> keys = table.getRegionLocator().getStartEndKeys();
-            if (keys == null || keys.getFirst() == null || keys.getFirst().length == 0) {
-                LOG.warn(
-                        "Unexpected region keys: {} appeared in HBase table: {}, all region information are: {}.",
-                        keys,
-                        table,
-                        table.getRegionLocator().getAllRegionLocations());
-                throw new IOException(
-                        "HBase Table expects at least one region in scan,"
-                                + " please check the HBase table status in HBase cluster");
-            }
-            final byte[] startRow = scan.getStartRow();
-            final byte[] stopRow = scan.getStopRow();
-            final boolean scanWithNoLowerBound = startRow.length == 0;
-            final boolean scanWithNoUpperBound = stopRow.length == 0;
-
-            final List<TableInputSplit> splits = new ArrayList<>(minNumSplits);
-            for (int i = 0; i < keys.getFirst().length; i++) {
-                final byte[] startKey = keys.getFirst()[i];
-                final byte[] endKey = keys.getSecond()[i];
-                final String regionLocation =
-                        table.getRegionLocator()
-                                .getRegionLocation(startKey, false)
-                                .getHostnamePort();
-                // Test if the given region is to be included in the InputSplit while splitting the
-                // regions of a table
-                if (!includeRegionInScan(startKey, endKey)) {
-                    continue;
-                }
-                // Find the region on which the given row is being served
-                final String[] hosts = new String[] {regionLocation};
-
-                // Determine if regions contains keys used by the scan
-                boolean isLastRegion = endKey.length == 0;
-                if ((scanWithNoLowerBound || isLastRegion || Bytes.compareTo(startRow, endKey) < 0)
-                        && (scanWithNoUpperBound || Bytes.compareTo(stopRow, startKey) > 0)) {
-
-                    final byte[] splitStart =
-                            scanWithNoLowerBound || Bytes.compareTo(startKey, startRow) >= 0
-                                    ? startKey
-                                    : startRow;
-                    final byte[] splitStop =
-                            (scanWithNoUpperBound || Bytes.compareTo(endKey, stopRow) <= 0)
-                                            && !isLastRegion
-                                    ? endKey
-                                    : stopRow;
-                    int id = splits.size();
-                    final TableInputSplit split =
-                            new TableInputSplit(
-                                    id, hosts, table.getTableName(), splitStart, splitStop);
-                    splits.add(split);
-                }
-            }
-            LOG.info("Created " + splits.size() + " splits");
-            for (TableInputSplit split : splits) {
-                logSplitInfo("created", split);
-            }
-            return splits.toArray(new TableInputSplit[splits.size()]);
-        } finally {
-            closeTable();
-        }
-    }
-
-    /**
-     * Test if the given region is to be included in the scan while splitting the regions of a
-     * table.
-     *
-     * @param startKey Start key of the region
-     * @param endKey End key of the region
-     * @return true, if this region needs to be included as part of the input (default).
-     */
-    protected boolean includeRegionInScan(final byte[] startKey, final byte[] endKey) {
-        return true;
-    }
-
-    @Override
-    public InputSplitAssigner getInputSplitAssigner(TableInputSplit[] inputSplits) {
-        return new LocatableInputSplitAssigner(inputSplits);
-    }
-
-    @Override
-    public BaseStatistics getStatistics(BaseStatistics cachedStatistics) {
-        return null;
-    }
-
-    @VisibleForTesting
-    public Connection getConnection() {
-        return connection;
-    }
-}
diff --git a/flink-connectors/flink-connector-hbase-1.4/src/main/java/org/apache/flink/connector/hbase1/source/HBaseDynamicTableSource.java b/flink-connectors/flink-connector-hbase-1.4/src/main/java/org/apache/flink/connector/hbase1/source/HBaseDynamicTableSource.java
deleted file mode 100644
index 0322c94..0000000
--- a/flink-connectors/flink-connector-hbase-1.4/src/main/java/org/apache/flink/connector/hbase1/source/HBaseDynamicTableSource.java
+++ /dev/null
@@ -1,78 +0,0 @@
-/*
- * 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.hbase1.source;
-
-import org.apache.flink.annotation.Internal;
-import org.apache.flink.api.common.io.InputFormat;
-import org.apache.flink.connector.hbase.source.AbstractHBaseDynamicTableSource;
-import org.apache.flink.connector.hbase.util.HBaseTableSchema;
-import org.apache.flink.table.connector.source.DynamicTableSource;
-import org.apache.flink.table.connector.source.lookup.cache.LookupCache;
-import org.apache.flink.table.data.RowData;
-
-import org.apache.hadoop.conf.Configuration;
-
-import javax.annotation.Nullable;
-
-import java.util.Objects;
-
-/** HBase table source implementation. */
-@Internal
-public class HBaseDynamicTableSource extends AbstractHBaseDynamicTableSource {
-
-    public HBaseDynamicTableSource(
-            Configuration conf,
-            String tableName,
-            HBaseTableSchema hbaseSchema,
-            String nullStringLiteral,
-            int maxRetryTimes,
-            @Nullable LookupCache cache) {
-        super(conf, tableName, hbaseSchema, nullStringLiteral, maxRetryTimes, cache);
-    }
-
-    @Override
-    public DynamicTableSource copy() {
-        return new HBaseDynamicTableSource(
-                conf, tableName, hbaseSchema, nullStringLiteral, maxRetryTimes, cache);
-    }
-
-    @Override
-    public InputFormat<RowData, ?> getInputFormat() {
-        return new HBaseRowDataInputFormat(conf, tableName, hbaseSchema, nullStringLiteral);
-    }
-
-    @Override
-    public boolean equals(Object o) {
-        if (!(o instanceof HBaseDynamicTableSource)) {
-            return false;
-        }
-        HBaseDynamicTableSource that = (HBaseDynamicTableSource) o;
-        return Objects.equals(conf, that.conf)
-                && Objects.equals(tableName, that.tableName)
-                && Objects.equals(hbaseSchema, that.hbaseSchema)
-                && Objects.equals(nullStringLiteral, that.nullStringLiteral)
-                && Objects.equals(maxRetryTimes, that.maxRetryTimes)
-                && Objects.equals(cache, that.cache);
-    }
-
-    @Override
-    public int hashCode() {
-        return Objects.hash(conf, tableName, hbaseSchema, nullStringLiteral, maxRetryTimes, cache);
-    }
-}
diff --git a/flink-connectors/flink-connector-hbase-1.4/src/main/java/org/apache/flink/connector/hbase1/source/HBaseRowDataInputFormat.java b/flink-connectors/flink-connector-hbase-1.4/src/main/java/org/apache/flink/connector/hbase1/source/HBaseRowDataInputFormat.java
deleted file mode 100644
index d48c2fa..0000000
--- a/flink-connectors/flink-connector-hbase-1.4/src/main/java/org/apache/flink/connector/hbase1/source/HBaseRowDataInputFormat.java
+++ /dev/null
@@ -1,97 +0,0 @@
-/*
- * 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.hbase1.source;
-
-import org.apache.flink.api.common.io.InputFormat;
-import org.apache.flink.connector.hbase.util.HBaseSerde;
-import org.apache.flink.connector.hbase.util.HBaseTableSchema;
-import org.apache.flink.table.data.RowData;
-
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.TableNotFoundException;
-import org.apache.hadoop.hbase.client.ConnectionFactory;
-import org.apache.hadoop.hbase.client.HTable;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.Scan;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-
-/**
- * {@link InputFormat} subclass that wraps the access for HTables. Returns the result as {@link
- * RowData}
- */
-public class HBaseRowDataInputFormat extends AbstractTableInputFormat<RowData> {
-    private static final long serialVersionUID = 1L;
-    private static final Logger LOG = LoggerFactory.getLogger(HBaseRowDataInputFormat.class);
-
-    private final String tableName;
-    private final HBaseTableSchema schema;
-    private final String nullStringLiteral;
-
-    private transient HBaseSerde serde;
-
-    public HBaseRowDataInputFormat(
-            org.apache.hadoop.conf.Configuration conf,
-            String tableName,
-            HBaseTableSchema schema,
-            String nullStringLiteral) {
-        super(conf);
-        this.tableName = tableName;
-        this.schema = schema;
-        this.nullStringLiteral = nullStringLiteral;
-    }
-
-    @Override
-    protected void initTable() throws IOException {
-        this.serde = new HBaseSerde(schema, nullStringLiteral);
-        if (table == null) {
-            connectToTable();
-        }
-        if (table != null && scan == null) {
-            scan = getScanner();
-        }
-    }
-
-    @Override
-    protected Scan getScanner() {
-        return serde.createScan();
-    }
-
-    @Override
-    public String getTableName() {
-        return tableName;
-    }
-
-    @Override
-    protected RowData mapResultToOutType(Result res) {
-        return serde.convertToReusedRow(res);
-    }
-
-    private void connectToTable() throws IOException {
-        try {
-            connection = ConnectionFactory.createConnection(getHadoopConfiguration());
-            table = (HTable) connection.getTable(TableName.valueOf(tableName));
-        } catch (TableNotFoundException tnfe) {
-            LOG.error("The table " + tableName + " not found ", tnfe);
-            throw new RuntimeException("HBase table '" + tableName + "' not found.", tnfe);
-        }
-    }
-}
diff --git a/flink-connectors/flink-connector-hbase-1.4/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory b/flink-connectors/flink-connector-hbase-1.4/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory
deleted file mode 100644
index 0b6f75d..0000000
--- a/flink-connectors/flink-connector-hbase-1.4/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory
+++ /dev/null
@@ -1,16 +0,0 @@
-# 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.
-
-org.apache.flink.connector.hbase1.HBase1DynamicTableFactory
diff --git a/flink-connectors/flink-connector-hbase-1.4/src/test/java/org/apache/flink/architecture/TestCodeArchitectureTest.java b/flink-connectors/flink-connector-hbase-1.4/src/test/java/org/apache/flink/architecture/TestCodeArchitectureTest.java
deleted file mode 100644
index 1854cf3..0000000
--- a/flink-connectors/flink-connector-hbase-1.4/src/test/java/org/apache/flink/architecture/TestCodeArchitectureTest.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/*
- * 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.architecture;
-
-import org.apache.flink.architecture.common.ImportOptions;
-
-import com.tngtech.archunit.core.importer.ImportOption;
-import com.tngtech.archunit.junit.AnalyzeClasses;
-import com.tngtech.archunit.junit.ArchTest;
-import com.tngtech.archunit.junit.ArchTests;
-
-/** Architecture tests for test code. */
-@AnalyzeClasses(
-        packages = "org.apache.flink.connector.hbase1",
-        importOptions = {
-            ImportOption.OnlyIncludeTests.class,
-            ImportOptions.ExcludeScalaImportOption.class,
-            ImportOptions.ExcludeShadedImportOption.class
-        })
-public class TestCodeArchitectureTest {
-
-    @ArchTest
-    public static final ArchTests COMMON_TESTS = ArchTests.in(TestCodeArchitectureTestBase.class);
-}
diff --git a/flink-connectors/flink-connector-hbase-1.4/src/test/java/org/apache/flink/connector/hbase1/HBaseConnectorITCase.java b/flink-connectors/flink-connector-hbase-1.4/src/test/java/org/apache/flink/connector/hbase1/HBaseConnectorITCase.java
deleted file mode 100644
index ad5b5de..0000000
--- a/flink-connectors/flink-connector-hbase-1.4/src/test/java/org/apache/flink/connector/hbase1/HBaseConnectorITCase.java
+++ /dev/null
@@ -1,504 +0,0 @@
-/*
- * 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.hbase1;
-
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.common.typeinfo.Types;
-import org.apache.flink.api.java.typeutils.RowTypeInfo;
-import org.apache.flink.connector.hbase.util.HBaseTableSchema;
-import org.apache.flink.connector.hbase1.source.AbstractTableInputFormat;
-import org.apache.flink.connector.hbase1.source.HBaseRowDataInputFormat;
-import org.apache.flink.connector.hbase1.util.HBaseTestBase;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.table.api.Table;
-import org.apache.flink.table.api.TableEnvironment;
-import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
-import org.apache.flink.table.functions.ScalarFunction;
-import org.apache.flink.test.util.TestBaseUtils;
-import org.apache.flink.types.Row;
-import org.apache.flink.util.CollectionUtil;
-
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.junit.Test;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-import java.util.stream.Collectors;
-
-import static org.apache.flink.table.api.Expressions.$;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-
-/** IT cases for HBase connector (source and sink). */
-public class HBaseConnectorITCase extends HBaseTestBase {
-
-    // -------------------------------------------------------------------------------------
-    // HBaseTableSource tests
-    // -------------------------------------------------------------------------------------
-
-    @Test
-    public void testTableSourceFullScan() {
-        TableEnvironment tEnv = TableEnvironment.create(batchSettings);
-        tEnv.executeSql(
-                "CREATE TABLE hTable ("
-                        + " family1 ROW<col1 INT>,"
-                        + " family2 ROW<col1 STRING, col2 BIGINT>,"
-                        + " family3 ROW<col1 DOUBLE, col2 BOOLEAN, col3 STRING>,"
-                        + " rowkey INT,"
-                        + " PRIMARY KEY (rowkey) NOT ENFORCED"
-                        + ") WITH ("
-                        + " 'connector' = 'hbase-1.4',"
-                        + " 'table-name' = '"
-                        + TEST_TABLE_1
-                        + "',"
-                        + " 'zookeeper.quorum' = '"
-                        + getZookeeperQuorum()
-                        + "'"
-                        + ")");
-
-        Table table =
-                tEnv.sqlQuery(
-                        "SELECT "
-                                + "  h.family1.col1, "
-                                + "  h.family2.col1, "
-                                + "  h.family2.col2, "
-                                + "  h.family3.col1, "
-                                + "  h.family3.col2, "
-                                + "  h.family3.col3 "
-                                + "FROM hTable AS h");
-
-        List<Row> results = CollectionUtil.iteratorToList(table.execute().collect());
-        String expected =
-                "+I[10, Hello-1, 100, 1.01, false, Welt-1]\n"
-                        + "+I[20, Hello-2, 200, 2.02, true, Welt-2]\n"
-                        + "+I[30, Hello-3, 300, 3.03, false, Welt-3]\n"
-                        + "+I[40, null, 400, 4.04, true, Welt-4]\n"
-                        + "+I[50, Hello-5, 500, 5.05, false, Welt-5]\n"
-                        + "+I[60, Hello-6, 600, 6.06, true, Welt-6]\n"
-                        + "+I[70, Hello-7, 700, 7.07, false, Welt-7]\n"
-                        + "+I[80, null, 800, 8.08, true, Welt-8]\n";
-
-        TestBaseUtils.compareResultAsText(results, expected);
-    }
-
-    @Test
-    public void testTableSourceProjection() {
-        TableEnvironment tEnv = TableEnvironment.create(batchSettings);
-
-        tEnv.executeSql(
-                "CREATE TABLE hTable ("
-                        + " family1 ROW<col1 INT>,"
-                        + " family2 ROW<col1 STRING, col2 BIGINT>,"
-                        + " family3 ROW<col1 DOUBLE, col2 BOOLEAN, col3 STRING>,"
-                        + " rowkey INT,"
-                        + " PRIMARY KEY (rowkey) NOT ENFORCED"
-                        + ") WITH ("
-                        + " 'connector' = 'hbase-1.4',"
-                        + " 'table-name' = '"
-                        + TEST_TABLE_1
-                        + "',"
-                        + " 'zookeeper.quorum' = '"
-                        + getZookeeperQuorum()
-                        + "'"
-                        + ")");
-
-        Table table =
-                tEnv.sqlQuery(
-                        "SELECT "
-                                + "  h.family1.col1, "
-                                + "  h.family3.col1, "
-                                + "  h.family3.col2, "
-                                + "  h.family3.col3 "
-                                + "FROM hTable AS h");
-
-        List<Row> results = CollectionUtil.iteratorToList(table.execute().collect());
-        String expected =
-                "+I[10, 1.01, false, Welt-1]\n"
-                        + "+I[20, 2.02, true, Welt-2]\n"
-                        + "+I[30, 3.03, false, Welt-3]\n"
-                        + "+I[40, 4.04, true, Welt-4]\n"
-                        + "+I[50, 5.05, false, Welt-5]\n"
-                        + "+I[60, 6.06, true, Welt-6]\n"
-                        + "+I[70, 7.07, false, Welt-7]\n"
-                        + "+I[80, 8.08, true, Welt-8]\n";
-
-        TestBaseUtils.compareResultAsText(results, expected);
-    }
-
-    @Test
-    public void testTableSourceFieldOrder() {
-        TableEnvironment tEnv = TableEnvironment.create(batchSettings);
-
-        tEnv.executeSql(
-                "CREATE TABLE hTable ("
-                        + " rowkey INT PRIMARY KEY NOT ENFORCED,"
-                        + " family2 ROW<col1 STRING, col2 BIGINT>,"
-                        + " family3 ROW<col1 DOUBLE, col2 BOOLEAN, col3 STRING>,"
-                        + " family1 ROW<col1 INT>"
-                        + ") WITH ("
-                        + " 'connector' = 'hbase-1.4',"
-                        + " 'table-name' = '"
-                        + TEST_TABLE_1
-                        + "',"
-                        + " 'zookeeper.quorum' = '"
-                        + getZookeeperQuorum()
-                        + "'"
-                        + ")");
-
-        Table table = tEnv.sqlQuery("SELECT * FROM hTable AS h");
-
-        List<Row> results = CollectionUtil.iteratorToList(table.execute().collect());
-        String expected =
-                "+I[1, +I[Hello-1, 100], +I[1.01, false, Welt-1], +I[10]]\n"
-                        + "+I[2, +I[Hello-2, 200], +I[2.02, true, Welt-2], +I[20]]\n"
-                        + "+I[3, +I[Hello-3, 300], +I[3.03, false, Welt-3], +I[30]]\n"
-                        + "+I[4, +I[null, 400], +I[4.04, true, Welt-4], +I[40]]\n"
-                        + "+I[5, +I[Hello-5, 500], +I[5.05, false, Welt-5], +I[50]]\n"
-                        + "+I[6, +I[Hello-6, 600], +I[6.06, true, Welt-6], +I[60]]\n"
-                        + "+I[7, +I[Hello-7, 700], +I[7.07, false, Welt-7], +I[70]]\n"
-                        + "+I[8, +I[null, 800], +I[8.08, true, Welt-8], +I[80]]\n";
-
-        TestBaseUtils.compareResultAsText(results, expected);
-    }
-
-    @Test
-    public void testTableSourceReadAsByteArray() {
-        TableEnvironment tEnv = TableEnvironment.create(batchSettings);
-
-        tEnv.executeSql(
-                "CREATE TABLE hTable ("
-                        + " family2 ROW<col1 BYTES, col2 BYTES>,"
-                        + " rowkey INT"
-                        + // no primary key syntax
-                        ") WITH ("
-                        + " 'connector' = 'hbase-1.4',"
-                        + " 'table-name' = '"
-                        + TEST_TABLE_1
-                        + "',"
-                        + " 'zookeeper.quorum' = '"
-                        + getZookeeperQuorum()
-                        + "'"
-                        + ")");
-        tEnv.registerFunction("toUTF8", new ToUTF8());
-        tEnv.registerFunction("toLong", new ToLong());
-
-        Table table =
-                tEnv.sqlQuery(
-                        "SELECT "
-                                + "  toUTF8(h.family2.col1), "
-                                + "  toLong(h.family2.col2) "
-                                + "FROM hTable AS h");
-
-        List<Row> results = CollectionUtil.iteratorToList(table.execute().collect());
-        String expected =
-                "+I[Hello-1, 100]\n"
-                        + "+I[Hello-2, 200]\n"
-                        + "+I[Hello-3, 300]\n"
-                        + "+I[null, 400]\n"
-                        + "+I[Hello-5, 500]\n"
-                        + "+I[Hello-6, 600]\n"
-                        + "+I[Hello-7, 700]\n"
-                        + "+I[null, 800]\n";
-
-        TestBaseUtils.compareResultAsText(results, expected);
-    }
-
-    @Test
-    public void testTableSink() throws Exception {
-        StreamExecutionEnvironment execEnv = StreamExecutionEnvironment.getExecutionEnvironment();
-        StreamTableEnvironment tEnv = StreamTableEnvironment.create(execEnv, streamSettings);
-
-        // register HBase table testTable1 which contains test data
-        String table1DDL = createHBaseTableDDL(TEST_TABLE_1, false);
-        tEnv.executeSql(table1DDL);
-
-        String table2DDL = createHBaseTableDDL(TEST_TABLE_2, false);
-        tEnv.executeSql(table2DDL);
-
-        String query =
-                "INSERT INTO "
-                        + TEST_TABLE_2
-                        + " SELECT"
-                        + " rowkey,"
-                        + " family1,"
-                        + " family2,"
-                        + " family3"
-                        + " FROM "
-                        + TEST_TABLE_1;
-
-        tEnv.executeSql(query).await();
-
-        // start a batch scan job to verify contents in HBase table
-        TableEnvironment batchEnv = TableEnvironment.create(batchSettings);
-        batchEnv.executeSql(table2DDL);
-
-        Table table =
-                batchEnv.sqlQuery(
-                        "SELECT "
-                                + "  h.rowkey, "
-                                + "  h.family1.col1, "
-                                + "  h.family2.col1, "
-                                + "  h.family2.col2, "
-                                + "  h.family3.col1, "
-                                + "  h.family3.col2, "
-                                + "  h.family3.col3 "
-                                + "FROM "
-                                + TEST_TABLE_2
-                                + " AS h");
-        List<Row> results = CollectionUtil.iteratorToList(table.execute().collect());
-        String expected =
-                "+I[1, 10, Hello-1, 100, 1.01, false, Welt-1]\n"
-                        + "+I[2, 20, Hello-2, 200, 2.02, true, Welt-2]\n"
-                        + "+I[3, 30, Hello-3, 300, 3.03, false, Welt-3]\n"
-                        + "+I[4, 40, null, 400, 4.04, true, Welt-4]\n"
-                        + "+I[5, 50, Hello-5, 500, 5.05, false, Welt-5]\n"
-                        + "+I[6, 60, Hello-6, 600, 6.06, true, Welt-6]\n"
-                        + "+I[7, 70, Hello-7, 700, 7.07, false, Welt-7]\n"
-                        + "+I[8, 80, null, 800, 8.08, true, Welt-8]\n";
-
-        TestBaseUtils.compareResultAsText(results, expected);
-    }
-
-    @Test
-    public void testTableSourceSinkWithDDL() throws Exception {
-        StreamExecutionEnvironment execEnv = StreamExecutionEnvironment.getExecutionEnvironment();
-        StreamTableEnvironment tEnv = StreamTableEnvironment.create(execEnv, streamSettings);
-
-        // register HBase table testTable1 which contains test data
-        String table1DDL = createHBaseTableDDL(TEST_TABLE_1, true);
-        tEnv.executeSql(table1DDL);
-
-        // register HBase table which is empty
-        String table3DDL = createHBaseTableDDL(TEST_TABLE_3, true);
-        tEnv.executeSql(table3DDL);
-
-        String insertStatement =
-                "INSERT INTO "
-                        + TEST_TABLE_3
-                        + " SELECT rowkey,"
-                        + " family1,"
-                        + " family2,"
-                        + " family3,"
-                        + " family4"
-                        + " from "
-                        + TEST_TABLE_1;
-        tEnv.executeSql(insertStatement).await();
-
-        // start a batch scan job to verify contents in HBase table
-        TableEnvironment batchEnv = TableEnvironment.create(batchSettings);
-        batchEnv.executeSql(table3DDL);
-        String query =
-                "SELECT "
-                        + "  h.rowkey, "
-                        + "  h.family1.col1, "
-                        + "  h.family2.col1, "
-                        + "  h.family2.col2, "
-                        + "  h.family3.col1, "
-                        + "  h.family3.col2, "
-                        + "  h.family3.col3, "
-                        + "  h.family4.col1, "
-                        + "  h.family4.col2, "
-                        + "  h.family4.col3, "
-                        + "  h.family4.col4 "
-                        + " FROM "
-                        + TEST_TABLE_3
-                        + " AS h";
-        Iterator<Row> collected = tEnv.executeSql(query).collect();
-        List<String> result =
-                CollectionUtil.iteratorToList(collected).stream()
-                        .map(Row::toString)
-                        .sorted()
-                        .collect(Collectors.toList());
-
-        List<String> expected = new ArrayList<>();
-        expected.add(
-                "+I[1, 10, Hello-1, 100, 1.01, false, Welt-1, 2019-08-18T19:00, 2019-08-18, 19:00, 12345678.0001]");
-        expected.add(
-                "+I[2, 20, Hello-2, 200, 2.02, true, Welt-2, 2019-08-18T19:01, 2019-08-18, 19:01, 12345678.0002]");
-        expected.add(
-                "+I[3, 30, Hello-3, 300, 3.03, false, Welt-3, 2019-08-18T19:02, 2019-08-18, 19:02, 12345678.0003]");
-        expected.add(
-                "+I[4, 40, null, 400, 4.04, true, Welt-4, 2019-08-18T19:03, 2019-08-18, 19:03, 12345678.0004]");
-        expected.add(
-                "+I[5, 50, Hello-5, 500, 5.05, false, Welt-5, 2019-08-19T19:10, 2019-08-19, 19:10, 12345678.0005]");
-        expected.add(
-                "+I[6, 60, Hello-6, 600, 6.06, true, Welt-6, 2019-08-19T19:20, 2019-08-19, 19:20, 12345678.0006]");
-        expected.add(
-                "+I[7, 70, Hello-7, 700, 7.07, false, Welt-7, 2019-08-19T19:30, 2019-08-19, 19:30, 12345678.0007]");
-        expected.add(
-                "+I[8, 80, null, 800, 8.08, true, Welt-8, 2019-08-19T19:40, 2019-08-19, 19:40, 12345678.0008]");
-        assertEquals(expected, result);
-    }
-
-    @Test
-    public void testHBaseLookupTableSource() {
-        StreamExecutionEnvironment execEnv = StreamExecutionEnvironment.getExecutionEnvironment();
-        StreamTableEnvironment tEnv = StreamTableEnvironment.create(execEnv, streamSettings);
-
-        tEnv.executeSql(
-                "CREATE TABLE "
-                        + TEST_TABLE_1
-                        + " ("
-                        + " family1 ROW<col1 INT>,"
-                        + " family2 ROW<col1 STRING, col2 BIGINT>,"
-                        + " family3 ROW<col1 DOUBLE, col2 BOOLEAN, col3 STRING>,"
-                        + " rowkey INT,"
-                        + " family4 ROW<col1 TIMESTAMP(3), col2 DATE, col3 TIME(3), col4 DECIMAL(12, 4)>,"
-                        + " PRIMARY KEY (rowkey) NOT ENFORCED"
-                        + ") WITH ("
-                        + " 'connector' = 'hbase-1.4',"
-                        + " 'table-name' = '"
-                        + TEST_TABLE_1
-                        + "',"
-                        + " 'zookeeper.quorum' = '"
-                        + getZookeeperQuorum()
-                        + "'"
-                        + ")");
-
-        // prepare a source table
-        String srcTableName = "src";
-        DataStream<Row> srcDs = execEnv.fromCollection(testData).returns(testTypeInfo);
-        Table in = tEnv.fromDataStream(srcDs, $("a"), $("b"), $("c"), $("proc").proctime());
-        tEnv.createTemporaryView(srcTableName, in);
-
-        // perform a temporal table join query
-        String dimJoinQuery =
-                "SELECT"
-                        + " a,"
-                        + " b,"
-                        + " h.family1.col1,"
-                        + " h.family2.col1,"
-                        + " h.family2.col2,"
-                        + " h.family3.col1,"
-                        + " h.family3.col2,"
-                        + " h.family3.col3,"
-                        + " h.family4.col1,"
-                        + " h.family4.col2,"
-                        + " h.family4.col3,"
-                        + " h.family4.col4 "
-                        + " FROM src JOIN "
-                        + TEST_TABLE_1
-                        + " FOR SYSTEM_TIME AS OF src.proc as h ON src.a = h.rowkey";
-        Iterator<Row> collected = tEnv.executeSql(dimJoinQuery).collect();
-        List<String> result =
-                CollectionUtil.iteratorToList(collected).stream()
-                        .map(Row::toString)
-                        .sorted()
-                        .collect(Collectors.toList());
-
-        List<String> expected = new ArrayList<>();
-        expected.add(
-                "+I[1, 1, 10, Hello-1, 100, 1.01, false, Welt-1, 2019-08-18T19:00, 2019-08-18, 19:00, 12345678.0001]");
-        expected.add(
-                "+I[2, 2, 20, Hello-2, 200, 2.02, true, Welt-2, 2019-08-18T19:01, 2019-08-18, 19:01, 12345678.0002]");
-        expected.add(
-                "+I[3, 2, 30, Hello-3, 300, 3.03, false, Welt-3, 2019-08-18T19:02, 2019-08-18, 19:02, 12345678.0003]");
-        expected.add(
-                "+I[3, 3, 30, Hello-3, 300, 3.03, false, Welt-3, 2019-08-18T19:02, 2019-08-18, 19:02, 12345678.0003]");
-
-        assertEquals(expected, result);
-    }
-
-    @Test
-    public void testTableInputFormatOpenClose() throws IOException {
-        HBaseTableSchema tableSchema = new HBaseTableSchema();
-        tableSchema.addColumn(FAMILY1, F1COL1, byte[].class);
-        AbstractTableInputFormat<?> inputFormat =
-                new HBaseRowDataInputFormat(getConf(), TEST_TABLE_1, tableSchema, "null");
-        inputFormat.open(inputFormat.createInputSplits(1)[0]);
-        assertNotNull(inputFormat.getConnection());
-        assertNotNull(inputFormat.getConnection().getTable(TableName.valueOf(TEST_TABLE_1)));
-
-        inputFormat.close();
-        assertNull(inputFormat.getConnection());
-    }
-
-    // -------------------------------------------------------------------------------------
-    // HBase lookup source tests
-    // -------------------------------------------------------------------------------------
-
-    // prepare a source collection.
-    private static final List<Row> testData = new ArrayList<>();
-    private static final RowTypeInfo testTypeInfo =
-            new RowTypeInfo(
-                    new TypeInformation[] {Types.INT, Types.LONG, Types.STRING},
-                    new String[] {"a", "b", "c"});
-
-    static {
-        testData.add(Row.of(1, 1L, "Hi"));
-        testData.add(Row.of(2, 2L, "Hello"));
-        testData.add(Row.of(3, 2L, "Hello world"));
-        testData.add(Row.of(3, 3L, "Hello world!"));
-    }
-
-    // ------------------------------- Utilities -------------------------------------------------
-
-    /** A {@link ScalarFunction} that maps byte arrays to UTF-8 strings. */
-    public static class ToUTF8 extends ScalarFunction {
-        private static final long serialVersionUID = 1L;
-
-        public String eval(byte[] bytes) {
-            return Bytes.toString(bytes);
-        }
-    }
-
-    /** A {@link ScalarFunction} that maps byte array to longs. */
-    public static class ToLong extends ScalarFunction {
-        private static final long serialVersionUID = 1L;
-
-        public long eval(byte[] bytes) {
-            return Bytes.toLong(bytes);
-        }
-    }
-
-    private String createHBaseTableDDL(String tableName, boolean testTimeAndDecimalTypes) {
-        StringBuilder family4Statement = new StringBuilder();
-        if (testTimeAndDecimalTypes) {
-            family4Statement.append(", family4 ROW<col1 TIMESTAMP(3)");
-            family4Statement.append(", col2 DATE");
-            family4Statement.append(", col3 TIME(3)");
-            family4Statement.append(", col4 DECIMAL(12, 4)");
-            family4Statement.append("> \n");
-        }
-
-        return "CREATE TABLE "
-                + tableName
-                + "(\n"
-                + "   rowkey INT,"
-                + "   family1 ROW<col1 INT>,\n"
-                + "   family2 ROW<col1 VARCHAR, col2 BIGINT>,\n"
-                + "   family3 ROW<col1 DOUBLE, col2 BOOLEAN, col3 VARCHAR>"
-                + family4Statement.toString()
-                + ") WITH (\n"
-                + "   'connector' = 'hbase-1.4',\n"
-                + "   'table-name' = '"
-                + tableName
-                + "',\n"
-                + "   'zookeeper.quorum' = '"
-                + getZookeeperQuorum()
-                + "',\n"
-                + "   'zookeeper.znode.parent' = '/hbase' "
-                + ")";
-    }
-}
diff --git a/flink-connectors/flink-connector-hbase-1.4/src/test/java/org/apache/flink/connector/hbase1/HBaseDynamicTableFactoryTest.java b/flink-connectors/flink-connector-hbase-1.4/src/test/java/org/apache/flink/connector/hbase1/HBaseDynamicTableFactoryTest.java
deleted file mode 100644
index 8a8f1d7..0000000
--- a/flink-connectors/flink-connector-hbase-1.4/src/test/java/org/apache/flink/connector/hbase1/HBaseDynamicTableFactoryTest.java
+++ /dev/null
@@ -1,397 +0,0 @@
-/*
- * 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.hbase1;
-
-import org.apache.flink.api.common.typeinfo.Types;
-import org.apache.flink.connector.hbase.options.HBaseWriteOptions;
-import org.apache.flink.connector.hbase.source.HBaseRowDataLookupFunction;
-import org.apache.flink.connector.hbase.util.HBaseTableSchema;
-import org.apache.flink.connector.hbase1.sink.HBaseDynamicTableSink;
-import org.apache.flink.connector.hbase1.source.HBaseDynamicTableSource;
-import org.apache.flink.table.catalog.Column;
-import org.apache.flink.table.catalog.ResolvedSchema;
-import org.apache.flink.table.connector.sink.DynamicTableSink;
-import org.apache.flink.table.connector.sink.SinkFunctionProvider;
-import org.apache.flink.table.connector.source.DynamicTableSource;
-import org.apache.flink.table.connector.source.LookupTableSource;
-import org.apache.flink.table.connector.source.lookup.LookupFunctionProvider;
-import org.apache.flink.table.connector.source.lookup.cache.DefaultLookupCache;
-import org.apache.flink.table.functions.LookupFunction;
-import org.apache.flink.table.runtime.connector.sink.SinkRuntimeProviderContext;
-import org.apache.flink.table.runtime.connector.source.LookupRuntimeProviderContext;
-import org.apache.flink.table.types.DataType;
-import org.apache.flink.util.ExceptionUtils;
-
-import org.apache.commons.collections.IteratorUtils;
-import org.apache.hadoop.hbase.HConstants;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
-
-import java.time.Duration;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Optional;
-
-import static org.apache.flink.connector.hbase.util.HBaseConfigurationUtil.getHBaseConfiguration;
-import static org.apache.flink.table.api.DataTypes.BIGINT;
-import static org.apache.flink.table.api.DataTypes.BOOLEAN;
-import static org.apache.flink.table.api.DataTypes.DATE;
-import static org.apache.flink.table.api.DataTypes.DECIMAL;
-import static org.apache.flink.table.api.DataTypes.DOUBLE;
-import static org.apache.flink.table.api.DataTypes.FIELD;
-import static org.apache.flink.table.api.DataTypes.INT;
-import static org.apache.flink.table.api.DataTypes.ROW;
-import static org.apache.flink.table.api.DataTypes.STRING;
-import static org.apache.flink.table.api.DataTypes.TIME;
-import static org.apache.flink.table.api.DataTypes.TIMESTAMP;
-import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSink;
-import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSource;
-import static org.assertj.core.api.Assertions.assertThat;
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-/** Unit test for {@link HBase1DynamicTableFactory}. */
-public class HBaseDynamicTableFactoryTest {
-
-    private static final String FAMILY1 = "f1";
-    private static final String FAMILY2 = "f2";
-    private static final String FAMILY3 = "f3";
-    private static final String FAMILY4 = "f4";
-    private static final String COL1 = "c1";
-    private static final String COL2 = "c2";
-    private static final String COL3 = "c3";
-    private static final String COL4 = "c4";
-    private static final String ROWKEY = "rowkey";
-
-    @Rule public final ExpectedException thrown = ExpectedException.none();
-
-    @SuppressWarnings("rawtypes")
-    @Test
-    public void testTableSourceFactory() {
-        ResolvedSchema schema =
-                ResolvedSchema.of(
-                        Column.physical(FAMILY1, ROW(FIELD(COL1, INT()))),
-                        Column.physical(FAMILY2, ROW(FIELD(COL1, INT()), FIELD(COL2, BIGINT()))),
-                        Column.physical(ROWKEY, BIGINT()),
-                        Column.physical(
-                                FAMILY3,
-                                ROW(
-                                        FIELD(COL1, DOUBLE()),
-                                        FIELD(COL2, BOOLEAN()),
-                                        FIELD(COL3, STRING()))),
-                        Column.physical(
-                                FAMILY4,
-                                ROW(
-                                        FIELD(COL1, DECIMAL(10, 3)),
-                                        FIELD(COL2, TIMESTAMP(3)),
-                                        FIELD(COL3, DATE()),
-                                        FIELD(COL4, TIME()))));
-
-        DynamicTableSource source = createTableSource(schema, getAllOptions());
-        assertTrue(source instanceof HBaseDynamicTableSource);
-        HBaseDynamicTableSource hbaseSource = (HBaseDynamicTableSource) source;
-
-        int[][] lookupKey = {{2}};
-        LookupTableSource.LookupRuntimeProvider lookupProvider =
-                hbaseSource.getLookupRuntimeProvider(new LookupRuntimeProviderContext(lookupKey));
-        assertTrue(lookupProvider instanceof LookupFunctionProvider);
-
-        LookupFunction tableFunction =
-                ((LookupFunctionProvider) lookupProvider).createLookupFunction();
-        assertTrue(tableFunction instanceof HBaseRowDataLookupFunction);
-        assertEquals(
-                "testHBastTable", ((HBaseRowDataLookupFunction) tableFunction).getHTableName());
-
-        HBaseTableSchema hbaseSchema = hbaseSource.getHBaseTableSchema();
-        assertEquals(2, hbaseSchema.getRowKeyIndex());
-        assertEquals(Optional.of(Types.LONG), hbaseSchema.getRowKeyTypeInfo());
-
-        assertArrayEquals(new String[] {"f1", "f2", "f3", "f4"}, hbaseSchema.getFamilyNames());
-        assertArrayEquals(new String[] {"c1"}, hbaseSchema.getQualifierNames("f1"));
-        assertArrayEquals(new String[] {"c1", "c2"}, hbaseSchema.getQualifierNames("f2"));
-        assertArrayEquals(new String[] {"c1", "c2", "c3"}, hbaseSchema.getQualifierNames("f3"));
-        assertArrayEquals(
-                new String[] {"c1", "c2", "c3", "c4"}, hbaseSchema.getQualifierNames("f4"));
-
-        assertArrayEquals(new DataType[] {INT()}, hbaseSchema.getQualifierDataTypes("f1"));
-        assertArrayEquals(
-                new DataType[] {INT(), BIGINT()}, hbaseSchema.getQualifierDataTypes("f2"));
-        assertArrayEquals(
-                new DataType[] {DOUBLE(), BOOLEAN(), STRING()},
-                hbaseSchema.getQualifierDataTypes("f3"));
-        assertArrayEquals(
-                new DataType[] {DECIMAL(10, 3), TIMESTAMP(3), DATE(), TIME()},
-                hbaseSchema.getQualifierDataTypes("f4"));
-    }
-
-    @Test
-    public void testLookupOptions() {
-        ResolvedSchema schema = ResolvedSchema.of(Column.physical(ROWKEY, STRING()));
-        Map<String, String> options = getAllOptions();
-        options.put("lookup.cache", "PARTIAL");
-        options.put("lookup.partial-cache.expire-after-access", "15213s");
-        options.put("lookup.partial-cache.expire-after-write", "18213s");
-        options.put("lookup.partial-cache.max-rows", "10000");
-        options.put("lookup.partial-cache.cache-missing-key", "false");
-        options.put("lookup.max-retries", "15513");
-
-        DynamicTableSource source = createTableSource(schema, options);
-        HBaseDynamicTableSource hbaseSource = (HBaseDynamicTableSource) source;
-        assertThat(((HBaseDynamicTableSource) source).getMaxRetryTimes()).isEqualTo(15513);
-        assertThat(hbaseSource.getCache()).isInstanceOf(DefaultLookupCache.class);
-        DefaultLookupCache cache = (DefaultLookupCache) hbaseSource.getCache();
-        assertThat(cache)
-                .isEqualTo(
-                        DefaultLookupCache.newBuilder()
-                                .expireAfterAccess(Duration.ofSeconds(15213))
-                                .expireAfterWrite(Duration.ofSeconds(18213))
-                                .maximumSize(10000)
-                                .cacheMissingKey(false)
-                                .build());
-    }
-
-    @Test
-    public void testTableSinkFactory() {
-        ResolvedSchema schema =
-                ResolvedSchema.of(
-                        Column.physical(ROWKEY, STRING()),
-                        Column.physical(FAMILY1, ROW(FIELD(COL1, DOUBLE()), FIELD(COL2, INT()))),
-                        Column.physical(FAMILY2, ROW(FIELD(COL1, INT()), FIELD(COL3, BIGINT()))),
-                        Column.physical(
-                                FAMILY3, ROW(FIELD(COL2, BOOLEAN()), FIELD(COL3, STRING()))),
-                        Column.physical(
-                                FAMILY4,
-                                ROW(
-                                        FIELD(COL1, DECIMAL(10, 3)),
-                                        FIELD(COL2, TIMESTAMP(3)),
-                                        FIELD(COL3, DATE()),
-                                        FIELD(COL4, TIME()))));
-
-        DynamicTableSink sink = createTableSink(schema, getAllOptions());
-        assertTrue(sink instanceof HBaseDynamicTableSink);
-        HBaseDynamicTableSink hbaseSink = (HBaseDynamicTableSink) sink;
-
-        HBaseTableSchema hbaseSchema = hbaseSink.getHBaseTableSchema();
-        assertEquals(0, hbaseSchema.getRowKeyIndex());
-        assertEquals(Optional.of(STRING()), hbaseSchema.getRowKeyDataType());
-
-        assertArrayEquals(new String[] {"f1", "f2", "f3", "f4"}, hbaseSchema.getFamilyNames());
-        assertArrayEquals(new String[] {"c1", "c2"}, hbaseSchema.getQualifierNames("f1"));
-        assertArrayEquals(new String[] {"c1", "c3"}, hbaseSchema.getQualifierNames("f2"));
-        assertArrayEquals(new String[] {"c2", "c3"}, hbaseSchema.getQualifierNames("f3"));
-        assertArrayEquals(
-                new String[] {"c1", "c2", "c3", "c4"}, hbaseSchema.getQualifierNames("f4"));
-
-        assertArrayEquals(
-                new DataType[] {DOUBLE(), INT()}, hbaseSchema.getQualifierDataTypes("f1"));
-        assertArrayEquals(
-                new DataType[] {INT(), BIGINT()}, hbaseSchema.getQualifierDataTypes("f2"));
-        assertArrayEquals(
-                new DataType[] {BOOLEAN(), STRING()}, hbaseSchema.getQualifierDataTypes("f3"));
-        assertArrayEquals(
-                new DataType[] {DECIMAL(10, 3), TIMESTAMP(3), DATE(), TIME()},
-                hbaseSchema.getQualifierDataTypes("f4"));
-
-        // verify hadoop Configuration
-        org.apache.hadoop.conf.Configuration expectedConfiguration = getHBaseConfiguration();
-        expectedConfiguration.set(HConstants.ZOOKEEPER_QUORUM, "localhost:2181");
-        expectedConfiguration.set(HConstants.ZOOKEEPER_ZNODE_PARENT, "/flink");
-        expectedConfiguration.set("hbase.security.authentication", "kerberos");
-
-        org.apache.hadoop.conf.Configuration actualConfiguration = hbaseSink.getConfiguration();
-
-        assertEquals(
-                IteratorUtils.toList(expectedConfiguration.iterator()),
-                IteratorUtils.toList(actualConfiguration.iterator()));
-
-        // verify tableName
-        assertEquals("testHBastTable", hbaseSink.getTableName());
-
-        HBaseWriteOptions expectedWriteOptions =
-                HBaseWriteOptions.builder()
-                        .setBufferFlushMaxRows(1000)
-                        .setBufferFlushIntervalMillis(1000)
-                        .setBufferFlushMaxSizeInBytes(2 * 1024 * 1024)
-                        .build();
-        HBaseWriteOptions actualWriteOptions = hbaseSink.getWriteOptions();
-        assertEquals(expectedWriteOptions, actualWriteOptions);
-    }
-
-    @Test
-    public void testBufferFlushOptions() {
-        Map<String, String> options = getAllOptions();
-        options.put("sink.buffer-flush.max-size", "10mb");
-        options.put("sink.buffer-flush.max-rows", "100");
-        options.put("sink.buffer-flush.interval", "10s");
-
-        ResolvedSchema schema = ResolvedSchema.of(Column.physical(ROWKEY, STRING()));
-
-        DynamicTableSink sink = createTableSink(schema, options);
-        HBaseWriteOptions expected =
-                HBaseWriteOptions.builder()
-                        .setBufferFlushMaxRows(100)
-                        .setBufferFlushIntervalMillis(10 * 1000)
-                        .setBufferFlushMaxSizeInBytes(10 * 1024 * 1024)
-                        .build();
-        HBaseWriteOptions actual = ((HBaseDynamicTableSink) sink).getWriteOptions();
-        assertEquals(expected, actual);
-    }
-
-    @Test
-    public void testParallelismOptions() {
-        Map<String, String> options = getAllOptions();
-        options.put("sink.parallelism", "2");
-
-        ResolvedSchema schema = ResolvedSchema.of(Column.physical(ROWKEY, STRING()));
-
-        DynamicTableSink sink = createTableSink(schema, options);
-        assertTrue(sink instanceof HBaseDynamicTableSink);
-        HBaseDynamicTableSink hbaseSink = (HBaseDynamicTableSink) sink;
-        SinkFunctionProvider provider =
-                (SinkFunctionProvider)
-                        hbaseSink.getSinkRuntimeProvider(new SinkRuntimeProviderContext(false));
-        assertEquals(2, (long) provider.getParallelism().get());
-    }
-
-    @Test
-    public void testDisabledBufferFlushOptions() {
-        Map<String, String> options = getAllOptions();
-        options.put("sink.buffer-flush.max-size", "0");
-        options.put("sink.buffer-flush.max-rows", "0");
-        options.put("sink.buffer-flush.interval", "0");
-
-        ResolvedSchema schema = ResolvedSchema.of(Column.physical(ROWKEY, STRING()));
-
-        DynamicTableSink sink = createTableSink(schema, options);
-        HBaseWriteOptions expected =
-                HBaseWriteOptions.builder()
-                        .setBufferFlushMaxRows(0)
-                        .setBufferFlushIntervalMillis(0)
-                        .setBufferFlushMaxSizeInBytes(0)
-                        .build();
-        HBaseWriteOptions actual = ((HBaseDynamicTableSink) sink).getWriteOptions();
-        assertEquals(expected, actual);
-    }
-
-    @Test
-    public void testUnknownOption() {
-        Map<String, String> options = getAllOptions();
-        options.put("sink.unknown.key", "unknown-value");
-        ResolvedSchema schema =
-                ResolvedSchema.of(
-                        Column.physical(ROWKEY, STRING()),
-                        Column.physical(FAMILY1, ROW(FIELD(COL1, DOUBLE()), FIELD(COL2, INT()))));
-
-        try {
-            createTableSource(schema, options);
-            fail("Should fail");
-        } catch (Exception e) {
-            assertTrue(
-                    ExceptionUtils.findThrowableWithMessage(
-                                    e, "Unsupported options:\n\nsink.unknown.key")
-                            .isPresent());
-        }
-
-        try {
-            createTableSink(schema, options);
-            fail("Should fail");
-        } catch (Exception e) {
-            assertTrue(
-                    ExceptionUtils.findThrowableWithMessage(
-                                    e, "Unsupported options:\n\nsink.unknown.key")
-                            .isPresent());
-        }
-    }
-
-    @Test
-    public void testTypeWithUnsupportedPrecision() {
-        Map<String, String> options = getAllOptions();
-        // test unsupported timestamp precision
-        ResolvedSchema schema =
-                ResolvedSchema.of(
-                        Column.physical(ROWKEY, STRING()),
-                        Column.physical(
-                                FAMILY1, ROW(FIELD(COL1, TIMESTAMP(6)), FIELD(COL2, INT()))));
-        try {
-            createTableSource(schema, options);
-            fail("Should fail");
-        } catch (Exception e) {
-            assertTrue(
-                    ExceptionUtils.findThrowableWithMessage(
-                                    e,
-                                    "The precision 6 of TIMESTAMP type is out of the range [0, 3]"
-                                            + " supported by HBase connector")
-                            .isPresent());
-        }
-
-        try {
-            createTableSink(schema, options);
-            fail("Should fail");
-        } catch (Exception e) {
-            assertTrue(
-                    ExceptionUtils.findThrowableWithMessage(
-                                    e,
-                                    "The precision 6 of TIMESTAMP type is out of the range [0, 3]"
-                                            + " supported by HBase connector")
-                            .isPresent());
-        }
-        // test unsupported time precision
-        schema =
-                ResolvedSchema.of(
-                        Column.physical(ROWKEY, STRING()),
-                        Column.physical(FAMILY1, ROW(FIELD(COL1, TIME(6)), FIELD(COL2, INT()))));
-
-        try {
-            createTableSource(schema, options);
-            fail("Should fail");
-        } catch (Exception e) {
-            assertTrue(
-                    ExceptionUtils.findThrowableWithMessage(
-                                    e,
-                                    "The precision 6 of TIME type is out of the range [0, 3]"
-                                            + " supported by HBase connector")
-                            .isPresent());
-        }
-
-        try {
-            createTableSink(schema, options);
-            fail("Should fail");
-        } catch (Exception e) {
-            assertTrue(
-                    ExceptionUtils.findThrowableWithMessage(
-                                    e,
-                                    "The precision 6 of TIME type is out of the range [0, 3]"
-                                            + " supported by HBase connector")
-                            .isPresent());
-        }
-    }
-
-    private Map<String, String> getAllOptions() {
-        Map<String, String> options = new HashMap<>();
-        options.put("connector", "hbase-1.4");
-        options.put("table-name", "testHBastTable");
-        options.put("zookeeper.quorum", "localhost:2181");
-        options.put("zookeeper.znode.parent", "/flink");
-        options.put("properties.hbase.security.authentication", "kerberos");
-        return options;
-    }
-}
diff --git a/flink-connectors/flink-connector-hbase-1.4/src/test/java/org/apache/flink/connector/hbase1/HBaseTablePlanTest.java b/flink-connectors/flink-connector-hbase-1.4/src/test/java/org/apache/flink/connector/hbase1/HBaseTablePlanTest.java
deleted file mode 100644
index f6e1ec1..0000000
--- a/flink-connectors/flink-connector-hbase-1.4/src/test/java/org/apache/flink/connector/hbase1/HBaseTablePlanTest.java
+++ /dev/null
@@ -1,138 +0,0 @@
-/*
- * 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.hbase1;
-
-import org.apache.flink.table.api.TableConfig;
-import org.apache.flink.table.planner.utils.StreamTableTestUtil;
-import org.apache.flink.table.planner.utils.TableTestBase;
-
-import org.junit.Test;
-
-import static org.apache.flink.core.testutils.FlinkMatchers.containsCause;
-
-/** Plan tests for HBase connector, for example, testing projection push down. */
-public class HBaseTablePlanTest extends TableTestBase {
-
-    private final StreamTableTestUtil util = streamTestUtil(TableConfig.getDefault());
-
-    @Test
-    public void testMultipleRowKey() {
-        util.tableEnv()
-                .executeSql(
-                        "CREATE TABLE hTable ("
-                                + " family1 ROW<col1 INT>,"
-                                + " family2 ROW<col1 STRING, col2 BIGINT>,"
-                                + " rowkey INT,"
-                                + " rowkey2 STRING "
-                                + ") WITH ("
-                                + " 'connector' = 'hbase-1.4',"
-                                + " 'table-name' = 'my_table',"
-                                + " 'zookeeper.quorum' = 'localhost:2021'"
-                                + ")");
-        thrown().expect(
-                        containsCause(
-                                new IllegalArgumentException(
-                                        "Row key can't be set multiple times.")));
-        util.verifyExecPlan("SELECT * FROM hTable");
-    }
-
-    @Test
-    public void testNoneRowKey() {
-        util.tableEnv()
-                .executeSql(
-                        "CREATE TABLE hTable ("
-                                + " family1 ROW<col1 INT>,"
-                                + " family2 ROW<col1 STRING, col2 BIGINT>"
-                                + ") WITH ("
-                                + " 'connector' = 'hbase-1.4',"
-                                + " 'table-name' = 'my_table',"
-                                + " 'zookeeper.quorum' = 'localhost:2021'"
-                                + ")");
-        thrown().expect(
-                        containsCause(
-                                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.")));
-        util.verifyExecPlan("SELECT * FROM hTable");
-    }
-
-    @Test
-    public void testInvalidPrimaryKey() {
-        util.tableEnv()
-                .executeSql(
-                        "CREATE TABLE hTable ("
-                                + " family1 ROW<col1 INT>,"
-                                + " family2 ROW<col1 STRING, col2 BIGINT>,"
-                                + " rowkey STRING, "
-                                + " PRIMARY KEY (family1) NOT ENFORCED "
-                                + ") WITH ("
-                                + " 'connector' = 'hbase-1.4',"
-                                + " 'table-name' = 'my_table',"
-                                + " 'zookeeper.quorum' = 'localhost:2021'"
-                                + ")");
-        thrown().expect(
-                        containsCause(
-                                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.")));
-        util.verifyExecPlan("SELECT * FROM hTable");
-    }
-
-    @Test
-    public void testUnsupportedDataType() {
-        util.tableEnv()
-                .executeSql(
-                        "CREATE TABLE hTable ("
-                                + " family1 ROW<col1 INT>,"
-                                + " family2 ROW<col1 STRING, col2 BIGINT>,"
-                                + " col1 ARRAY<STRING>, "
-                                + " rowkey STRING, "
-                                + " PRIMARY KEY (rowkey) NOT ENFORCED "
-                                + ") WITH ("
-                                + " 'connector' = 'hbase-1.4',"
-                                + " 'table-name' = 'my_table',"
-                                + " 'zookeeper.quorum' = 'localhost:2021'"
-                                + ")");
-        thrown().expect(
-                        containsCause(
-                                new IllegalArgumentException(
-                                        "Unsupported field type 'ARRAY<STRING>' for HBase.")));
-        util.verifyExecPlan("SELECT * FROM hTable");
-    }
-
-    @Test
-    public void testProjectionPushDown() {
-        util.tableEnv()
-                .executeSql(
-                        "CREATE TABLE hTable ("
-                                + " family1 ROW<col1 INT>,"
-                                + " family2 ROW<col1 STRING, col2 BIGINT>,"
-                                + " family3 ROW<col1 DOUBLE, col2 BOOLEAN, col3 STRING>,"
-                                + " rowkey INT,"
-                                + " PRIMARY KEY (rowkey) NOT ENFORCED"
-                                + ") WITH ("
-                                + " 'connector' = 'hbase-1.4',"
-                                + " 'table-name' = 'my_table',"
-                                + " 'zookeeper.quorum' = 'localhost:2021'"
-                                + ")");
-        util.verifyExecPlan("SELECT h.family3, h.family2.col2 FROM hTable AS h");
-    }
-}
diff --git a/flink-connectors/flink-connector-hbase-1.4/src/test/java/org/apache/flink/connector/hbase1/util/HBaseTestBase.java b/flink-connectors/flink-connector-hbase-1.4/src/test/java/org/apache/flink/connector/hbase1/util/HBaseTestBase.java
deleted file mode 100644
index cc2de79..0000000
--- a/flink-connectors/flink-connector-hbase-1.4/src/test/java/org/apache/flink/connector/hbase1/util/HBaseTestBase.java
+++ /dev/null
@@ -1,270 +0,0 @@
-/*
- * 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.hbase1.util;
-
-import org.apache.flink.table.api.EnvironmentSettings;
-
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.HTable;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.junit.Before;
-import org.junit.BeforeClass;
-
-import java.io.IOException;
-import java.math.BigDecimal;
-import java.sql.Date;
-import java.sql.Time;
-import java.sql.Timestamp;
-import java.util.ArrayList;
-import java.util.List;
-
-import static org.apache.flink.table.utils.DateTimeUtils.toInternal;
-
-/** Abstract IT case class for HBase. */
-public abstract class HBaseTestBase extends HBaseTestingClusterAutoStarter {
-
-    protected static final String TEST_TABLE_1 = "testTable1";
-    protected static final String TEST_TABLE_2 = "testTable2";
-    protected static final String TEST_TABLE_3 = "testTable3";
-
-    protected static final String ROW_KEY = "rowkey";
-
-    protected static final String FAMILY1 = "family1";
-    protected static final String F1COL1 = "col1";
-
-    protected static final String FAMILY2 = "family2";
-    protected static final String F2COL1 = "col1";
-    protected static final String F2COL2 = "col2";
-
-    protected static final String FAMILY3 = "family3";
-    protected static final String F3COL1 = "col1";
-    protected static final String F3COL2 = "col2";
-    protected static final String F3COL3 = "col3";
-
-    protected static final String FAMILY4 = "family4";
-    protected static final String F4COL1 = "col1";
-    protected static final String F4COL2 = "col2";
-    protected static final String F4COL3 = "col3";
-    protected static final String F4COL4 = "col4";
-
-    private static final byte[][] FAMILIES =
-            new byte[][] {
-                Bytes.toBytes(FAMILY1),
-                Bytes.toBytes(FAMILY2),
-                Bytes.toBytes(FAMILY3),
-                Bytes.toBytes(FAMILY4)
-            };
-
-    private static final byte[][] SPLIT_KEYS = new byte[][] {Bytes.toBytes(4)};
-
-    protected EnvironmentSettings streamSettings;
-    protected EnvironmentSettings batchSettings;
-
-    @BeforeClass
-    public static void activateHBaseCluster() throws IOException {
-        prepareTables();
-    }
-
-    @Before
-    public void before() {
-        this.streamSettings = EnvironmentSettings.inStreamingMode();
-        this.batchSettings = EnvironmentSettings.inBatchMode();
-    }
-
-    private static void prepareTables() throws IOException {
-        createHBaseTable1();
-        createHBaseTable2();
-        createHBaseTable3();
-    }
-
-    private static void createHBaseTable1() throws IOException {
-        // create a table
-        TableName tableName = TableName.valueOf(TEST_TABLE_1);
-        createTable(tableName, FAMILIES, SPLIT_KEYS);
-
-        // get the HTable instance
-        HTable table = openTable(tableName);
-        List<Put> puts = new ArrayList<>();
-        // add some data
-        puts.add(
-                putRow(
-                        1,
-                        10,
-                        "Hello-1",
-                        100L,
-                        1.01,
-                        false,
-                        "Welt-1",
-                        Timestamp.valueOf("2019-08-18 19:00:00"),
-                        Date.valueOf("2019-08-18"),
-                        Time.valueOf("19:00:00"),
-                        new BigDecimal("12345678.0001")));
-        puts.add(
-                putRow(
-                        2,
-                        20,
-                        "Hello-2",
-                        200L,
-                        2.02,
-                        true,
-                        "Welt-2",
-                        Timestamp.valueOf("2019-08-18 19:01:00"),
-                        Date.valueOf("2019-08-18"),
-                        Time.valueOf("19:01:00"),
-                        new BigDecimal("12345678.0002")));
-        puts.add(
-                putRow(
-                        3,
-                        30,
-                        "Hello-3",
-                        300L,
-                        3.03,
-                        false,
-                        "Welt-3",
-                        Timestamp.valueOf("2019-08-18 19:02:00"),
-                        Date.valueOf("2019-08-18"),
-                        Time.valueOf("19:02:00"),
-                        new BigDecimal("12345678.0003")));
-        puts.add(
-                putRow(
-                        4,
-                        40,
-                        null,
-                        400L,
-                        4.04,
-                        true,
-                        "Welt-4",
-                        Timestamp.valueOf("2019-08-18 19:03:00"),
-                        Date.valueOf("2019-08-18"),
-                        Time.valueOf("19:03:00"),
-                        new BigDecimal("12345678.0004")));
-        puts.add(
-                putRow(
-                        5,
-                        50,
-                        "Hello-5",
-                        500L,
-                        5.05,
-                        false,
-                        "Welt-5",
-                        Timestamp.valueOf("2019-08-19 19:10:00"),
-                        Date.valueOf("2019-08-19"),
-                        Time.valueOf("19:10:00"),
-                        new BigDecimal("12345678.0005")));
-        puts.add(
-                putRow(
-                        6,
-                        60,
-                        "Hello-6",
-                        600L,
-                        6.06,
-                        true,
-                        "Welt-6",
-                        Timestamp.valueOf("2019-08-19 19:20:00"),
-                        Date.valueOf("2019-08-19"),
-                        Time.valueOf("19:20:00"),
-                        new BigDecimal("12345678.0006")));
-        puts.add(
-                putRow(
-                        7,
-                        70,
-                        "Hello-7",
-                        700L,
-                        7.07,
-                        false,
-                        "Welt-7",
-                        Timestamp.valueOf("2019-08-19 19:30:00"),
-                        Date.valueOf("2019-08-19"),
-                        Time.valueOf("19:30:00"),
-                        new BigDecimal("12345678.0007")));
-        puts.add(
-                putRow(
-                        8,
-                        80,
-                        null,
-                        800L,
-                        8.08,
-                        true,
-                        "Welt-8",
-                        Timestamp.valueOf("2019-08-19 19:40:00"),
-                        Date.valueOf("2019-08-19"),
-                        Time.valueOf("19:40:00"),
-                        new BigDecimal("12345678.0008")));
-
-        // append rows to table
-        table.put(puts);
-        table.close();
-    }
-
-    private static void createHBaseTable2() {
-        // create a table
-        TableName tableName = TableName.valueOf(TEST_TABLE_2);
-        createTable(tableName, FAMILIES, SPLIT_KEYS);
-    }
-
-    private static void createHBaseTable3() {
-        // create a table
-        byte[][] families =
-                new byte[][] {
-                    Bytes.toBytes(FAMILY1),
-                    Bytes.toBytes(FAMILY2),
-                    Bytes.toBytes(FAMILY3),
-                    Bytes.toBytes(FAMILY4),
-                };
-        TableName tableName = TableName.valueOf(TEST_TABLE_3);
-        createTable(tableName, families, SPLIT_KEYS);
-    }
-
-    private static Put putRow(
-            int rowKey,
-            int f1c1,
-            String f2c1,
-            long f2c2,
-            double f3c1,
-            boolean f3c2,
-            String f3c3,
-            Timestamp f4c1,
-            Date f4c2,
-            Time f4c3,
-            BigDecimal f4c4) {
-        Put put = new Put(Bytes.toBytes(rowKey));
-        // family 1
-        put.addColumn(Bytes.toBytes(FAMILY1), Bytes.toBytes(F1COL1), Bytes.toBytes(f1c1));
-        // family 2
-        if (f2c1 != null) {
-            put.addColumn(Bytes.toBytes(FAMILY2), Bytes.toBytes(F2COL1), Bytes.toBytes(f2c1));
-        }
-        put.addColumn(Bytes.toBytes(FAMILY2), Bytes.toBytes(F2COL2), Bytes.toBytes(f2c2));
-        // family 3
-        put.addColumn(Bytes.toBytes(FAMILY3), Bytes.toBytes(F3COL1), Bytes.toBytes(f3c1));
-        put.addColumn(Bytes.toBytes(FAMILY3), Bytes.toBytes(F3COL2), Bytes.toBytes(f3c2));
-        put.addColumn(Bytes.toBytes(FAMILY3), Bytes.toBytes(F3COL3), Bytes.toBytes(f3c3));
-
-        // family 4
-        put.addColumn(
-                Bytes.toBytes(FAMILY4), Bytes.toBytes(F4COL1), Bytes.toBytes(toInternal(f4c1)));
-        put.addColumn(
-                Bytes.toBytes(FAMILY4), Bytes.toBytes(F4COL2), Bytes.toBytes(toInternal(f4c2)));
-        put.addColumn(
-                Bytes.toBytes(FAMILY4), Bytes.toBytes(F4COL3), Bytes.toBytes(toInternal(f4c3)));
-        put.addColumn(Bytes.toBytes(FAMILY4), Bytes.toBytes(F4COL4), Bytes.toBytes(f4c4));
-        return put;
-    }
-}
diff --git a/flink-connectors/flink-connector-hbase-1.4/src/test/java/org/apache/flink/connector/hbase1/util/HBaseTestingClusterAutoStarter.java b/flink-connectors/flink-connector-hbase-1.4/src/test/java/org/apache/flink/connector/hbase1/util/HBaseTestingClusterAutoStarter.java
deleted file mode 100644
index 45b71e6..0000000
--- a/flink-connectors/flink-connector-hbase-1.4/src/test/java/org/apache/flink/connector/hbase1/util/HBaseTestingClusterAutoStarter.java
+++ /dev/null
@@ -1,181 +0,0 @@
-/*
- * Copyright The Apache Software Foundation
- *
- * 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.hbase1.util;
-
-import org.apache.flink.test.util.AbstractTestBase;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.MasterNotRunningException;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.ZooKeeperConnectionException;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
-import org.apache.hadoop.hbase.client.HTable;
-import org.apache.hadoop.util.VersionUtil;
-import org.junit.AfterClass;
-import org.junit.Assume;
-import org.junit.BeforeClass;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-
-/**
- * By using this class as the super class of a set of tests you will have a HBase testing cluster
- * available that is very suitable for writing tests for scanning and filtering against. This is
- * usable by any downstream application because the HBase cluster is 'injected' because a
- * dynamically generated hbase-site.xml is added to the classpath. Because of this classpath
- * manipulation it is not possible to start a second testing cluster in the same JVM. So if you have
- * this you should either put all hbase related tests in a single class or force surefire to setup a
- * new JVM for each testclass. See:
- * http://maven.apache.org/surefire/maven-surefire-plugin/examples/fork-options-and-parallel-execution.html
- */
-//
-// NOTE: The code in this file is based on code from the
-// Apache HBase project, licensed under the Apache License v 2.0
-//
-// https://github.com/apache/hbase/blob/master/hbase-server/src/test/java/org/apache/hadoop/hbase/filter/FilterTestingCluster.java
-//
-public abstract class HBaseTestingClusterAutoStarter extends AbstractTestBase {
-
-    private static final Log LOG = LogFactory.getLog(HBaseTestingClusterAutoStarter.class);
-
-    private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
-    private static HBaseAdmin admin = null;
-    private static List<TableName> createdTables = new ArrayList<>();
-
-    private static Configuration conf;
-
-    protected static void createTable(
-            TableName tableName, byte[][] columnFamilyName, byte[][] splitKeys) {
-        LOG.info("HBase minicluster: Creating table " + tableName.getNameAsString());
-
-        assertNotNull("HBaseAdmin is not initialized successfully.", admin);
-        HTableDescriptor desc = new HTableDescriptor(tableName);
-        for (byte[] fam : columnFamilyName) {
-            HColumnDescriptor colDef = new HColumnDescriptor(fam);
-            desc.addFamily(colDef);
-        }
-
-        try {
-            admin.createTable(desc, splitKeys);
-            createdTables.add(tableName);
-            assertTrue("Fail to create the table", admin.tableExists(tableName));
-        } catch (IOException e) {
-            assertNull("Exception found while creating table", e);
-        }
-    }
-
-    protected static HTable openTable(TableName tableName) throws IOException {
-        HTable table = (HTable) admin.getConnection().getTable(tableName);
-        assertTrue("Fail to create the table", admin.tableExists(tableName));
-        return table;
-    }
-
-    private static void deleteTables() {
-        if (admin != null) {
-            for (TableName tableName : createdTables) {
-                try {
-                    if (admin.tableExists(tableName)) {
-                        admin.disableTable(tableName);
-                        admin.deleteTable(tableName);
-                    }
-                } catch (IOException e) {
-                    assertNull("Exception found deleting the table", e);
-                }
-            }
-        }
-    }
-
-    private static void initialize(Configuration c) {
-        conf = HBaseConfiguration.create(c);
-        // the default retry number is 35 in hbase-1.4, set 35 for test
-        conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 35);
-        try {
-            admin = TEST_UTIL.getHBaseAdmin();
-        } catch (MasterNotRunningException e) {
-            assertNull("Master is not running", e);
-        } catch (ZooKeeperConnectionException e) {
-            assertNull("Cannot connect to ZooKeeper", e);
-        } catch (IOException e) {
-            assertNull("IOException", e);
-        }
-    }
-
-    @BeforeClass
-    public static void setUp() throws Exception {
-        // HBase 1.4 does not work with Hadoop 3
-        // because it uses Guava 12.0.1, Hadoop 3 uses Guava 27.0-jre.
-        // There is no Guava version in between that works with both.
-        Assume.assumeTrue(
-                "This test is skipped for Hadoop versions above 3",
-                VersionUtil.compareVersions(System.getProperty("hadoop.version"), "3.0.0") < 0);
-
-        LOG.info("HBase minicluster: Starting");
-
-        TEST_UTIL.startMiniCluster(1);
-
-        // https://issues.apache.org/jira/browse/HBASE-11711
-        TEST_UTIL.getConfiguration().setInt("hbase.master.info.port", -1);
-
-        // Make sure the zookeeper quorum value contains the right port number (varies per run).
-        LOG.info("Hbase minicluster client port: " + TEST_UTIL.getZkCluster().getClientPort());
-        TEST_UTIL
-                .getConfiguration()
-                .set(
-                        "hbase.zookeeper.quorum",
-                        "localhost:" + TEST_UTIL.getZkCluster().getClientPort());
-
-        initialize(TEST_UTIL.getConfiguration());
-        LOG.info("HBase minicluster: Running");
-    }
-
-    /** Returns zookeeper quorum value contains the right port number (varies per run). */
-    protected static String getZookeeperQuorum() {
-        return "localhost:" + TEST_UTIL.getZkCluster().getClientPort();
-    }
-
-    public static Configuration getConf() {
-        return conf;
-    }
-
-    @AfterClass
-    public static void tearDown() throws Exception {
-        if (conf == null) {
-            LOG.info("Skipping Hbase tear down. It was never started");
-            return;
-        }
-        LOG.info("HBase minicluster: Shutting down");
-        deleteTables();
-        TEST_UTIL.shutdownMiniCluster();
-        LOG.info("HBase minicluster: Down");
-    }
-}
diff --git a/flink-connectors/flink-connector-hbase-1.4/src/test/java/org/slf4j/impl/Log4jLoggerAdapter.java b/flink-connectors/flink-connector-hbase-1.4/src/test/java/org/slf4j/impl/Log4jLoggerAdapter.java
deleted file mode 100644
index 0287808..0000000
--- a/flink-connectors/flink-connector-hbase-1.4/src/test/java/org/slf4j/impl/Log4jLoggerAdapter.java
+++ /dev/null
@@ -1,22 +0,0 @@
-/*
- * 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.slf4j.impl;
-
-/** Fake appender to work around HBase referring to it directly. */
-public interface Log4jLoggerAdapter {}
diff --git a/flink-connectors/flink-connector-hbase-1.4/src/test/resources/archunit.properties b/flink-connectors/flink-connector-hbase-1.4/src/test/resources/archunit.properties
deleted file mode 100644
index 15be88c..0000000
--- a/flink-connectors/flink-connector-hbase-1.4/src/test/resources/archunit.properties
+++ /dev/null
@@ -1,31 +0,0 @@
-#
-# 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.
-#
-
-# By default we allow removing existing violations, but fail when new violations are added.
-freeze.store.default.allowStoreUpdate=true
-
-# Enable this if a new (frozen) rule has been added in order to create the initial store and record the existing violations.
-#freeze.store.default.allowStoreCreation=true
-
-# Enable this to add allow new violations to be recorded.
-# NOTE: Adding new violations should be avoided when possible. If the rule was correct to flag a new
-#       violation, please try to avoid creating the violation. If the violation was created due to a
-#       shortcoming of the rule, file a JIRA issue so the rule can be improved.
-#freeze.refreeze=true
-
-freeze.store.default.path=archunit-violations
diff --git a/flink-connectors/flink-connector-hbase-1.4/src/test/resources/hbase-site.xml b/flink-connectors/flink-connector-hbase-1.4/src/test/resources/hbase-site.xml
deleted file mode 100644
index 1e58ef4..0000000
--- a/flink-connectors/flink-connector-hbase-1.4/src/test/resources/hbase-site.xml
+++ /dev/null
@@ -1,29 +0,0 @@
-<?xml version="1.0"?>
-<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
-<!--
-  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.
--->
-
-<!-- Values used when running unit tests.  Specify any values in here that
-     should override the default values. -->
-
-<configuration>
-    <property>
-        <name>hbase_conf_key</name>
-        <value>hbase_conf_value!</value>
-    </property>
-</configuration>
diff --git a/flink-connectors/flink-connector-hbase-1.4/src/test/resources/log4j2-test.properties b/flink-connectors/flink-connector-hbase-1.4/src/test/resources/log4j2-test.properties
deleted file mode 100644
index 835c2ec..0000000
--- a/flink-connectors/flink-connector-hbase-1.4/src/test/resources/log4j2-test.properties
+++ /dev/null
@@ -1,28 +0,0 @@
-################################################################################
-#  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.
-################################################################################
-
-# Set root logger level to OFF to not flood build logs
-# set manually to INFO for debugging purposes
-rootLogger.level = OFF
-rootLogger.appenderRef.test.ref = TestLogger
-
-appender.testlogger.name = TestLogger
-appender.testlogger.type = CONSOLE
-appender.testlogger.target = SYSTEM_ERR
-appender.testlogger.layout.type = PatternLayout
-appender.testlogger.layout.pattern = %-4r [%t] %-5p %c %x - %m%n
diff --git a/flink-connectors/flink-connector-hbase-1.4/src/test/resources/org/apache/flink/connector/hbase1/HBaseTablePlanTest.xml b/flink-connectors/flink-connector-hbase-1.4/src/test/resources/org/apache/flink/connector/hbase1/HBaseTablePlanTest.xml
deleted file mode 100644
index 9885ccb..0000000
--- a/flink-connectors/flink-connector-hbase-1.4/src/test/resources/org/apache/flink/connector/hbase1/HBaseTablePlanTest.xml
+++ /dev/null
@@ -1,36 +0,0 @@
-<?xml version="1.0" ?>
-<!--
-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.
--->
-<Root>
-  <TestCase name="testProjectionPushDown">
-    <Resource name="sql">
-      <![CDATA[SELECT h.family3, h.family2.col2 FROM hTable AS h]]>
-    </Resource>
-    <Resource name="ast">
-      <![CDATA[
-LogicalProject(family3=[$2], col2=[$1.col2])
-+- LogicalTableScan(table=[[default_catalog, default_database, hTable]])
-]]>
-    </Resource>
-    <Resource name="optimized exec plan">
-      <![CDATA[
-Calc(select=[family3, family2.col2 AS col2])
-+- TableSourceScan(table=[[default_catalog, default_database, hTable, project=[family3, family2]]], fields=[family3, family2])
-]]>
-    </Resource>
-  </TestCase>
-</Root>
diff --git a/flink-connectors/flink-connector-hbase-2.2/archunit-violations/346fff47-b126-4f8c-ba73-799d99856912 b/flink-connectors/flink-connector-hbase-2.2/archunit-violations/346fff47-b126-4f8c-ba73-799d99856912
deleted file mode 100644
index e69de29..0000000
--- a/flink-connectors/flink-connector-hbase-2.2/archunit-violations/346fff47-b126-4f8c-ba73-799d99856912
+++ /dev/null
diff --git a/flink-connectors/flink-connector-hbase-2.2/archunit-violations/ac3990d6-8d39-4299-8765-3ae2f91d76af b/flink-connectors/flink-connector-hbase-2.2/archunit-violations/ac3990d6-8d39-4299-8765-3ae2f91d76af
deleted file mode 100644
index e69de29..0000000
--- a/flink-connectors/flink-connector-hbase-2.2/archunit-violations/ac3990d6-8d39-4299-8765-3ae2f91d76af
+++ /dev/null
diff --git a/flink-connectors/flink-connector-hbase-2.2/archunit-violations/stored.rules b/flink-connectors/flink-connector-hbase-2.2/archunit-violations/stored.rules
deleted file mode 100644
index 642f0d0..0000000
--- a/flink-connectors/flink-connector-hbase-2.2/archunit-violations/stored.rules
+++ /dev/null
@@ -1,4 +0,0 @@
-#
-#Tue Feb 22 12:17:50 CET 2022
-Tests\ inheriting\ from\ AbstractTestBase\ should\ have\ name\ ending\ with\ ITCase=346fff47-b126-4f8c-ba73-799d99856912
-ITCASE\ tests\ should\ use\ a\ MiniCluster\ resource\ or\ extension=ac3990d6-8d39-4299-8765-3ae2f91d76af
diff --git a/flink-connectors/flink-connector-hbase-2.2/pom.xml b/flink-connectors/flink-connector-hbase-2.2/pom.xml
deleted file mode 100644
index fef7e56..0000000
--- a/flink-connectors/flink-connector-hbase-2.2/pom.xml
+++ /dev/null
@@ -1,487 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-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.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
-
-	<modelVersion>4.0.0</modelVersion>
-
-	<parent>
-		<groupId>org.apache.flink</groupId>
-		<artifactId>flink-connectors</artifactId>
-		<version>1.19-SNAPSHOT</version>
-	</parent>
-
-	<artifactId>flink-connector-hbase-2.2</artifactId>
-	<name>Flink : Connectors : HBase 2.2</name>
-	<packaging>jar</packaging>
-
-	<properties>
-		<hbase.version>2.2.3</hbase.version>
-	</properties>
-
-	<dependencies>
-
-		<!-- Core -->
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-core</artifactId>
-			<version>${project.version}</version>
-			<scope>provided</scope>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-java</artifactId>
-			<version>${project.version}</version>
-			<scope>provided</scope>
-		</dependency>
-
-		<!-- Flink HBase -->
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-connector-hbase-base</artifactId>
-			<version>${project.version}</version>
-			<exclusions>
-				<exclusion>
-					<groupId>org.apache.hbase</groupId>
-					<artifactId>hbase-server</artifactId>
-				</exclusion>
-			</exclusions>
-		</dependency>
-
-		<!-- Table ecosystem -->
-
-		<!-- Projects depending on this project won't depend on flink-table-*. -->
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-table-api-java-bridge</artifactId>
-			<version>${project.version}</version>
-			<scope>provided</scope>
-			<optional>true</optional>
-		</dependency>
-
-		<!-- HBase -->
-
-		<dependency>
-			<groupId>org.apache.hadoop</groupId>
-			<artifactId>hadoop-common</artifactId>
-			<scope>provided</scope>
-			<exclusions>
-				<exclusion>
-					<groupId>ch.qos.reload4j</groupId>
-					<artifactId>reload4j</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.slf4j</groupId>
-					<artifactId>slf4j-reload4j</artifactId>
-				</exclusion>
-			</exclusions>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.hbase</groupId>
-			<artifactId>hbase-client</artifactId>
-			<version>${hbase.version}</version>
-			<exclusions>
-				<!-- Remove unneeded dependency, which is conflicting with our jetty-util version. -->
-				<exclusion>
-					<groupId>org.mortbay.jetty</groupId>
-					<artifactId>jetty-util</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.mortbay.jetty</groupId>
-					<artifactId>jetty</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.mortbay.jetty</groupId>
-					<artifactId>jetty-sslengine</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.mortbay.jetty</groupId>
-					<artifactId>jsp-2.1</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.mortbay.jetty</groupId>
-					<artifactId>jsp-api-2.1</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.mortbay.jetty</groupId>
-					<artifactId>servlet-api-2.5</artifactId>
-				</exclusion>
-				<!-- Bug in hbase annotations, can be removed when fixed. See FLINK-2153. -->
-				<exclusion>
-					<groupId>org.apache.hbase</groupId>
-					<artifactId>hbase-annotations</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>com.sun.jersey</groupId>
-					<artifactId>jersey-core</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.apache.hadoop</groupId>
-					<artifactId>hadoop-common</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.apache.hadoop</groupId>
-					<artifactId>hadoop-auth</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.apache.hadoop</groupId>
-					<artifactId>hadoop-annotations</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.apache.hadoop</groupId>
-					<artifactId>hadoop-mapreduce-client-core</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.apache.hadoop</groupId>
-					<artifactId>hadoop-client</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.apache.hadoop</groupId>
-					<artifactId>hadoop-hdfs</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>log4j</groupId>
-					<artifactId>log4j</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.slf4j</groupId>
-					<artifactId>slf4j-log4j12</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>ch.qos.reload4j</groupId>
-					<artifactId>reload4j</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.slf4j</groupId>
-					<artifactId>slf4j-reload4j</artifactId>
-				</exclusion>
-			</exclusions>
-		</dependency>
-
-		<dependency>
-			<!-- Bump hbase netty dependency -->
-			<groupId>io.netty</groupId>
-			<artifactId>netty-all</artifactId>
-			<version>4.1.46.Final</version>
-		</dependency>
-
-		<dependency>
-			<!-- Replaces Hbase log4j dependency -->
-			<groupId>org.apache.logging.log4j</groupId>
-			<artifactId>log4j-1.2-api</artifactId>
-			<scope>provided</scope>
-		</dependency>
-
-		<!-- Tests -->
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-connector-hbase-base</artifactId>
-			<version>${project.version}</version>
-			<exclusions>
-				<!-- exclude HBase dependencies -->
-				<exclusion>
-					<groupId>org.apache.hbase</groupId>
-					<artifactId>hbase-server</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.apache.hbase</groupId>
-					<artifactId>hbase-hadoop-compat</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.apache.hbase</groupId>
-					<artifactId>hbase-hadoop2-compat</artifactId>
-				</exclusion>
-			</exclusions>
-			<type>test-jar</type>
-			<scope>test</scope>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-clients</artifactId>
-			<version>${project.version}</version>
-			<scope>test</scope>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-hadoop-compatibility_${scala.binary.version}</artifactId>
-			<version>${project.version}</version>
-			<scope>test</scope>
-			<exclusions>
-				<exclusion>
-					<groupId>org.apache.flink</groupId>
-					<artifactId>flink-shaded-include-yarn_${scala.binary.version}</artifactId>
-				</exclusion>
-			</exclusions>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-test-utils</artifactId>
-			<version>${project.version}</version>
-			<scope>test</scope>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-table-common</artifactId>
-			<version>${project.version}</version>
-			<type>test-jar</type>
-			<scope>test</scope>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-table-planner_${scala.binary.version}</artifactId>
-			<version>${project.version}</version>
-			<scope>test</scope>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-table-api-scala-bridge_${scala.binary.version}</artifactId>
-			<version>${project.version}</version>
-			<scope>test</scope>
-		</dependency>
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-table-planner_${scala.binary.version}</artifactId>
-			<version>${project.version}</version>
-			<type>test-jar</type>
-			<scope>test</scope>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.hbase</groupId>
-			<artifactId>hbase-server</artifactId>
-			<version>${hbase.version}</version>
-			<type>test-jar</type>
-			<scope>test</scope>
-			<exclusions>
-				<exclusion>
-					<groupId>log4j</groupId>
-					<artifactId>log4j</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.slf4j</groupId>
-					<artifactId>slf4j-log4j12</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.glassfish</groupId>
-					<artifactId>javax.el</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>ch.qos.reload4j</groupId>
-					<artifactId>reload4j</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.slf4j</groupId>
-					<artifactId>slf4j-reload4j</artifactId>
-				</exclusion>
-			</exclusions>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.hbase</groupId>
-			<artifactId>hbase-zookeeper</artifactId>
-			<version>${hbase.version}</version>
-			<type>test-jar</type>
-			<scope>test</scope>
-			<exclusions>
-				<exclusion>
-					<groupId>log4j</groupId>
-					<artifactId>log4j</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.slf4j</groupId>
-					<artifactId>slf4j-log4j12</artifactId>
-				</exclusion>
-			</exclusions>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.hbase</groupId>
-			<artifactId>hbase-common</artifactId>
-			<version>${hbase.version}</version>
-			<type>test-jar</type>
-			<scope>test</scope>
-			<exclusions>
-				<exclusion>
-					<groupId>log4j</groupId>
-					<artifactId>log4j</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.slf4j</groupId>
-					<artifactId>slf4j-log4j12</artifactId>
-				</exclusion>
-			</exclusions>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.hbase</groupId>
-			<artifactId>hbase-mapreduce</artifactId>
-			<version>${hbase.version}</version>
-			<scope>test</scope>
-			<exclusions>
-				<exclusion>
-					<groupId>log4j</groupId>
-					<artifactId>log4j</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.slf4j</groupId>
-					<artifactId>slf4j-log4j12</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.glassfish</groupId>
-					<artifactId>javax.el</artifactId>
-				</exclusion>
-			</exclusions>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.hadoop</groupId>
-			<artifactId>hadoop-minicluster</artifactId>
-			<version>${flink.hadoop.version}</version>
-			<scope>test</scope>
-			<exclusions>
-				<exclusion>
-					<groupId>log4j</groupId>
-					<artifactId>log4j</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.slf4j</groupId>
-					<artifactId>slf4j-log4j12</artifactId>
-				</exclusion>
-				<exclusion>
-					<!-- This dependency is no longer shipped with the JDK since Java 9.-->
-					<groupId>jdk.tools</groupId>
-					<artifactId>jdk.tools</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>ch.qos.reload4j</groupId>
-					<artifactId>reload4j</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.slf4j</groupId>
-					<artifactId>slf4j-reload4j</artifactId>
-				</exclusion>
-			</exclusions>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.hbase</groupId>
-			<artifactId>hbase-hadoop-compat</artifactId>
-			<version>${hbase.version}</version>
-			<scope>test</scope>
-			<type>test-jar</type>
-			<exclusions>
-				<exclusion>
-					<groupId>log4j</groupId>
-					<artifactId>log4j</artifactId>
-				</exclusion>
-			</exclusions>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.hadoop</groupId>
-			<artifactId>hadoop-hdfs</artifactId>
-			<version>${flink.hadoop.version}</version>
-			<type>test-jar</type>
-			<scope>test</scope>
-			<exclusions>
-				<exclusion>
-					<groupId>log4j</groupId>
-					<artifactId>log4j</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>ch.qos.reload4j</groupId>
-					<artifactId>reload4j</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.slf4j</groupId>
-					<artifactId>slf4j-reload4j</artifactId>
-				</exclusion>
-			</exclusions>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.hbase</groupId>
-			<artifactId>hbase-hadoop2-compat</artifactId>
-			<version>${hbase.version}</version>
-			<scope>test</scope>
-			<type>test-jar</type>
-			<exclusions>
-				<exclusion>
-					<groupId>log4j</groupId>
-					<artifactId>log4j</artifactId>
-				</exclusion>
-			</exclusions>
-		</dependency>
-
-		<!-- ArchUit test dependencies -->
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-architecture-tests-test</artifactId>
-			<scope>test</scope>
-		</dependency>
-
-	</dependencies>
-
-	<dependencyManagement>
-		<dependencies>
-			<dependency>
-				<!-- HBase only works with Zookeper 3.4 -->
-				<groupId>org.apache.zookeeper</groupId>
-				<artifactId>zookeeper</artifactId>
-				<version>3.4.14</version>
-			</dependency>
-		</dependencies>
-	</dependencyManagement>
-
-	<profiles>
-		<profile>
-			<id>java17</id>
-			<activation>
-				<jdk>[17,)</jdk>
-			</activation>
-
-			<build>
-				<plugins>
-					<plugin>
-						<groupId>org.apache.maven.plugins</groupId>
-						<artifactId>maven-surefire-plugin</artifactId>
-						<configuration>
-							<!-- hbase currently does not support Java 17, see HBASE-26038 -->
-							<skip>true</skip>
-						</configuration>
-					</plugin>
-				</plugins>
-			</build>
-		</profile>
-	</profiles>
-
-</project>
diff --git a/flink-connectors/flink-connector-hbase-2.2/src/main/java/org/apache/flink/connector/hbase2/HBase2DynamicTableFactory.java b/flink-connectors/flink-connector-hbase-2.2/src/main/java/org/apache/flink/connector/hbase2/HBase2DynamicTableFactory.java
deleted file mode 100644
index 8c55137..0000000
--- a/flink-connectors/flink-connector-hbase-2.2/src/main/java/org/apache/flink/connector/hbase2/HBase2DynamicTableFactory.java
+++ /dev/null
@@ -1,183 +0,0 @@
-/*
- * 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.hbase2;
-
-import org.apache.flink.annotation.Internal;
-import org.apache.flink.configuration.ConfigOption;
-import org.apache.flink.configuration.ReadableConfig;
-import org.apache.flink.connector.hbase.options.HBaseWriteOptions;
-import org.apache.flink.connector.hbase.util.HBaseTableSchema;
-import org.apache.flink.connector.hbase2.sink.HBaseDynamicTableSink;
-import org.apache.flink.connector.hbase2.source.HBaseDynamicTableSource;
-import org.apache.flink.table.connector.sink.DynamicTableSink;
-import org.apache.flink.table.connector.source.DynamicTableSource;
-import org.apache.flink.table.connector.source.lookup.LookupOptions;
-import org.apache.flink.table.connector.source.lookup.cache.DefaultLookupCache;
-import org.apache.flink.table.connector.source.lookup.cache.LookupCache;
-import org.apache.flink.table.factories.DynamicTableSinkFactory;
-import org.apache.flink.table.factories.DynamicTableSourceFactory;
-import org.apache.flink.table.factories.FactoryUtil.TableFactoryHelper;
-
-import org.apache.hadoop.conf.Configuration;
-
-import java.time.Duration;
-import java.util.HashSet;
-import java.util.Set;
-import java.util.stream.Collectors;
-import java.util.stream.Stream;
-
-import static org.apache.flink.connector.hbase.table.HBaseConnectorOptions.LOOKUP_ASYNC;
-import static org.apache.flink.connector.hbase.table.HBaseConnectorOptions.LOOKUP_CACHE_MAX_ROWS;
-import static org.apache.flink.connector.hbase.table.HBaseConnectorOptions.LOOKUP_CACHE_TTL;
-import static org.apache.flink.connector.hbase.table.HBaseConnectorOptions.LOOKUP_MAX_RETRIES;
-import static org.apache.flink.connector.hbase.table.HBaseConnectorOptions.NULL_STRING_LITERAL;
-import static org.apache.flink.connector.hbase.table.HBaseConnectorOptions.SINK_BUFFER_FLUSH_INTERVAL;
-import static org.apache.flink.connector.hbase.table.HBaseConnectorOptions.SINK_BUFFER_FLUSH_MAX_ROWS;
-import static org.apache.flink.connector.hbase.table.HBaseConnectorOptions.SINK_BUFFER_FLUSH_MAX_SIZE;
-import static org.apache.flink.connector.hbase.table.HBaseConnectorOptions.SINK_PARALLELISM;
-import static org.apache.flink.connector.hbase.table.HBaseConnectorOptions.TABLE_NAME;
-import static org.apache.flink.connector.hbase.table.HBaseConnectorOptions.ZOOKEEPER_QUORUM;
-import static org.apache.flink.connector.hbase.table.HBaseConnectorOptions.ZOOKEEPER_ZNODE_PARENT;
-import static org.apache.flink.connector.hbase.table.HBaseConnectorOptionsUtil.PROPERTIES_PREFIX;
-import static org.apache.flink.connector.hbase.table.HBaseConnectorOptionsUtil.getHBaseConfiguration;
-import static org.apache.flink.connector.hbase.table.HBaseConnectorOptionsUtil.getHBaseWriteOptions;
-import static org.apache.flink.connector.hbase.table.HBaseConnectorOptionsUtil.validatePrimaryKey;
-import static org.apache.flink.table.factories.FactoryUtil.createTableFactoryHelper;
-
-/** HBase connector factory. */
-@Internal
-public class HBase2DynamicTableFactory
-        implements DynamicTableSourceFactory, DynamicTableSinkFactory {
-
-    private static final String IDENTIFIER = "hbase-2.2";
-
-    @Override
-    public DynamicTableSource createDynamicTableSource(Context context) {
-        TableFactoryHelper helper = createTableFactoryHelper(this, context);
-        helper.validateExcept(PROPERTIES_PREFIX);
-
-        final ReadableConfig tableOptions = helper.getOptions();
-
-        validatePrimaryKey(context.getPhysicalRowDataType(), context.getPrimaryKeyIndexes());
-
-        String tableName = tableOptions.get(TABLE_NAME);
-        Configuration hbaseConf = getHBaseConfiguration(tableOptions);
-        String nullStringLiteral = tableOptions.get(NULL_STRING_LITERAL);
-        HBaseTableSchema hbaseSchema =
-                HBaseTableSchema.fromDataType(context.getPhysicalRowDataType());
-
-        LookupCache cache = null;
-
-        // Backward compatible to legacy caching options
-        if (tableOptions.get(LOOKUP_CACHE_MAX_ROWS) > 0
-                && tableOptions.get(LOOKUP_CACHE_TTL).compareTo(Duration.ZERO) > 0) {
-            cache =
-                    DefaultLookupCache.newBuilder()
-                            .maximumSize(tableOptions.get(LOOKUP_CACHE_MAX_ROWS))
-                            .expireAfterWrite(tableOptions.get(LOOKUP_CACHE_TTL))
-                            .build();
-        }
-
-        if (tableOptions
-                .get(LookupOptions.CACHE_TYPE)
-                .equals(LookupOptions.LookupCacheType.PARTIAL)) {
-            cache = DefaultLookupCache.fromConfig(tableOptions);
-        }
-
-        return new HBaseDynamicTableSource(
-                hbaseConf,
-                tableName,
-                hbaseSchema,
-                nullStringLiteral,
-                tableOptions.get(LookupOptions.MAX_RETRIES),
-                tableOptions.get(LOOKUP_ASYNC),
-                cache);
-    }
-
-    @Override
-    public DynamicTableSink createDynamicTableSink(Context context) {
-        TableFactoryHelper helper = createTableFactoryHelper(this, context);
-        helper.validateExcept(PROPERTIES_PREFIX);
-
-        final ReadableConfig tableOptions = helper.getOptions();
-
-        validatePrimaryKey(context.getPhysicalRowDataType(), context.getPrimaryKeyIndexes());
-
-        String tableName = tableOptions.get(TABLE_NAME);
-        Configuration hbaseConf = getHBaseConfiguration(tableOptions);
-        HBaseWriteOptions hBaseWriteOptions = getHBaseWriteOptions(tableOptions);
-        String nullStringLiteral = tableOptions.get(NULL_STRING_LITERAL);
-        HBaseTableSchema hbaseSchema =
-                HBaseTableSchema.fromDataType(context.getPhysicalRowDataType());
-
-        return new HBaseDynamicTableSink(
-                tableName, hbaseSchema, hbaseConf, hBaseWriteOptions, nullStringLiteral);
-    }
-
-    @Override
-    public String factoryIdentifier() {
-        return IDENTIFIER;
-    }
-
-    @Override
-    public Set<ConfigOption<?>> requiredOptions() {
-        Set<ConfigOption<?>> set = new HashSet<>();
-        set.add(TABLE_NAME);
-        return set;
-    }
-
-    @Override
-    public Set<ConfigOption<?>> optionalOptions() {
-        Set<ConfigOption<?>> set = new HashSet<>();
-        set.add(ZOOKEEPER_ZNODE_PARENT);
-        set.add(ZOOKEEPER_QUORUM);
-        set.add(NULL_STRING_LITERAL);
-        set.add(SINK_BUFFER_FLUSH_MAX_SIZE);
-        set.add(SINK_BUFFER_FLUSH_MAX_ROWS);
-        set.add(SINK_BUFFER_FLUSH_INTERVAL);
-        set.add(SINK_PARALLELISM);
-        set.add(LOOKUP_ASYNC);
-        set.add(LOOKUP_CACHE_MAX_ROWS);
-        set.add(LOOKUP_CACHE_TTL);
-        set.add(LOOKUP_MAX_RETRIES);
-        set.add(LookupOptions.CACHE_TYPE);
-        set.add(LookupOptions.MAX_RETRIES);
-        set.add(LookupOptions.PARTIAL_CACHE_EXPIRE_AFTER_ACCESS);
-        set.add(LookupOptions.PARTIAL_CACHE_EXPIRE_AFTER_WRITE);
-        set.add(LookupOptions.PARTIAL_CACHE_CACHE_MISSING_KEY);
-        set.add(LookupOptions.PARTIAL_CACHE_MAX_ROWS);
-        return set;
-    }
-
-    @Override
-    public Set<ConfigOption<?>> forwardOptions() {
-        return Stream.of(
-                        TABLE_NAME,
-                        ZOOKEEPER_ZNODE_PARENT,
-                        ZOOKEEPER_QUORUM,
-                        NULL_STRING_LITERAL,
-                        LOOKUP_CACHE_MAX_ROWS,
-                        LOOKUP_CACHE_TTL,
-                        LOOKUP_MAX_RETRIES,
-                        SINK_BUFFER_FLUSH_MAX_SIZE,
-                        SINK_BUFFER_FLUSH_MAX_ROWS,
-                        SINK_BUFFER_FLUSH_INTERVAL)
-                .collect(Collectors.toSet());
-    }
-}
diff --git a/flink-connectors/flink-connector-hbase-2.2/src/main/java/org/apache/flink/connector/hbase2/sink/HBaseDynamicTableSink.java b/flink-connectors/flink-connector-hbase-2.2/src/main/java/org/apache/flink/connector/hbase2/sink/HBaseDynamicTableSink.java
deleted file mode 100644
index 6ea9ba3..0000000
--- a/flink-connectors/flink-connector-hbase-2.2/src/main/java/org/apache/flink/connector/hbase2/sink/HBaseDynamicTableSink.java
+++ /dev/null
@@ -1,116 +0,0 @@
-/*
- * 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.hbase2.sink;
-
-import org.apache.flink.annotation.Internal;
-import org.apache.flink.annotation.VisibleForTesting;
-import org.apache.flink.connector.hbase.options.HBaseWriteOptions;
-import org.apache.flink.connector.hbase.sink.HBaseSinkFunction;
-import org.apache.flink.connector.hbase.sink.RowDataToMutationConverter;
-import org.apache.flink.connector.hbase.util.HBaseTableSchema;
-import org.apache.flink.table.connector.ChangelogMode;
-import org.apache.flink.table.connector.sink.DynamicTableSink;
-import org.apache.flink.table.connector.sink.SinkFunctionProvider;
-import org.apache.flink.table.data.RowData;
-import org.apache.flink.types.RowKind;
-
-import org.apache.hadoop.conf.Configuration;
-
-/** HBase table sink implementation. */
-@Internal
-public class HBaseDynamicTableSink implements DynamicTableSink {
-
-    private final String tableName;
-    private final HBaseTableSchema hbaseTableSchema;
-    private final Configuration hbaseConf;
-    private final HBaseWriteOptions writeOptions;
-    private final String nullStringLiteral;
-
-    public HBaseDynamicTableSink(
-            String tableName,
-            HBaseTableSchema hbaseTableSchema,
-            Configuration hbaseConf,
-            HBaseWriteOptions writeOptions,
-            String nullStringLiteral) {
-
-        this.tableName = tableName;
-        this.hbaseTableSchema = hbaseTableSchema;
-        this.hbaseConf = hbaseConf;
-        this.writeOptions = writeOptions;
-        this.nullStringLiteral = nullStringLiteral;
-    }
-
-    @Override
-    public SinkRuntimeProvider getSinkRuntimeProvider(Context context) {
-        HBaseSinkFunction<RowData> sinkFunction =
-                new HBaseSinkFunction<>(
-                        tableName,
-                        hbaseConf,
-                        new RowDataToMutationConverter(hbaseTableSchema, nullStringLiteral),
-                        writeOptions.getBufferFlushMaxSizeInBytes(),
-                        writeOptions.getBufferFlushMaxRows(),
-                        writeOptions.getBufferFlushIntervalMillis());
-        return SinkFunctionProvider.of(sinkFunction, writeOptions.getParallelism());
-    }
-
-    @Override
-    public ChangelogMode getChangelogMode(ChangelogMode requestedMode) {
-        // UPSERT mode
-        ChangelogMode.Builder builder = ChangelogMode.newBuilder();
-        for (RowKind kind : requestedMode.getContainedKinds()) {
-            if (kind != RowKind.UPDATE_BEFORE) {
-                builder.addContainedKind(kind);
-            }
-        }
-        return builder.build();
-    }
-
-    @Override
-    public DynamicTableSink copy() {
-        return new HBaseDynamicTableSink(
-                tableName, hbaseTableSchema, hbaseConf, writeOptions, nullStringLiteral);
-    }
-
-    @Override
-    public String asSummaryString() {
-        return "HBase";
-    }
-
-    // -------------------------------------------------------------------------------------------
-
-    @VisibleForTesting
-    public HBaseTableSchema getHBaseTableSchema() {
-        return this.hbaseTableSchema;
-    }
-
-    @VisibleForTesting
-    public HBaseWriteOptions getWriteOptions() {
-        return writeOptions;
-    }
-
-    @VisibleForTesting
-    public Configuration getConfiguration() {
-        return this.hbaseConf;
-    }
-
-    @VisibleForTesting
-    public String getTableName() {
-        return this.tableName;
-    }
-}
diff --git a/flink-connectors/flink-connector-hbase-2.2/src/main/java/org/apache/flink/connector/hbase2/source/AbstractTableInputFormat.java b/flink-connectors/flink-connector-hbase-2.2/src/main/java/org/apache/flink/connector/hbase2/source/AbstractTableInputFormat.java
deleted file mode 100644
index d340fe8..0000000
--- a/flink-connectors/flink-connector-hbase-2.2/src/main/java/org/apache/flink/connector/hbase2/source/AbstractTableInputFormat.java
+++ /dev/null
@@ -1,318 +0,0 @@
-/*
- * 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.hbase2.source;
-
-import org.apache.flink.annotation.Internal;
-import org.apache.flink.annotation.VisibleForTesting;
-import org.apache.flink.api.common.io.InputFormat;
-import org.apache.flink.api.common.io.LocatableInputSplitAssigner;
-import org.apache.flink.api.common.io.RichInputFormat;
-import org.apache.flink.api.common.io.statistics.BaseStatistics;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.connector.hbase.source.TableInputSplit;
-import org.apache.flink.connector.hbase.util.HBaseConfigurationUtil;
-import org.apache.flink.core.io.InputSplitAssigner;
-import org.apache.flink.util.IOUtils;
-
-import org.apache.hadoop.hbase.client.Connection;
-import org.apache.hadoop.hbase.client.HTable;
-import org.apache.hadoop.hbase.client.RegionLocator;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.ResultScanner;
-import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.client.Table;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.Pair;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-
-/** Abstract {@link InputFormat} to read data from HBase tables. */
-@Internal
-public abstract class AbstractTableInputFormat<T> extends RichInputFormat<T, TableInputSplit> {
-
-    protected static final Logger LOG = LoggerFactory.getLogger(AbstractTableInputFormat.class);
-    private static final long serialVersionUID = 1L;
-
-    // helper variable to decide whether the input is exhausted or not
-    protected boolean endReached = false;
-
-    protected transient Connection connection = null;
-    protected transient Table table = null;
-    protected transient RegionLocator regionLocator = null;
-    protected transient Scan scan = null;
-
-    /** HBase iterator wrapper. */
-    protected ResultScanner resultScanner = null;
-
-    protected byte[] currentRow;
-    protected long scannedRows;
-
-    // Configuration is not serializable
-    protected byte[] serializedConfig;
-
-    public AbstractTableInputFormat(org.apache.hadoop.conf.Configuration hConf) {
-        serializedConfig = HBaseConfigurationUtil.serializeConfiguration(hConf);
-    }
-
-    /**
-     * Creates a {@link Scan} object and opens the {@link HTable} connection to initialize the HBase
-     * table.
-     *
-     * @throws IOException Thrown, if the connection could not be opened due to an I/O problem.
-     */
-    protected abstract void initTable() throws IOException;
-
-    /**
-     * Returns an instance of Scan that retrieves the required subset of records from the HBase
-     * table.
-     *
-     * @return The appropriate instance of Scan for this use case.
-     */
-    protected abstract Scan getScanner();
-
-    /**
-     * What table is to be read.
-     *
-     * <p>Per instance of a TableInputFormat derivative only a single table name is possible.
-     *
-     * @return The name of the table
-     */
-    protected abstract String getTableName();
-
-    /**
-     * HBase returns an instance of {@link Result}.
-     *
-     * <p>This method maps the returned {@link Result} instance into the output type {@link T}.
-     *
-     * @param r The Result instance from HBase that needs to be converted
-     * @return The appropriate instance of {@link T} that contains the data of Result.
-     */
-    protected abstract T mapResultToOutType(Result r);
-
-    @Override
-    public void configure(Configuration parameters) {}
-
-    protected org.apache.hadoop.conf.Configuration getHadoopConfiguration() {
-        return HBaseConfigurationUtil.deserializeConfiguration(
-                serializedConfig, HBaseConfigurationUtil.getHBaseConfiguration());
-    }
-
-    /**
-     * Creates a {@link Scan} object and opens the {@link HTable} connection. The connection is
-     * opened in this method and closed in {@link #close()}.
-     *
-     * @param split The split to be opened.
-     * @throws IOException Thrown, if the spit could not be opened due to an I/O problem.
-     */
-    @Override
-    public void open(TableInputSplit split) throws IOException {
-        initTable();
-
-        if (split == null) {
-            throw new IOException("Input split is null!");
-        }
-
-        logSplitInfo("opening", split);
-
-        // set scan range
-        currentRow = split.getStartRow();
-        scan.setStartRow(currentRow);
-        scan.setStopRow(split.getEndRow());
-
-        resultScanner = table.getScanner(scan);
-        endReached = false;
-        scannedRows = 0;
-    }
-
-    @Override
-    public T nextRecord(T reuse) throws IOException {
-        if (resultScanner == null) {
-            throw new IOException("No table result scanner provided!");
-        }
-        Result res;
-        try {
-            res = resultScanner.next();
-        } catch (Exception e) {
-            resultScanner.close();
-            // workaround for timeout on scan
-            LOG.warn(
-                    "Error after scan of " + scannedRows + " rows. Retry with a new scanner...", e);
-            scan.withStartRow(currentRow, false);
-            resultScanner = table.getScanner(scan);
-            res = resultScanner.next();
-        }
-
-        if (res != null) {
-            scannedRows++;
-            currentRow = res.getRow();
-            return mapResultToOutType(res);
-        }
-
-        endReached = true;
-        return null;
-    }
-
-    private void logSplitInfo(String action, TableInputSplit split) {
-        int splitId = split.getSplitNumber();
-        String splitStart = Bytes.toString(split.getStartRow());
-        String splitEnd = Bytes.toString(split.getEndRow());
-        String splitStartKey = splitStart.isEmpty() ? "-" : splitStart;
-        String splitStopKey = splitEnd.isEmpty() ? "-" : splitEnd;
-        String[] hostnames = split.getHostnames();
-        LOG.info(
-                "{} split (this={})[{}|{}|{}|{}]",
-                action,
-                this,
-                splitId,
-                hostnames,
-                splitStartKey,
-                splitStopKey);
-    }
-
-    @Override
-    public boolean reachedEnd() throws IOException {
-        return endReached;
-    }
-
-    @Override
-    public void close() throws IOException {
-        LOG.info("Closing split (scanned {} rows)", scannedRows);
-        currentRow = null;
-        IOUtils.closeQuietly(resultScanner);
-        resultScanner = null;
-        closeTable();
-    }
-
-    public void closeTable() {
-        if (table != null) {
-            try {
-                table.close();
-            } catch (IOException e) {
-                LOG.warn("Exception occurs while closing HBase Table.", e);
-            }
-            table = null;
-        }
-        if (connection != null) {
-            try {
-                connection.close();
-            } catch (IOException e) {
-                LOG.warn("Exception occurs while closing HBase Connection.", e);
-            }
-            connection = null;
-        }
-    }
-
-    @Override
-    public TableInputSplit[] createInputSplits(final int minNumSplits) throws IOException {
-        try {
-            initTable();
-
-            // Get the starting and ending row keys for every region in the currently open table
-            final Pair<byte[][], byte[][]> keys = regionLocator.getStartEndKeys();
-            if (keys == null || keys.getFirst() == null || keys.getFirst().length == 0) {
-                LOG.warn(
-                        "Unexpected region keys: {} appeared in HBase table: {}, all region information are: {}.",
-                        keys,
-                        table,
-                        regionLocator.getAllRegionLocations());
-                throw new IOException(
-                        "HBase Table expects at least one region in scan,"
-                                + " please check the HBase table status in HBase cluster");
-            }
-            final byte[] startRow = scan.getStartRow();
-            final byte[] stopRow = scan.getStopRow();
-            final boolean scanWithNoLowerBound = startRow.length == 0;
-            final boolean scanWithNoUpperBound = stopRow.length == 0;
-
-            final List<TableInputSplit> splits = new ArrayList<>(minNumSplits);
-            for (int i = 0; i < keys.getFirst().length; i++) {
-                final byte[] startKey = keys.getFirst()[i];
-                final byte[] endKey = keys.getSecond()[i];
-                final String regionLocation =
-                        regionLocator.getRegionLocation(startKey, false).getHostnamePort();
-                // Test if the given region is to be included in the InputSplit while splitting the
-                // regions of a table
-                if (!includeRegionInScan(startKey, endKey)) {
-                    continue;
-                }
-                // Find the region on which the given row is being served
-                final String[] hosts = new String[] {regionLocation};
-
-                // Determine if regions contains keys used by the scan
-                boolean isLastRegion = endKey.length == 0;
-                if ((scanWithNoLowerBound || isLastRegion || Bytes.compareTo(startRow, endKey) < 0)
-                        && (scanWithNoUpperBound || Bytes.compareTo(stopRow, startKey) > 0)) {
-
-                    final byte[] splitStart =
-                            scanWithNoLowerBound || Bytes.compareTo(startKey, startRow) >= 0
-                                    ? startKey
-                                    : startRow;
-                    final byte[] splitStop =
-                            (scanWithNoUpperBound || Bytes.compareTo(endKey, stopRow) <= 0)
-                                            && !isLastRegion
-                                    ? endKey
-                                    : stopRow;
-                    int id = splits.size();
-                    final TableInputSplit split =
-                            new TableInputSplit(
-                                    id, hosts, table.getName().getName(), splitStart, splitStop);
-                    splits.add(split);
-                }
-            }
-            LOG.info("Created " + splits.size() + " splits");
-            for (TableInputSplit split : splits) {
-                logSplitInfo("created", split);
-            }
-            return splits.toArray(new TableInputSplit[splits.size()]);
-        } finally {
-            closeTable();
-        }
-    }
-
-    /**
-     * Test if the given region is to be included in the scan while splitting the regions of a
-     * table.
-     *
-     * @param startKey Start key of the region
-     * @param endKey End key of the region
-     * @return true, if this region needs to be included as part of the input (default).
-     */
-    protected boolean includeRegionInScan(final byte[] startKey, final byte[] endKey) {
-        return true;
-    }
-
-    @Override
-    public InputSplitAssigner getInputSplitAssigner(TableInputSplit[] inputSplits) {
-        return new LocatableInputSplitAssigner(inputSplits);
-    }
-
-    @Override
-    public BaseStatistics getStatistics(BaseStatistics cachedStatistics) {
-        return null;
-    }
-
-    @VisibleForTesting
-    public Connection getConnection() {
-        return connection;
-    }
-}
diff --git a/flink-connectors/flink-connector-hbase-2.2/src/main/java/org/apache/flink/connector/hbase2/source/HBaseDynamicTableSource.java b/flink-connectors/flink-connector-hbase-2.2/src/main/java/org/apache/flink/connector/hbase2/source/HBaseDynamicTableSource.java
deleted file mode 100644
index 7c88703..0000000
--- a/flink-connectors/flink-connector-hbase-2.2/src/main/java/org/apache/flink/connector/hbase2/source/HBaseDynamicTableSource.java
+++ /dev/null
@@ -1,126 +0,0 @@
-/*
- * 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.hbase2.source;
-
-import org.apache.flink.annotation.Internal;
-import org.apache.flink.api.common.io.InputFormat;
-import org.apache.flink.connector.hbase.source.AbstractHBaseDynamicTableSource;
-import org.apache.flink.connector.hbase.source.HBaseRowDataLookupFunction;
-import org.apache.flink.connector.hbase.util.HBaseTableSchema;
-import org.apache.flink.table.connector.source.DynamicTableSource;
-import org.apache.flink.table.connector.source.lookup.AsyncLookupFunctionProvider;
-import org.apache.flink.table.connector.source.lookup.LookupFunctionProvider;
-import org.apache.flink.table.connector.source.lookup.PartialCachingAsyncLookupProvider;
-import org.apache.flink.table.connector.source.lookup.PartialCachingLookupProvider;
-import org.apache.flink.table.connector.source.lookup.cache.LookupCache;
-import org.apache.flink.table.data.RowData;
-import org.apache.flink.table.types.DataType;
-
-import org.apache.hadoop.conf.Configuration;
-
-import javax.annotation.Nullable;
-
-import java.util.Objects;
-
-import static org.apache.flink.util.Preconditions.checkArgument;
-
-/** HBase table source implementation. */
-@Internal
-public class HBaseDynamicTableSource extends AbstractHBaseDynamicTableSource {
-
-    private final boolean lookupAsync;
-
-    public HBaseDynamicTableSource(
-            Configuration conf,
-            String tableName,
-            HBaseTableSchema hbaseSchema,
-            String nullStringLiteral,
-            int maxRetryTimes,
-            boolean lookupAsync,
-            @Nullable LookupCache cache) {
-        super(conf, tableName, hbaseSchema, nullStringLiteral, maxRetryTimes, cache);
-        this.lookupAsync = lookupAsync;
-    }
-
-    @Override
-    public LookupRuntimeProvider getLookupRuntimeProvider(LookupContext context) {
-        checkArgument(
-                context.getKeys().length == 1 && context.getKeys()[0].length == 1,
-                "Currently, HBase table can only be lookup by single rowkey.");
-        checkArgument(
-                hbaseSchema.getRowKeyName().isPresent(),
-                "HBase schema must have a row key when used in lookup mode.");
-        checkArgument(
-                DataType.getFieldNames(hbaseSchema.convertToDataType())
-                        .get(context.getKeys()[0][0])
-                        .equals(hbaseSchema.getRowKeyName().get()),
-                "Currently, HBase table only supports lookup by rowkey field.");
-        if (lookupAsync) {
-            HBaseRowDataAsyncLookupFunction asyncLookupFunction =
-                    new HBaseRowDataAsyncLookupFunction(
-                            conf, tableName, hbaseSchema, nullStringLiteral, maxRetryTimes);
-            if (cache != null) {
-                return PartialCachingAsyncLookupProvider.of(asyncLookupFunction, cache);
-            } else {
-                return AsyncLookupFunctionProvider.of(asyncLookupFunction);
-            }
-        } else {
-            HBaseRowDataLookupFunction lookupFunction =
-                    new HBaseRowDataLookupFunction(
-                            conf, tableName, hbaseSchema, nullStringLiteral, maxRetryTimes);
-            if (cache != null) {
-                return PartialCachingLookupProvider.of(lookupFunction, cache);
-            } else {
-                return LookupFunctionProvider.of(lookupFunction);
-            }
-        }
-    }
-
-    @Override
-    public DynamicTableSource copy() {
-        return new HBaseDynamicTableSource(
-                conf, tableName, hbaseSchema, nullStringLiteral, maxRetryTimes, lookupAsync, cache);
-    }
-
-    @Override
-    protected InputFormat<RowData, ?> getInputFormat() {
-        return new HBaseRowDataInputFormat(conf, tableName, hbaseSchema, nullStringLiteral);
-    }
-
-    @Override
-    public boolean equals(Object o) {
-        if (!(o instanceof HBaseDynamicTableSource)) {
-            return false;
-        }
-        HBaseDynamicTableSource that = (HBaseDynamicTableSource) o;
-        return Objects.equals(conf, that.conf)
-                && Objects.equals(tableName, that.tableName)
-                && Objects.equals(hbaseSchema, that.hbaseSchema)
-                && Objects.equals(nullStringLiteral, that.nullStringLiteral)
-                && Objects.equals(maxRetryTimes, that.maxRetryTimes)
-                && Objects.equals(cache, that.cache)
-                && Objects.equals(lookupAsync, that.lookupAsync);
-    }
-
-    @Override
-    public int hashCode() {
-        return Objects.hash(
-                conf, tableName, hbaseSchema, nullStringLiteral, maxRetryTimes, cache, lookupAsync);
-    }
-}
diff --git a/flink-connectors/flink-connector-hbase-2.2/src/main/java/org/apache/flink/connector/hbase2/source/HBaseRowDataAsyncLookupFunction.java b/flink-connectors/flink-connector-hbase-2.2/src/main/java/org/apache/flink/connector/hbase2/source/HBaseRowDataAsyncLookupFunction.java
deleted file mode 100644
index 697d5fd..0000000
--- a/flink-connectors/flink-connector-hbase-2.2/src/main/java/org/apache/flink/connector/hbase2/source/HBaseRowDataAsyncLookupFunction.java
+++ /dev/null
@@ -1,223 +0,0 @@
-/*
- * 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.hbase2.source;
-
-import org.apache.flink.annotation.Internal;
-import org.apache.flink.annotation.VisibleForTesting;
-import org.apache.flink.connector.hbase.util.HBaseConfigurationUtil;
-import org.apache.flink.connector.hbase.util.HBaseSerde;
-import org.apache.flink.connector.hbase.util.HBaseTableSchema;
-import org.apache.flink.table.data.GenericRowData;
-import org.apache.flink.table.data.RowData;
-import org.apache.flink.table.functions.AsyncLookupFunction;
-import org.apache.flink.table.functions.FunctionContext;
-import org.apache.flink.util.StringUtils;
-import org.apache.flink.util.concurrent.ExecutorThreadFactory;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.TableNotFoundException;
-import org.apache.hadoop.hbase.client.AsyncConnection;
-import org.apache.hadoop.hbase.client.AsyncTable;
-import org.apache.hadoop.hbase.client.ConnectionFactory;
-import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.ScanResultConsumer;
-import org.apache.hadoop.hbase.util.Threads;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-
-/**
- * The HBaseRowDataAsyncLookupFunction is an implementation to lookup HBase data by rowkey in async
- * fashion. It looks up the result as {@link RowData}.
- */
-@Internal
-public class HBaseRowDataAsyncLookupFunction extends AsyncLookupFunction {
-
-    private static final Logger LOG =
-            LoggerFactory.getLogger(HBaseRowDataAsyncLookupFunction.class);
-    private static final long serialVersionUID = 1L;
-
-    private final String hTableName;
-    private final byte[] serializedConfig;
-    private final HBaseTableSchema hbaseTableSchema;
-    private final String nullStringLiteral;
-
-    private transient AsyncConnection asyncConnection;
-    private transient AsyncTable<ScanResultConsumer> table;
-    private transient HBaseSerde serde;
-
-    private final int maxRetryTimes;
-
-    /** The size for thread pool. */
-    private static final int THREAD_POOL_SIZE = 16;
-
-    public HBaseRowDataAsyncLookupFunction(
-            Configuration configuration,
-            String hTableName,
-            HBaseTableSchema hbaseTableSchema,
-            String nullStringLiteral,
-            int maxRetryTimes) {
-        this.serializedConfig = HBaseConfigurationUtil.serializeConfiguration(configuration);
-        this.hTableName = hTableName;
-        this.hbaseTableSchema = hbaseTableSchema;
-        this.nullStringLiteral = nullStringLiteral;
-        this.maxRetryTimes = maxRetryTimes;
-    }
-
-    @Override
-    public void open(FunctionContext context) {
-        LOG.info("start open ...");
-        final ExecutorService threadPool =
-                Executors.newFixedThreadPool(
-                        THREAD_POOL_SIZE,
-                        new ExecutorThreadFactory(
-                                "hbase-async-lookup-worker", Threads.LOGGING_EXCEPTION_HANDLER));
-        Configuration config = prepareRuntimeConfiguration();
-        CompletableFuture<AsyncConnection> asyncConnectionFuture =
-                ConnectionFactory.createAsyncConnection(config);
-        try {
-            asyncConnection = asyncConnectionFuture.get();
-            table = asyncConnection.getTable(TableName.valueOf(hTableName), threadPool);
-        } catch (InterruptedException | ExecutionException e) {
-            LOG.error("Exception while creating connection to HBase.", e);
-            throw new RuntimeException("Cannot create connection to HBase.", e);
-        }
-        this.serde = new HBaseSerde(hbaseTableSchema, nullStringLiteral);
-        LOG.info("end open.");
-    }
-
-    /**
-     * The invoke entry point of lookup function.
-     *
-     * @param keyRow A {@link RowData} that wraps lookup keys. Currently only support single rowkey.
-     */
-    @Override
-    public CompletableFuture<Collection<RowData>> asyncLookup(RowData keyRow) {
-        int currentRetry = 0;
-        CompletableFuture<Collection<RowData>> future = new CompletableFuture<>();
-        // fetch result
-        fetchResult(future, currentRetry, ((GenericRowData) keyRow).getField(0));
-        return future;
-    }
-
-    /**
-     * Execute async fetch result .
-     *
-     * @param resultFuture The result or exception is returned.
-     * @param currentRetry Current number of retries.
-     * @param rowKey the lookup key.
-     */
-    private void fetchResult(
-            CompletableFuture<Collection<RowData>> resultFuture, int currentRetry, Object rowKey) {
-        Get get = serde.createGet(rowKey);
-        CompletableFuture<Result> responseFuture = table.get(get);
-        responseFuture.whenCompleteAsync(
-                (result, throwable) -> {
-                    if (throwable != null) {
-                        if (throwable instanceof TableNotFoundException) {
-                            LOG.error("Table '{}' not found ", hTableName, throwable);
-                            resultFuture.completeExceptionally(
-                                    new RuntimeException(
-                                            "HBase table '" + hTableName + "' not found.",
-                                            throwable));
-                        } else {
-                            LOG.error(
-                                    String.format(
-                                            "HBase asyncLookup error, retry times = %d",
-                                            currentRetry),
-                                    throwable);
-                            if (currentRetry >= maxRetryTimes) {
-                                resultFuture.completeExceptionally(throwable);
-                            } else {
-                                try {
-                                    Thread.sleep(1000 * currentRetry);
-                                } catch (InterruptedException e1) {
-                                    resultFuture.completeExceptionally(e1);
-                                }
-                                fetchResult(resultFuture, currentRetry + 1, rowKey);
-                            }
-                        }
-                    } else {
-                        if (result.isEmpty()) {
-                            resultFuture.complete(Collections.emptyList());
-                        } else {
-                            resultFuture.complete(
-                                    Collections.singletonList(serde.convertToNewRow(result)));
-                        }
-                    }
-                });
-    }
-
-    private Configuration prepareRuntimeConfiguration() {
-        // create default configuration from current runtime env (`hbase-site.xml` in classpath)
-        // first,
-        // and overwrite configuration using serialized configuration from client-side env
-        // (`hbase-site.xml` in classpath).
-        // user params from client-side have the highest priority
-        Configuration runtimeConfig =
-                HBaseConfigurationUtil.deserializeConfiguration(
-                        serializedConfig, HBaseConfigurationUtil.getHBaseConfiguration());
-
-        // do validation: check key option(s) in final runtime configuration
-        if (StringUtils.isNullOrWhitespaceOnly(runtimeConfig.get(HConstants.ZOOKEEPER_QUORUM))) {
-            LOG.error(
-                    "can not connect to HBase without {} configuration",
-                    HConstants.ZOOKEEPER_QUORUM);
-            throw new IllegalArgumentException(
-                    "check HBase configuration failed, lost: '"
-                            + HConstants.ZOOKEEPER_QUORUM
-                            + "'!");
-        }
-
-        return runtimeConfig;
-    }
-
-    @Override
-    public void close() {
-        LOG.info("start close ...");
-        if (null != table) {
-            table = null;
-        }
-        if (null != asyncConnection) {
-            try {
-                asyncConnection.close();
-                asyncConnection = null;
-            } catch (IOException e) {
-                // ignore exception when close.
-                LOG.warn("exception when close connection", e);
-            }
-        }
-        LOG.info("end close.");
-    }
-
-    @VisibleForTesting
-    public String getHTableName() {
-        return hTableName;
-    }
-}
diff --git a/flink-connectors/flink-connector-hbase-2.2/src/main/java/org/apache/flink/connector/hbase2/source/HBaseRowDataInputFormat.java b/flink-connectors/flink-connector-hbase-2.2/src/main/java/org/apache/flink/connector/hbase2/source/HBaseRowDataInputFormat.java
deleted file mode 100644
index 1bd9a6a..0000000
--- a/flink-connectors/flink-connector-hbase-2.2/src/main/java/org/apache/flink/connector/hbase2/source/HBaseRowDataInputFormat.java
+++ /dev/null
@@ -1,100 +0,0 @@
-/*
- * 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.hbase2.source;
-
-import org.apache.flink.api.common.io.InputFormat;
-import org.apache.flink.connector.hbase.util.HBaseSerde;
-import org.apache.flink.connector.hbase.util.HBaseTableSchema;
-import org.apache.flink.table.data.RowData;
-
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.TableNotFoundException;
-import org.apache.hadoop.hbase.client.ConnectionFactory;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.Scan;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-
-/**
- * {@link InputFormat} subclass that wraps the access for HTables. Returns the result as {@link
- * RowData}
- */
-public class HBaseRowDataInputFormat extends AbstractTableInputFormat<RowData> {
-    private static final long serialVersionUID = 1L;
-    private static final Logger LOG = LoggerFactory.getLogger(HBaseRowDataInputFormat.class);
-
-    private final String tableName;
-    private final HBaseTableSchema schema;
-    private final String nullStringLiteral;
-
-    private transient HBaseSerde serde;
-
-    public HBaseRowDataInputFormat(
-            org.apache.hadoop.conf.Configuration conf,
-            String tableName,
-            HBaseTableSchema schema,
-            String nullStringLiteral) {
-        super(conf);
-        this.tableName = tableName;
-        this.schema = schema;
-        this.nullStringLiteral = nullStringLiteral;
-    }
-
-    @Override
-    protected void initTable() throws IOException {
-        this.serde = new HBaseSerde(schema, nullStringLiteral);
-        if (table == null) {
-            connectToTable();
-        }
-        if (table != null && scan == null) {
-            scan = getScanner();
-        }
-    }
-
-    @Override
-    protected Scan getScanner() {
-        return serde.createScan();
-    }
-
-    @Override
-    public String getTableName() {
-        return tableName;
-    }
-
-    @Override
-    protected RowData mapResultToOutType(Result res) {
-        return serde.convertToReusedRow(res);
-    }
-
-    private void connectToTable() throws IOException {
-        try {
-            if (connection == null) {
-                connection = ConnectionFactory.createConnection(getHadoopConfiguration());
-            }
-            TableName name = TableName.valueOf(getTableName());
-            table = connection.getTable(name);
-            regionLocator = connection.getRegionLocator(name);
-        } catch (TableNotFoundException tnfe) {
-            LOG.error("The table " + tableName + " not found ", tnfe);
-            throw new RuntimeException("HBase table '" + tableName + "' not found.", tnfe);
-        }
-    }
-}
diff --git a/flink-connectors/flink-connector-hbase-2.2/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory b/flink-connectors/flink-connector-hbase-2.2/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory
deleted file mode 100644
index 904d775..0000000
--- a/flink-connectors/flink-connector-hbase-2.2/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory
+++ /dev/null
@@ -1,16 +0,0 @@
-# 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.
-
-org.apache.flink.connector.hbase2.HBase2DynamicTableFactory
diff --git a/flink-connectors/flink-connector-hbase-2.2/src/test/java/org/apache/flink/architecture/TestCodeArchitectureTest.java b/flink-connectors/flink-connector-hbase-2.2/src/test/java/org/apache/flink/architecture/TestCodeArchitectureTest.java
deleted file mode 100644
index 3584360..0000000
--- a/flink-connectors/flink-connector-hbase-2.2/src/test/java/org/apache/flink/architecture/TestCodeArchitectureTest.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/*
- * 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.architecture;
-
-import org.apache.flink.architecture.common.ImportOptions;
-
-import com.tngtech.archunit.core.importer.ImportOption;
-import com.tngtech.archunit.junit.AnalyzeClasses;
-import com.tngtech.archunit.junit.ArchTest;
-import com.tngtech.archunit.junit.ArchTests;
-
-/** Architecture tests for test code. */
-@AnalyzeClasses(
-        packages = "org.apache.flink.connector.hbase2",
-        importOptions = {
-            ImportOption.OnlyIncludeTests.class,
-            ImportOptions.ExcludeScalaImportOption.class,
-            ImportOptions.ExcludeShadedImportOption.class
-        })
-public class TestCodeArchitectureTest {
-
-    @ArchTest
-    public static final ArchTests COMMON_TESTS = ArchTests.in(TestCodeArchitectureTestBase.class);
-}
diff --git a/flink-connectors/flink-connector-hbase-2.2/src/test/java/org/apache/flink/connector/hbase2/HBaseConnectorITCase.java b/flink-connectors/flink-connector-hbase-2.2/src/test/java/org/apache/flink/connector/hbase2/HBaseConnectorITCase.java
deleted file mode 100644
index 0a645aa..0000000
--- a/flink-connectors/flink-connector-hbase-2.2/src/test/java/org/apache/flink/connector/hbase2/HBaseConnectorITCase.java
+++ /dev/null
@@ -1,558 +0,0 @@
-/*
- * 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.hbase2;
-
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.common.typeinfo.Types;
-import org.apache.flink.api.java.typeutils.RowTypeInfo;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.connector.hbase.util.HBaseTableSchema;
-import org.apache.flink.connector.hbase2.source.AbstractTableInputFormat;
-import org.apache.flink.connector.hbase2.source.HBaseRowDataInputFormat;
-import org.apache.flink.connector.hbase2.util.HBaseTestBase;
-import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.table.api.Table;
-import org.apache.flink.table.api.TableEnvironment;
-import org.apache.flink.table.api.TableResult;
-import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
-import org.apache.flink.table.functions.ScalarFunction;
-import org.apache.flink.test.util.MiniClusterWithClientResource;
-import org.apache.flink.test.util.TestBaseUtils;
-import org.apache.flink.types.Row;
-import org.apache.flink.types.RowKind;
-import org.apache.flink.util.CollectionUtil;
-
-import org.apache.flink.shaded.guava31.com.google.common.collect.Lists;
-
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.junit.ClassRule;
-import org.junit.Test;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-import java.util.stream.Collectors;
-
-import static org.apache.flink.table.api.Expressions.$;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-
-/** IT cases for HBase connector (including source and sink). */
-public class HBaseConnectorITCase extends HBaseTestBase {
-
-    @ClassRule
-    public static final MiniClusterWithClientResource MINI_CLUSTER =
-            new MiniClusterWithClientResource(
-                    new MiniClusterResourceConfiguration.Builder()
-                            .setConfiguration(new Configuration())
-                            .build());
-
-    // -------------------------------------------------------------------------------------
-    // HBaseTableSource tests
-    // -------------------------------------------------------------------------------------
-
-    @Test
-    public void testTableSourceFullScan() {
-        TableEnvironment tEnv = TableEnvironment.create(batchSettings);
-
-        tEnv.executeSql(
-                "CREATE TABLE hTable ("
-                        + " family1 ROW<col1 INT>,"
-                        + " family2 ROW<col1 STRING, col2 BIGINT>,"
-                        + " family3 ROW<col1 DOUBLE, col2 BOOLEAN, col3 STRING>,"
-                        + " rowkey INT,"
-                        + " PRIMARY KEY (rowkey) NOT ENFORCED"
-                        + ") WITH ("
-                        + " 'connector' = 'hbase-2.2',"
-                        + " 'table-name' = '"
-                        + TEST_TABLE_1
-                        + "',"
-                        + " 'zookeeper.quorum' = '"
-                        + getZookeeperQuorum()
-                        + "'"
-                        + ")");
-
-        Table table =
-                tEnv.sqlQuery(
-                        "SELECT "
-                                + "  h.family1.col1, "
-                                + "  h.family2.col1, "
-                                + "  h.family2.col2, "
-                                + "  h.family3.col1, "
-                                + "  h.family3.col2, "
-                                + "  h.family3.col3 "
-                                + "FROM hTable AS h");
-
-        List<Row> results = CollectionUtil.iteratorToList(table.execute().collect());
-        String expected =
-                "+I[10, Hello-1, 100, 1.01, false, Welt-1]\n"
-                        + "+I[20, Hello-2, 200, 2.02, true, Welt-2]\n"
-                        + "+I[30, Hello-3, 300, 3.03, false, Welt-3]\n"
-                        + "+I[40, null, 400, 4.04, true, Welt-4]\n"
-                        + "+I[50, Hello-5, 500, 5.05, false, Welt-5]\n"
-                        + "+I[60, Hello-6, 600, 6.06, true, Welt-6]\n"
-                        + "+I[70, Hello-7, 700, 7.07, false, Welt-7]\n"
-                        + "+I[80, null, 800, 8.08, true, Welt-8]\n";
-
-        TestBaseUtils.compareResultAsText(results, expected);
-    }
-
-    @Test
-    public void testTableSourceProjection() {
-        TableEnvironment tEnv = TableEnvironment.create(batchSettings);
-
-        tEnv.executeSql(
-                "CREATE TABLE hTable ("
-                        + " family1 ROW<col1 INT>,"
-                        + " family2 ROW<col1 STRING, col2 BIGINT>,"
-                        + " family3 ROW<col1 DOUBLE, col2 BOOLEAN, col3 STRING>,"
-                        + " rowkey INT,"
-                        + " PRIMARY KEY (rowkey) NOT ENFORCED"
-                        + ") WITH ("
-                        + " 'connector' = 'hbase-2.2',"
-                        + " 'table-name' = '"
-                        + TEST_TABLE_1
-                        + "',"
-                        + " 'zookeeper.quorum' = '"
-                        + getZookeeperQuorum()
-                        + "'"
-                        + ")");
-
-        Table table =
-                tEnv.sqlQuery(
-                        "SELECT "
-                                + "  h.family1.col1, "
-                                + "  h.family3.col1, "
-                                + "  h.family3.col2, "
-                                + "  h.family3.col3 "
-                                + "FROM hTable AS h");
-
-        List<Row> results = CollectionUtil.iteratorToList(table.execute().collect());
-        String expected =
-                "+I[10, 1.01, false, Welt-1]\n"
-                        + "+I[20, 2.02, true, Welt-2]\n"
-                        + "+I[30, 3.03, false, Welt-3]\n"
-                        + "+I[40, 4.04, true, Welt-4]\n"
-                        + "+I[50, 5.05, false, Welt-5]\n"
-                        + "+I[60, 6.06, true, Welt-6]\n"
-                        + "+I[70, 7.07, false, Welt-7]\n"
-                        + "+I[80, 8.08, true, Welt-8]\n";
-
-        TestBaseUtils.compareResultAsText(results, expected);
-    }
-
-    @Test
-    public void testTableSourceFieldOrder() {
-        TableEnvironment tEnv = TableEnvironment.create(batchSettings);
-
-        tEnv.executeSql(
-                "CREATE TABLE hTable ("
-                        + " rowkey INT PRIMARY KEY NOT ENFORCED,"
-                        + " family2 ROW<col1 STRING, col2 BIGINT>,"
-                        + " family3 ROW<col1 DOUBLE, col2 BOOLEAN, col3 STRING>,"
-                        + " family1 ROW<col1 INT>"
-                        + ") WITH ("
-                        + " 'connector' = 'hbase-2.2',"
-                        + " 'table-name' = '"
-                        + TEST_TABLE_1
-                        + "',"
-                        + " 'zookeeper.quorum' = '"
-                        + getZookeeperQuorum()
-                        + "'"
-                        + ")");
-
-        Table table = tEnv.sqlQuery("SELECT * FROM hTable AS h");
-
-        List<Row> results = CollectionUtil.iteratorToList(table.execute().collect());
-        String expected =
-                "+I[1, +I[Hello-1, 100], +I[1.01, false, Welt-1], +I[10]]\n"
-                        + "+I[2, +I[Hello-2, 200], +I[2.02, true, Welt-2], +I[20]]\n"
-                        + "+I[3, +I[Hello-3, 300], +I[3.03, false, Welt-3], +I[30]]\n"
-                        + "+I[4, +I[null, 400], +I[4.04, true, Welt-4], +I[40]]\n"
-                        + "+I[5, +I[Hello-5, 500], +I[5.05, false, Welt-5], +I[50]]\n"
-                        + "+I[6, +I[Hello-6, 600], +I[6.06, true, Welt-6], +I[60]]\n"
-                        + "+I[7, +I[Hello-7, 700], +I[7.07, false, Welt-7], +I[70]]\n"
-                        + "+I[8, +I[null, 800], +I[8.08, true, Welt-8], +I[80]]\n";
-
-        TestBaseUtils.compareResultAsText(results, expected);
-    }
-
-    @Test
-    public void testTableSourceReadAsByteArray() {
-        TableEnvironment tEnv = TableEnvironment.create(batchSettings);
-
-        tEnv.executeSql(
-                "CREATE TABLE hTable ("
-                        + " family2 ROW<col1 BYTES, col2 BYTES>,"
-                        + " rowkey INT"
-                        + // no primary key syntax
-                        ") WITH ("
-                        + " 'connector' = 'hbase-2.2',"
-                        + " 'table-name' = '"
-                        + TEST_TABLE_1
-                        + "',"
-                        + " 'zookeeper.quorum' = '"
-                        + getZookeeperQuorum()
-                        + "'"
-                        + ")");
-        tEnv.registerFunction("toUTF8", new ToUTF8());
-        tEnv.registerFunction("toLong", new ToLong());
-
-        Table table =
-                tEnv.sqlQuery(
-                        "SELECT "
-                                + "  toUTF8(h.family2.col1), "
-                                + "  toLong(h.family2.col2) "
-                                + "FROM hTable AS h");
-
-        List<Row> results = CollectionUtil.iteratorToList(table.execute().collect());
-        String expected =
-                "+I[Hello-1, 100]\n"
-                        + "+I[Hello-2, 200]\n"
-                        + "+I[Hello-3, 300]\n"
-                        + "+I[null, 400]\n"
-                        + "+I[Hello-5, 500]\n"
-                        + "+I[Hello-6, 600]\n"
-                        + "+I[Hello-7, 700]\n"
-                        + "+I[null, 800]\n";
-
-        TestBaseUtils.compareResultAsText(results, expected);
-    }
-
-    @Test
-    public void testTableSink() throws Exception {
-        StreamExecutionEnvironment execEnv = StreamExecutionEnvironment.getExecutionEnvironment();
-        StreamTableEnvironment tEnv = StreamTableEnvironment.create(execEnv, streamSettings);
-
-        // register HBase table testTable1 which contains test data
-        String table1DDL = createHBaseTableDDL(TEST_TABLE_1, false);
-        tEnv.executeSql(table1DDL);
-
-        String table2DDL = createHBaseTableDDL(TEST_TABLE_2, false);
-        tEnv.executeSql(table2DDL);
-
-        String query =
-                "INSERT INTO "
-                        + TEST_TABLE_2
-                        + " SELECT"
-                        + " rowkey,"
-                        + " family1,"
-                        + " family2,"
-                        + " family3"
-                        + " FROM "
-                        + TEST_TABLE_1;
-
-        TableResult tableResult = tEnv.executeSql(query);
-
-        // wait to finish
-        tableResult.await();
-
-        assertEquals(
-                "Expected INSERT rowKind", RowKind.INSERT, tableResult.collect().next().getKind());
-
-        // start a batch scan job to verify contents in HBase table
-        TableEnvironment batchEnv = TableEnvironment.create(batchSettings);
-        batchEnv.executeSql(table2DDL);
-
-        List<String> expected = new ArrayList<>();
-        expected.add("+I[1, 10, Hello-1, 100, 1.01, false, Welt-1]\n");
-        expected.add("+I[2, 20, Hello-2, 200, 2.02, true, Welt-2]\n");
-        expected.add("+I[3, 30, Hello-3, 300, 3.03, false, Welt-3]\n");
-        expected.add("+I[4, 40, null, 400, 4.04, true, Welt-4]\n");
-        expected.add("+I[5, 50, Hello-5, 500, 5.05, false, Welt-5]\n");
-        expected.add("+I[6, 60, Hello-6, 600, 6.06, true, Welt-6]\n");
-        expected.add("+I[7, 70, Hello-7, 700, 7.07, false, Welt-7]\n");
-        expected.add("+I[8, 80, null, 800, 8.08, true, Welt-8]\n");
-
-        Table countTable =
-                batchEnv.sqlQuery("SELECT COUNT(h.rowkey) FROM " + TEST_TABLE_2 + " AS h");
-
-        assertEquals(new Long(expected.size()), countTable.execute().collect().next().getField(0));
-
-        Table table =
-                batchEnv.sqlQuery(
-                        "SELECT "
-                                + "  h.rowkey, "
-                                + "  h.family1.col1, "
-                                + "  h.family2.col1, "
-                                + "  h.family2.col2, "
-                                + "  h.family3.col1, "
-                                + "  h.family3.col2, "
-                                + "  h.family3.col3 "
-                                + "FROM "
-                                + TEST_TABLE_2
-                                + " AS h");
-
-        TableResult tableResult2 = table.execute();
-
-        List<Row> results = CollectionUtil.iteratorToList(tableResult2.collect());
-
-        TestBaseUtils.compareResultAsText(results, String.join("", expected));
-    }
-
-    @Test
-    public void testTableSourceSinkWithDDL() throws Exception {
-        StreamExecutionEnvironment execEnv = StreamExecutionEnvironment.getExecutionEnvironment();
-        StreamTableEnvironment tEnv = StreamTableEnvironment.create(execEnv, streamSettings);
-
-        // register HBase table testTable1 which contains test data
-        String table1DDL = createHBaseTableDDL(TEST_TABLE_1, true);
-        tEnv.executeSql(table1DDL);
-
-        // register HBase table which is empty
-        String table3DDL = createHBaseTableDDL(TEST_TABLE_3, true);
-        tEnv.executeSql(table3DDL);
-
-        String insertStatement =
-                "INSERT INTO "
-                        + TEST_TABLE_3
-                        + " SELECT rowkey,"
-                        + " family1,"
-                        + " family2,"
-                        + " family3,"
-                        + " family4"
-                        + " from "
-                        + TEST_TABLE_1;
-
-        TableResult tableResult = tEnv.executeSql(insertStatement);
-
-        // wait to finish
-        tableResult.await();
-
-        assertEquals(
-                "Expected INSERT rowKind", RowKind.INSERT, tableResult.collect().next().getKind());
-
-        // start a batch scan job to verify contents in HBase table
-        TableEnvironment batchEnv = TableEnvironment.create(batchSettings);
-        batchEnv.executeSql(table3DDL);
-
-        List<String> expected = new ArrayList<>();
-        expected.add(
-                "+I[1, 10, Hello-1, 100, 1.01, false, Welt-1, 2019-08-18T19:00, 2019-08-18, 19:00, 12345678.0001]");
-        expected.add(
-                "+I[2, 20, Hello-2, 200, 2.02, true, Welt-2, 2019-08-18T19:01, 2019-08-18, 19:01, 12345678.0002]");
-        expected.add(
-                "+I[3, 30, Hello-3, 300, 3.03, false, Welt-3, 2019-08-18T19:02, 2019-08-18, 19:02, 12345678.0003]");
-        expected.add(
-                "+I[4, 40, null, 400, 4.04, true, Welt-4, 2019-08-18T19:03, 2019-08-18, 19:03, 12345678.0004]");
-        expected.add(
-                "+I[5, 50, Hello-5, 500, 5.05, false, Welt-5, 2019-08-19T19:10, 2019-08-19, 19:10, 12345678.0005]");
-        expected.add(
-                "+I[6, 60, Hello-6, 600, 6.06, true, Welt-6, 2019-08-19T19:20, 2019-08-19, 19:20, 12345678.0006]");
-        expected.add(
-                "+I[7, 70, Hello-7, 700, 7.07, false, Welt-7, 2019-08-19T19:30, 2019-08-19, 19:30, 12345678.0007]");
-        expected.add(
-                "+I[8, 80, null, 800, 8.08, true, Welt-8, 2019-08-19T19:40, 2019-08-19, 19:40, 12345678.0008]");
-
-        String query =
-                "SELECT "
-                        + "  h.rowkey, "
-                        + "  h.family1.col1, "
-                        + "  h.family2.col1, "
-                        + "  h.family2.col2, "
-                        + "  h.family3.col1, "
-                        + "  h.family3.col2, "
-                        + "  h.family3.col3, "
-                        + "  h.family4.col1, "
-                        + "  h.family4.col2, "
-                        + "  h.family4.col3, "
-                        + "  h.family4.col4 "
-                        + " FROM "
-                        + TEST_TABLE_3
-                        + " AS h";
-
-        TableResult tableResult3 = batchEnv.executeSql(query);
-
-        List<String> result =
-                Lists.newArrayList(tableResult3.collect()).stream()
-                        .map(Row::toString)
-                        .sorted()
-                        .collect(Collectors.toList());
-
-        assertEquals(expected, result);
-    }
-
-    @Test
-    public void testHBaseLookupTableSource() {
-        verifyHBaseLookupJoin(false);
-    }
-
-    @Test
-    public void testHBaseAsyncLookupTableSource() {
-        verifyHBaseLookupJoin(true);
-    }
-
-    @Test
-    public void testTableInputFormatOpenClose() throws IOException {
-        HBaseTableSchema tableSchema = new HBaseTableSchema();
-        tableSchema.addColumn(FAMILY1, F1COL1, byte[].class);
-        AbstractTableInputFormat<?> inputFormat =
-                new HBaseRowDataInputFormat(getConf(), TEST_TABLE_1, tableSchema, "null");
-        inputFormat.open(inputFormat.createInputSplits(1)[0]);
-        assertNotNull(inputFormat.getConnection());
-        assertNotNull(inputFormat.getConnection().getTable(TableName.valueOf(TEST_TABLE_1)));
-
-        inputFormat.close();
-        assertNull(inputFormat.getConnection());
-    }
-
-    private void verifyHBaseLookupJoin(boolean async) {
-        StreamExecutionEnvironment execEnv = StreamExecutionEnvironment.getExecutionEnvironment();
-        StreamTableEnvironment tEnv = StreamTableEnvironment.create(execEnv, streamSettings);
-
-        tEnv.executeSql(
-                "CREATE TABLE "
-                        + TEST_TABLE_1
-                        + " ("
-                        + " family1 ROW<col1 INT>,"
-                        + " family2 ROW<col1 STRING, col2 BIGINT>,"
-                        + " family3 ROW<col1 DOUBLE, col2 BOOLEAN, col3 STRING>,"
-                        + " rowkey INT,"
-                        + " family4 ROW<col1 TIMESTAMP(3), col2 DATE, col3 TIME(3), col4 DECIMAL(12, 4)>,"
-                        + " PRIMARY KEY (rowkey) NOT ENFORCED"
-                        + ") WITH ("
-                        + " 'connector' = 'hbase-2.2',"
-                        + " 'lookup.async' = '"
-                        + async
-                        + "',"
-                        + " 'table-name' = '"
-                        + TEST_TABLE_1
-                        + "',"
-                        + " 'zookeeper.quorum' = '"
-                        + getZookeeperQuorum()
-                        + "'"
-                        + ")");
-
-        // prepare a source table
-        String srcTableName = "src";
-        DataStream<Row> srcDs = execEnv.fromCollection(testData).returns(testTypeInfo);
-        Table in = tEnv.fromDataStream(srcDs, $("a"), $("b"), $("c"), $("proc").proctime());
-        tEnv.createTemporaryView(srcTableName, in);
-
-        // perform a temporal table join query
-        String dimJoinQuery =
-                "SELECT"
-                        + " a,"
-                        + " b,"
-                        + " h.family1.col1,"
-                        + " h.family2.col1,"
-                        + " h.family2.col2,"
-                        + " h.family3.col1,"
-                        + " h.family3.col2,"
-                        + " h.family3.col3,"
-                        + " h.family4.col1,"
-                        + " h.family4.col2,"
-                        + " h.family4.col3,"
-                        + " h.family4.col4 "
-                        + " FROM src JOIN "
-                        + TEST_TABLE_1
-                        + " FOR SYSTEM_TIME AS OF src.proc as h ON src.a = h.rowkey";
-        Iterator<Row> collected = tEnv.executeSql(dimJoinQuery).collect();
-        List<String> result =
-                Lists.newArrayList(collected).stream()
-                        .map(Row::toString)
-                        .sorted()
-                        .collect(Collectors.toList());
-
-        List<String> expected = new ArrayList<>();
-        expected.add(
-                "+I[1, 1, 10, Hello-1, 100, 1.01, false, Welt-1, 2019-08-18T19:00, 2019-08-18, 19:00, 12345678.0001]");
-        expected.add(
-                "+I[2, 2, 20, Hello-2, 200, 2.02, true, Welt-2, 2019-08-18T19:01, 2019-08-18, 19:01, 12345678.0002]");
-        expected.add(
-                "+I[3, 2, 30, Hello-3, 300, 3.03, false, Welt-3, 2019-08-18T19:02, 2019-08-18, 19:02, 12345678.0003]");
-        expected.add(
-                "+I[3, 3, 30, Hello-3, 300, 3.03, false, Welt-3, 2019-08-18T19:02, 2019-08-18, 19:02, 12345678.0003]");
-
-        assertEquals(expected, result);
-    }
-
-    // -------------------------------------------------------------------------------------
-    // HBase lookup source tests
-    // -------------------------------------------------------------------------------------
-
-    // prepare a source collection.
-    private static final List<Row> testData = new ArrayList<>();
-    private static final RowTypeInfo testTypeInfo =
-            new RowTypeInfo(
-                    new TypeInformation[] {Types.INT, Types.LONG, Types.STRING},
-                    new String[] {"a", "b", "c"});
-
-    static {
-        testData.add(Row.of(1, 1L, "Hi"));
-        testData.add(Row.of(2, 2L, "Hello"));
-        testData.add(Row.of(3, 2L, "Hello world"));
-        testData.add(Row.of(3, 3L, "Hello world!"));
-    }
-
-    // ------------------------------- Utilities -------------------------------------------------
-
-    /** A {@link ScalarFunction} that maps byte arrays to UTF-8 strings. */
-    public static class ToUTF8 extends ScalarFunction {
-        private static final long serialVersionUID = 1L;
-
-        public String eval(byte[] bytes) {
-            return Bytes.toString(bytes);
-        }
-    }
-
-    /** A {@link ScalarFunction} that maps byte array to longs. */
-    public static class ToLong extends ScalarFunction {
-        private static final long serialVersionUID = 1L;
-
-        public long eval(byte[] bytes) {
-            return Bytes.toLong(bytes);
-        }
-    }
-
-    private String createHBaseTableDDL(String tableName, boolean testTimeAndDecimalTypes) {
-        StringBuilder family4Statement = new StringBuilder();
-        if (testTimeAndDecimalTypes) {
-            family4Statement.append(", family4 ROW<col1 TIMESTAMP(3)");
-            family4Statement.append(", col2 DATE");
-            family4Statement.append(", col3 TIME(3)");
-            family4Statement.append(", col4 DECIMAL(12, 4)");
-            family4Statement.append("> \n");
-        }
-
-        return "CREATE TABLE "
-                + tableName
-                + "(\n"
-                + "   rowkey INT,"
-                + "   family1 ROW<col1 INT>,\n"
-                + "   family2 ROW<col1 VARCHAR, col2 BIGINT>,\n"
-                + "   family3 ROW<col1 DOUBLE, col2 BOOLEAN, col3 VARCHAR>"
-                + family4Statement.toString()
-                + ") WITH (\n"
-                + "   'connector' = 'hbase-2.2',\n"
-                + "   'table-name' = '"
-                + tableName
-                + "',\n"
-                + "   'zookeeper.quorum' = '"
-                + getZookeeperQuorum()
-                + "',\n"
-                + "   'zookeeper.znode.parent' = '/hbase' "
-                + ")";
-    }
-}
diff --git a/flink-connectors/flink-connector-hbase-2.2/src/test/java/org/apache/flink/connector/hbase2/HBaseDynamicTableFactoryTest.java b/flink-connectors/flink-connector-hbase-2.2/src/test/java/org/apache/flink/connector/hbase2/HBaseDynamicTableFactoryTest.java
deleted file mode 100644
index a535f0e..0000000
--- a/flink-connectors/flink-connector-hbase-2.2/src/test/java/org/apache/flink/connector/hbase2/HBaseDynamicTableFactoryTest.java
+++ /dev/null
@@ -1,426 +0,0 @@
-/*
- * 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.hbase2;
-
-import org.apache.flink.api.common.typeinfo.Types;
-import org.apache.flink.connector.hbase.options.HBaseWriteOptions;
-import org.apache.flink.connector.hbase.source.HBaseRowDataLookupFunction;
-import org.apache.flink.connector.hbase.util.HBaseConfigurationUtil;
-import org.apache.flink.connector.hbase.util.HBaseTableSchema;
-import org.apache.flink.connector.hbase2.sink.HBaseDynamicTableSink;
-import org.apache.flink.connector.hbase2.source.HBaseDynamicTableSource;
-import org.apache.flink.connector.hbase2.source.HBaseRowDataAsyncLookupFunction;
-import org.apache.flink.table.catalog.Column;
-import org.apache.flink.table.catalog.ResolvedSchema;
-import org.apache.flink.table.connector.sink.DynamicTableSink;
-import org.apache.flink.table.connector.sink.SinkFunctionProvider;
-import org.apache.flink.table.connector.source.DynamicTableSource;
-import org.apache.flink.table.connector.source.LookupTableSource;
-import org.apache.flink.table.connector.source.lookup.AsyncLookupFunctionProvider;
-import org.apache.flink.table.connector.source.lookup.LookupFunctionProvider;
-import org.apache.flink.table.connector.source.lookup.cache.DefaultLookupCache;
-import org.apache.flink.table.functions.AsyncLookupFunction;
-import org.apache.flink.table.functions.LookupFunction;
-import org.apache.flink.table.runtime.connector.sink.SinkRuntimeProviderContext;
-import org.apache.flink.table.runtime.connector.source.LookupRuntimeProviderContext;
-import org.apache.flink.table.types.DataType;
-import org.apache.flink.util.ExceptionUtils;
-
-import org.apache.commons.collections.IteratorUtils;
-import org.apache.hadoop.hbase.HConstants;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
-
-import java.time.Duration;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Optional;
-
-import static org.apache.flink.table.api.DataTypes.BIGINT;
-import static org.apache.flink.table.api.DataTypes.BOOLEAN;
-import static org.apache.flink.table.api.DataTypes.DATE;
-import static org.apache.flink.table.api.DataTypes.DECIMAL;
-import static org.apache.flink.table.api.DataTypes.DOUBLE;
-import static org.apache.flink.table.api.DataTypes.FIELD;
-import static org.apache.flink.table.api.DataTypes.INT;
-import static org.apache.flink.table.api.DataTypes.ROW;
-import static org.apache.flink.table.api.DataTypes.STRING;
-import static org.apache.flink.table.api.DataTypes.TIME;
-import static org.apache.flink.table.api.DataTypes.TIMESTAMP;
-import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSink;
-import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSource;
-import static org.assertj.core.api.Assertions.assertThat;
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-/** Unit test for {@link HBase2DynamicTableFactory}. */
-public class HBaseDynamicTableFactoryTest {
-
-    private static final String FAMILY1 = "f1";
-    private static final String FAMILY2 = "f2";
-    private static final String FAMILY3 = "f3";
-    private static final String FAMILY4 = "f4";
-    private static final String COL1 = "c1";
-    private static final String COL2 = "c2";
-    private static final String COL3 = "c3";
-    private static final String COL4 = "c4";
-    private static final String ROWKEY = "rowkey";
-
-    @Rule public final ExpectedException thrown = ExpectedException.none();
-
-    @SuppressWarnings("rawtypes")
-    @Test
-    public void testTableSourceFactory() {
-        ResolvedSchema schema =
-                ResolvedSchema.of(
-                        Column.physical(FAMILY1, ROW(FIELD(COL1, INT()))),
-                        Column.physical(FAMILY2, ROW(FIELD(COL1, INT()), FIELD(COL2, BIGINT()))),
-                        Column.physical(ROWKEY, BIGINT()),
-                        Column.physical(
-                                FAMILY3,
-                                ROW(
-                                        FIELD(COL1, DOUBLE()),
-                                        FIELD(COL2, BOOLEAN()),
-                                        FIELD(COL3, STRING()))),
-                        Column.physical(
-                                FAMILY4,
-                                ROW(
-                                        FIELD(COL1, DECIMAL(10, 3)),
-                                        FIELD(COL2, TIMESTAMP(3)),
-                                        FIELD(COL3, DATE()),
-                                        FIELD(COL4, TIME()))));
-
-        DynamicTableSource source = createTableSource(schema, getAllOptions());
-        assertTrue(source instanceof HBaseDynamicTableSource);
-        HBaseDynamicTableSource hbaseSource = (HBaseDynamicTableSource) source;
-
-        int[][] lookupKey = {{2}};
-        LookupTableSource.LookupRuntimeProvider lookupProvider =
-                hbaseSource.getLookupRuntimeProvider(new LookupRuntimeProviderContext(lookupKey));
-        assertTrue(lookupProvider instanceof LookupFunctionProvider);
-
-        LookupFunction tableFunction =
-                ((LookupFunctionProvider) lookupProvider).createLookupFunction();
-        assertTrue(tableFunction instanceof HBaseRowDataLookupFunction);
-        assertEquals(
-                "testHBastTable", ((HBaseRowDataLookupFunction) tableFunction).getHTableName());
-
-        HBaseTableSchema hbaseSchema = hbaseSource.getHBaseTableSchema();
-        assertEquals(2, hbaseSchema.getRowKeyIndex());
-        assertEquals(Optional.of(Types.LONG), hbaseSchema.getRowKeyTypeInfo());
-
-        assertArrayEquals(new String[] {"f1", "f2", "f3", "f4"}, hbaseSchema.getFamilyNames());
-        assertArrayEquals(new String[] {"c1"}, hbaseSchema.getQualifierNames("f1"));
-        assertArrayEquals(new String[] {"c1", "c2"}, hbaseSchema.getQualifierNames("f2"));
-        assertArrayEquals(new String[] {"c1", "c2", "c3"}, hbaseSchema.getQualifierNames("f3"));
-        assertArrayEquals(
-                new String[] {"c1", "c2", "c3", "c4"}, hbaseSchema.getQualifierNames("f4"));
-
-        assertArrayEquals(new DataType[] {INT()}, hbaseSchema.getQualifierDataTypes("f1"));
-        assertArrayEquals(
-                new DataType[] {INT(), BIGINT()}, hbaseSchema.getQualifierDataTypes("f2"));
-        assertArrayEquals(
-                new DataType[] {DOUBLE(), BOOLEAN(), STRING()},
-                hbaseSchema.getQualifierDataTypes("f3"));
-        assertArrayEquals(
-                new DataType[] {DECIMAL(10, 3), TIMESTAMP(3), DATE(), TIME()},
-                hbaseSchema.getQualifierDataTypes("f4"));
-    }
-
-    @Test
-    public void testLookupOptions() {
-        ResolvedSchema schema = ResolvedSchema.of(Column.physical(ROWKEY, STRING()));
-        Map<String, String> options = getAllOptions();
-        options.put("lookup.cache", "PARTIAL");
-        options.put("lookup.partial-cache.expire-after-access", "15213s");
-        options.put("lookup.partial-cache.expire-after-write", "18213s");
-        options.put("lookup.partial-cache.max-rows", "10000");
-        options.put("lookup.partial-cache.cache-missing-key", "false");
-        options.put("lookup.max-retries", "15513");
-
-        DynamicTableSource source = createTableSource(schema, options);
-        HBaseDynamicTableSource hbaseSource = (HBaseDynamicTableSource) source;
-        assertThat(((HBaseDynamicTableSource) source).getMaxRetryTimes()).isEqualTo(15513);
-        assertThat(hbaseSource.getCache()).isInstanceOf(DefaultLookupCache.class);
-        DefaultLookupCache cache = (DefaultLookupCache) hbaseSource.getCache();
-        assertThat(cache)
-                .isEqualTo(
-                        DefaultLookupCache.newBuilder()
-                                .expireAfterAccess(Duration.ofSeconds(15213))
-                                .expireAfterWrite(Duration.ofSeconds(18213))
-                                .maximumSize(10000)
-                                .cacheMissingKey(false)
-                                .build());
-    }
-
-    @Test
-    public void testTableSinkFactory() {
-        ResolvedSchema schema =
-                ResolvedSchema.of(
-                        Column.physical(ROWKEY, STRING()),
-                        Column.physical(FAMILY1, ROW(FIELD(COL1, DOUBLE()), FIELD(COL2, INT()))),
-                        Column.physical(FAMILY2, ROW(FIELD(COL1, INT()), FIELD(COL3, BIGINT()))),
-                        Column.physical(
-                                FAMILY3, ROW(FIELD(COL2, BOOLEAN()), FIELD(COL3, STRING()))),
-                        Column.physical(
-                                FAMILY4,
-                                ROW(
-                                        FIELD(COL1, DECIMAL(10, 3)),
-                                        FIELD(COL2, TIMESTAMP(3)),
-                                        FIELD(COL3, DATE()),
-                                        FIELD(COL4, TIME()))));
-
-        DynamicTableSink sink = createTableSink(schema, getAllOptions());
-        assertTrue(sink instanceof HBaseDynamicTableSink);
-        HBaseDynamicTableSink hbaseSink = (HBaseDynamicTableSink) sink;
-
-        HBaseTableSchema hbaseSchema = hbaseSink.getHBaseTableSchema();
-        assertEquals(0, hbaseSchema.getRowKeyIndex());
-        assertEquals(Optional.of(STRING()), hbaseSchema.getRowKeyDataType());
-
-        assertArrayEquals(new String[] {"f1", "f2", "f3", "f4"}, hbaseSchema.getFamilyNames());
-        assertArrayEquals(new String[] {"c1", "c2"}, hbaseSchema.getQualifierNames("f1"));
-        assertArrayEquals(new String[] {"c1", "c3"}, hbaseSchema.getQualifierNames("f2"));
-        assertArrayEquals(new String[] {"c2", "c3"}, hbaseSchema.getQualifierNames("f3"));
-        assertArrayEquals(
-                new String[] {"c1", "c2", "c3", "c4"}, hbaseSchema.getQualifierNames("f4"));
-
-        assertArrayEquals(
-                new DataType[] {DOUBLE(), INT()}, hbaseSchema.getQualifierDataTypes("f1"));
-        assertArrayEquals(
-                new DataType[] {INT(), BIGINT()}, hbaseSchema.getQualifierDataTypes("f2"));
-        assertArrayEquals(
-                new DataType[] {BOOLEAN(), STRING()}, hbaseSchema.getQualifierDataTypes("f3"));
-        assertArrayEquals(
-                new DataType[] {DECIMAL(10, 3), TIMESTAMP(3), DATE(), TIME()},
-                hbaseSchema.getQualifierDataTypes("f4"));
-
-        // verify hadoop Configuration
-        org.apache.hadoop.conf.Configuration expectedConfiguration =
-                HBaseConfigurationUtil.getHBaseConfiguration();
-        expectedConfiguration.set(HConstants.ZOOKEEPER_QUORUM, "localhost:2181");
-        expectedConfiguration.set(HConstants.ZOOKEEPER_ZNODE_PARENT, "/flink");
-        expectedConfiguration.set("hbase.security.authentication", "kerberos");
-
-        org.apache.hadoop.conf.Configuration actualConfiguration = hbaseSink.getConfiguration();
-
-        assertEquals(
-                IteratorUtils.toList(expectedConfiguration.iterator()),
-                IteratorUtils.toList(actualConfiguration.iterator()));
-
-        // verify tableName
-        assertEquals("testHBastTable", hbaseSink.getTableName());
-
-        HBaseWriteOptions expectedWriteOptions =
-                HBaseWriteOptions.builder()
-                        .setBufferFlushMaxRows(1000)
-                        .setBufferFlushIntervalMillis(1000)
-                        .setBufferFlushMaxSizeInBytes(2 * 1024 * 1024)
-                        .build();
-        HBaseWriteOptions actualWriteOptions = hbaseSink.getWriteOptions();
-        assertEquals(expectedWriteOptions, actualWriteOptions);
-    }
-
-    @Test
-    public void testBufferFlushOptions() {
-        Map<String, String> options = getAllOptions();
-        options.put("sink.buffer-flush.max-size", "10mb");
-        options.put("sink.buffer-flush.max-rows", "100");
-        options.put("sink.buffer-flush.interval", "10s");
-
-        ResolvedSchema schema = ResolvedSchema.of(Column.physical(ROWKEY, STRING()));
-
-        DynamicTableSink sink = createTableSink(schema, options);
-        HBaseWriteOptions expected =
-                HBaseWriteOptions.builder()
-                        .setBufferFlushMaxRows(100)
-                        .setBufferFlushIntervalMillis(10 * 1000)
-                        .setBufferFlushMaxSizeInBytes(10 * 1024 * 1024)
-                        .build();
-        HBaseWriteOptions actual = ((HBaseDynamicTableSink) sink).getWriteOptions();
-        assertEquals(expected, actual);
-    }
-
-    @Test
-    public void testParallelismOptions() {
-        Map<String, String> options = getAllOptions();
-        options.put("sink.parallelism", "2");
-
-        ResolvedSchema schema = ResolvedSchema.of(Column.physical(ROWKEY, STRING()));
-
-        DynamicTableSink sink = createTableSink(schema, options);
-        assertTrue(sink instanceof HBaseDynamicTableSink);
-        HBaseDynamicTableSink hbaseSink = (HBaseDynamicTableSink) sink;
-        SinkFunctionProvider provider =
-                (SinkFunctionProvider)
-                        hbaseSink.getSinkRuntimeProvider(new SinkRuntimeProviderContext(false));
-        assertEquals(2, (long) provider.getParallelism().get());
-    }
-
-    @Test
-    public void testLookupAsync() {
-        Map<String, String> options = getAllOptions();
-        options.put("lookup.async", "true");
-        ResolvedSchema schema =
-                ResolvedSchema.of(
-                        Column.physical(ROWKEY, STRING()),
-                        Column.physical(FAMILY1, ROW(FIELD(COL1, DOUBLE()), FIELD(COL2, INT()))));
-        DynamicTableSource source = createTableSource(schema, options);
-        assertTrue(source instanceof HBaseDynamicTableSource);
-        HBaseDynamicTableSource hbaseSource = (HBaseDynamicTableSource) source;
-
-        int[][] lookupKey = {{0}};
-        LookupTableSource.LookupRuntimeProvider lookupProvider =
-                hbaseSource.getLookupRuntimeProvider(new LookupRuntimeProviderContext(lookupKey));
-        assertTrue(lookupProvider instanceof AsyncLookupFunctionProvider);
-
-        AsyncLookupFunction asyncTableFunction =
-                ((AsyncLookupFunctionProvider) lookupProvider).createAsyncLookupFunction();
-        assertTrue(asyncTableFunction instanceof HBaseRowDataAsyncLookupFunction);
-        assertEquals(
-                "testHBastTable",
-                ((HBaseRowDataAsyncLookupFunction) asyncTableFunction).getHTableName());
-    }
-
-    @Test
-    public void testDisabledBufferFlushOptions() {
-        Map<String, String> options = getAllOptions();
-        options.put("sink.buffer-flush.max-size", "0");
-        options.put("sink.buffer-flush.max-rows", "0");
-        options.put("sink.buffer-flush.interval", "0");
-
-        ResolvedSchema schema = ResolvedSchema.of(Column.physical(ROWKEY, STRING()));
-
-        DynamicTableSink sink = createTableSink(schema, options);
-        HBaseWriteOptions expected =
-                HBaseWriteOptions.builder()
-                        .setBufferFlushMaxRows(0)
-                        .setBufferFlushIntervalMillis(0)
-                        .setBufferFlushMaxSizeInBytes(0)
-                        .build();
-        HBaseWriteOptions actual = ((HBaseDynamicTableSink) sink).getWriteOptions();
-        assertEquals(expected, actual);
-    }
-
-    @Test
-    public void testUnknownOption() {
-        Map<String, String> options = getAllOptions();
-        options.put("sink.unknown.key", "unknown-value");
-        ResolvedSchema schema =
-                ResolvedSchema.of(
-                        Column.physical(ROWKEY, STRING()),
-                        Column.physical(FAMILY1, ROW(FIELD(COL1, DOUBLE()), FIELD(COL2, INT()))));
-
-        try {
-            createTableSource(schema, options);
-            fail("Should fail");
-        } catch (Exception e) {
-            assertTrue(
-                    ExceptionUtils.findThrowableWithMessage(
-                                    e, "Unsupported options:\n\nsink.unknown.key")
-                            .isPresent());
-        }
-
-        try {
-            createTableSink(schema, options);
-            fail("Should fail");
-        } catch (Exception e) {
-            assertTrue(
-                    ExceptionUtils.findThrowableWithMessage(
-                                    e, "Unsupported options:\n\nsink.unknown.key")
-                            .isPresent());
-        }
-    }
-
-    @Test
-    public void testTypeWithUnsupportedPrecision() {
-        Map<String, String> options = getAllOptions();
-        // test unsupported timestamp precision
-        ResolvedSchema schema =
-                ResolvedSchema.of(
-                        Column.physical(ROWKEY, STRING()),
-                        Column.physical(
-                                FAMILY1, ROW(FIELD(COL1, TIMESTAMP(6)), FIELD(COL2, INT()))));
-        try {
-            createTableSource(schema, options);
-            fail("Should fail");
-        } catch (Exception e) {
-            assertTrue(
-                    ExceptionUtils.findThrowableWithMessage(
-                                    e,
-                                    "The precision 6 of TIMESTAMP type is out of the range [0, 3]"
-                                            + " supported by HBase connector")
-                            .isPresent());
-        }
-
-        try {
-            createTableSink(schema, options);
-            fail("Should fail");
-        } catch (Exception e) {
-            assertTrue(
-                    ExceptionUtils.findThrowableWithMessage(
-                                    e,
-                                    "The precision 6 of TIMESTAMP type is out of the range [0, 3]"
-                                            + " supported by HBase connector")
-                            .isPresent());
-        }
-        // test unsupported time precision
-        schema =
-                ResolvedSchema.of(
-                        Column.physical(ROWKEY, STRING()),
-                        Column.physical(FAMILY1, ROW(FIELD(COL1, TIME(6)), FIELD(COL2, INT()))));
-
-        try {
-            createTableSource(schema, options);
-            fail("Should fail");
-        } catch (Exception e) {
-            assertTrue(
-                    ExceptionUtils.findThrowableWithMessage(
-                                    e,
-                                    "The precision 6 of TIME type is out of the range [0, 3]"
-                                            + " supported by HBase connector")
-                            .isPresent());
-        }
-
-        try {
-            createTableSink(schema, options);
-            fail("Should fail");
-        } catch (Exception e) {
-            assertTrue(
-                    ExceptionUtils.findThrowableWithMessage(
-                                    e,
-                                    "The precision 6 of TIME type is out of the range [0, 3]"
-                                            + " supported by HBase connector")
-                            .isPresent());
-        }
-    }
-
-    private Map<String, String> getAllOptions() {
-        Map<String, String> options = new HashMap<>();
-        options.put("connector", "hbase-2.2");
-        options.put("table-name", "testHBastTable");
-        options.put("zookeeper.quorum", "localhost:2181");
-        options.put("zookeeper.znode.parent", "/flink");
-        options.put("properties.hbase.security.authentication", "kerberos");
-        return options;
-    }
-}
diff --git a/flink-connectors/flink-connector-hbase-2.2/src/test/java/org/apache/flink/connector/hbase2/HBaseTablePlanTest.java b/flink-connectors/flink-connector-hbase-2.2/src/test/java/org/apache/flink/connector/hbase2/HBaseTablePlanTest.java
deleted file mode 100644
index 048e08d..0000000
--- a/flink-connectors/flink-connector-hbase-2.2/src/test/java/org/apache/flink/connector/hbase2/HBaseTablePlanTest.java
+++ /dev/null
@@ -1,138 +0,0 @@
-/*
- * 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.hbase2;
-
-import org.apache.flink.table.api.TableConfig;
-import org.apache.flink.table.planner.utils.StreamTableTestUtil;
-import org.apache.flink.table.planner.utils.TableTestBase;
-
-import org.junit.Test;
-
-import static org.apache.flink.core.testutils.FlinkMatchers.containsCause;
-
-/** Plan tests for HBase connector, for example, testing projection push down. */
-public class HBaseTablePlanTest extends TableTestBase {
-
-    private final StreamTableTestUtil util = streamTestUtil(TableConfig.getDefault());
-
-    @Test
-    public void testMultipleRowKey() {
-        util.tableEnv()
-                .executeSql(
-                        "CREATE TABLE hTable ("
-                                + " family1 ROW<col1 INT>,"
-                                + " family2 ROW<col1 STRING, col2 BIGINT>,"
-                                + " rowkey INT,"
-                                + " rowkey2 STRING "
-                                + ") WITH ("
-                                + " 'connector' = 'hbase-2.2',"
-                                + " 'table-name' = 'my_table',"
-                                + " 'zookeeper.quorum' = 'localhost:2021'"
-                                + ")");
-        thrown().expect(
-                        containsCause(
-                                new IllegalArgumentException(
-                                        "Row key can't be set multiple times.")));
-        util.verifyExecPlan("SELECT * FROM hTable");
-    }
-
-    @Test
-    public void testNoneRowKey() {
-        util.tableEnv()
-                .executeSql(
-                        "CREATE TABLE hTable ("
-                                + " family1 ROW<col1 INT>,"
-                                + " family2 ROW<col1 STRING, col2 BIGINT>"
-                                + ") WITH ("
-                                + " 'connector' = 'hbase-2.2',"
-                                + " 'table-name' = 'my_table',"
-                                + " 'zookeeper.quorum' = 'localhost:2021'"
-                                + ")");
-        thrown().expect(
-                        containsCause(
-                                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.")));
-        util.verifyExecPlan("SELECT * FROM hTable");
-    }
-
-    @Test
-    public void testInvalidPrimaryKey() {
-        util.tableEnv()
-                .executeSql(
-                        "CREATE TABLE hTable ("
-                                + " family1 ROW<col1 INT>,"
-                                + " family2 ROW<col1 STRING, col2 BIGINT>,"
-                                + " rowkey STRING, "
-                                + " PRIMARY KEY (family1) NOT ENFORCED "
-                                + ") WITH ("
-                                + " 'connector' = 'hbase-2.2',"
-                                + " 'table-name' = 'my_table',"
-                                + " 'zookeeper.quorum' = 'localhost:2021'"
-                                + ")");
-        thrown().expect(
-                        containsCause(
-                                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.")));
-        util.verifyExecPlan("SELECT * FROM hTable");
-    }
-
-    @Test
-    public void testUnsupportedDataType() {
-        util.tableEnv()
-                .executeSql(
-                        "CREATE TABLE hTable ("
-                                + " family1 ROW<col1 INT>,"
-                                + " family2 ROW<col1 STRING, col2 BIGINT>,"
-                                + " col1 ARRAY<STRING>, "
-                                + " rowkey STRING, "
-                                + " PRIMARY KEY (rowkey) NOT ENFORCED "
-                                + ") WITH ("
-                                + " 'connector' = 'hbase-2.2',"
-                                + " 'table-name' = 'my_table',"
-                                + " 'zookeeper.quorum' = 'localhost:2021'"
-                                + ")");
-        thrown().expect(
-                        containsCause(
-                                new IllegalArgumentException(
-                                        "Unsupported field type 'ARRAY<STRING>' for HBase.")));
-        util.verifyExecPlan("SELECT * FROM hTable");
-    }
-
-    @Test
-    public void testProjectionPushDown() {
-        util.tableEnv()
-                .executeSql(
-                        "CREATE TABLE hTable ("
-                                + " family1 ROW<col1 INT>,"
-                                + " family2 ROW<col1 STRING, col2 BIGINT>,"
-                                + " family3 ROW<col1 DOUBLE, col2 BOOLEAN, col3 STRING>,"
-                                + " rowkey INT,"
-                                + " PRIMARY KEY (rowkey) NOT ENFORCED"
-                                + ") WITH ("
-                                + " 'connector' = 'hbase-2.2',"
-                                + " 'table-name' = 'my_table',"
-                                + " 'zookeeper.quorum' = 'localhost:2021'"
-                                + ")");
-        util.verifyExecPlan("SELECT h.family3, h.family2.col2 FROM hTable AS h");
-    }
-}
diff --git a/flink-connectors/flink-connector-hbase-2.2/src/test/java/org/apache/flink/connector/hbase2/source/HBaseRowDataAsyncLookupFunctionTest.java b/flink-connectors/flink-connector-hbase-2.2/src/test/java/org/apache/flink/connector/hbase2/source/HBaseRowDataAsyncLookupFunctionTest.java
deleted file mode 100644
index 1c1ad72..0000000
--- a/flink-connectors/flink-connector-hbase-2.2/src/test/java/org/apache/flink/connector/hbase2/source/HBaseRowDataAsyncLookupFunctionTest.java
+++ /dev/null
@@ -1,111 +0,0 @@
-/*
- * 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.hbase2.source;
-
-import org.apache.flink.connector.hbase.util.HBaseTableSchema;
-import org.apache.flink.connector.hbase2.util.HBaseTestBase;
-import org.apache.flink.table.api.DataTypes;
-import org.apache.flink.table.connector.source.lookup.LookupOptions;
-import org.apache.flink.table.data.RowData;
-import org.apache.flink.table.types.DataType;
-
-import org.apache.flink.shaded.guava31.com.google.common.collect.Lists;
-
-import org.junit.Test;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.CountDownLatch;
-import java.util.stream.Collectors;
-
-import static org.apache.flink.table.api.DataTypes.BIGINT;
-import static org.apache.flink.table.api.DataTypes.DOUBLE;
-import static org.apache.flink.table.api.DataTypes.FIELD;
-import static org.apache.flink.table.api.DataTypes.INT;
-import static org.apache.flink.table.api.DataTypes.ROW;
-import static org.apache.flink.table.api.DataTypes.STRING;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-/** Test suite for {@link HBaseRowDataAsyncLookupFunction}. */
-public class HBaseRowDataAsyncLookupFunctionTest extends HBaseTestBase {
-    @Test
-    public void testEval() throws Exception {
-        HBaseRowDataAsyncLookupFunction lookupFunction = buildRowDataAsyncLookupFunction();
-
-        lookupFunction.open(null);
-        final List<String> result = new ArrayList<>();
-        int[] rowkeys = {1, 2, 1, 12, 3, 12, 4, 3};
-        CountDownLatch latch = new CountDownLatch(rowkeys.length);
-        for (int rowkey : rowkeys) {
-            CompletableFuture<Collection<RowData>> future = new CompletableFuture<>();
-            lookupFunction.eval(future, rowkey);
-            future.whenComplete(
-                    (rs, t) -> {
-                        synchronized (result) {
-                            if (rs.isEmpty()) {
-                                result.add(rowkey + ": null");
-                            } else {
-                                rs.forEach(row -> result.add(rowkey + ": " + row.toString()));
-                            }
-                        }
-                        latch.countDown();
-                    });
-        }
-        // this verifies lookup calls are async
-        assertTrue(result.size() < rowkeys.length);
-        latch.await();
-        lookupFunction.close();
-        List<String> sortResult =
-                Lists.newArrayList(result).stream().sorted().collect(Collectors.toList());
-        List<String> expected = new ArrayList<>();
-        expected.add("12: null");
-        expected.add("12: null");
-        expected.add("1: +I(1,+I(10),+I(Hello-1,100),+I(1.01,false,Welt-1))");
-        expected.add("1: +I(1,+I(10),+I(Hello-1,100),+I(1.01,false,Welt-1))");
-        expected.add("2: +I(2,+I(20),+I(Hello-2,200),+I(2.02,true,Welt-2))");
-        expected.add("3: +I(3,+I(30),+I(Hello-3,300),+I(3.03,false,Welt-3))");
-        expected.add("3: +I(3,+I(30),+I(Hello-3,300),+I(3.03,false,Welt-3))");
-        expected.add("4: +I(4,+I(40),+I(null,400),+I(4.04,true,Welt-4))");
-        assertEquals(expected, sortResult);
-    }
-
-    private HBaseRowDataAsyncLookupFunction buildRowDataAsyncLookupFunction() {
-        DataType dataType =
-                ROW(
-                        FIELD(ROW_KEY, INT()),
-                        FIELD(FAMILY1, ROW(FIELD(F1COL1, INT()))),
-                        FIELD(FAMILY2, ROW(FIELD(F2COL1, STRING()), FIELD(F2COL2, BIGINT()))),
-                        FIELD(
-                                FAMILY3,
-                                ROW(
-                                        FIELD(F3COL1, DOUBLE()),
-                                        FIELD(F3COL2, DataTypes.BOOLEAN()),
-                                        FIELD(F3COL3, STRING()))));
-        HBaseTableSchema hbaseSchema = HBaseTableSchema.fromDataType(dataType);
-        return new HBaseRowDataAsyncLookupFunction(
-                getConf(),
-                TEST_TABLE_1,
-                hbaseSchema,
-                "null",
-                LookupOptions.MAX_RETRIES.defaultValue());
-    }
-}
diff --git a/flink-connectors/flink-connector-hbase-2.2/src/test/java/org/apache/flink/connector/hbase2/util/HBaseTestBase.java b/flink-connectors/flink-connector-hbase-2.2/src/test/java/org/apache/flink/connector/hbase2/util/HBaseTestBase.java
deleted file mode 100644
index 1e639ba..0000000
--- a/flink-connectors/flink-connector-hbase-2.2/src/test/java/org/apache/flink/connector/hbase2/util/HBaseTestBase.java
+++ /dev/null
@@ -1,270 +0,0 @@
-/*
- * 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.hbase2.util;
-
-import org.apache.flink.table.api.EnvironmentSettings;
-
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.client.Table;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.junit.Before;
-import org.junit.BeforeClass;
-
-import java.io.IOException;
-import java.math.BigDecimal;
-import java.sql.Date;
-import java.sql.Time;
-import java.sql.Timestamp;
-import java.util.ArrayList;
-import java.util.List;
-
-import static org.apache.flink.table.utils.DateTimeUtils.toInternal;
-
-/** Abstract IT case class for HBase. */
-public abstract class HBaseTestBase extends HBaseTestingClusterAutoStarter {
-
-    protected static final String TEST_TABLE_1 = "testTable1";
-    protected static final String TEST_TABLE_2 = "testTable2";
-    protected static final String TEST_TABLE_3 = "testTable3";
-
-    protected static final String ROW_KEY = "rowkey";
-
-    protected static final String FAMILY1 = "family1";
-    protected static final String F1COL1 = "col1";
-
-    protected static final String FAMILY2 = "family2";
-    protected static final String F2COL1 = "col1";
-    protected static final String F2COL2 = "col2";
-
-    protected static final String FAMILY3 = "family3";
-    protected static final String F3COL1 = "col1";
-    protected static final String F3COL2 = "col2";
-    protected static final String F3COL3 = "col3";
-
-    protected static final String FAMILY4 = "family4";
-    protected static final String F4COL1 = "col1";
-    protected static final String F4COL2 = "col2";
-    protected static final String F4COL3 = "col3";
-    protected static final String F4COL4 = "col4";
-
-    private static final byte[][] FAMILIES =
-            new byte[][] {
-                Bytes.toBytes(FAMILY1),
-                Bytes.toBytes(FAMILY2),
-                Bytes.toBytes(FAMILY3),
-                Bytes.toBytes(FAMILY4)
-            };
-
-    private static final byte[][] SPLIT_KEYS = new byte[][] {Bytes.toBytes(4)};
-
-    protected EnvironmentSettings streamSettings;
-    protected EnvironmentSettings batchSettings;
-
-    @BeforeClass
-    public static void activateHBaseCluster() throws IOException {
-        prepareTables();
-    }
-
-    @Before
-    public void before() {
-        this.streamSettings = EnvironmentSettings.inStreamingMode();
-        this.batchSettings = EnvironmentSettings.inBatchMode();
-    }
-
-    private static void prepareTables() throws IOException {
-        createHBaseTable1();
-        createHBaseTable2();
-        createHBaseTable3();
-    }
-
-    private static void createHBaseTable1() throws IOException {
-        // create a table
-        TableName tableName = TableName.valueOf(TEST_TABLE_1);
-        createTable(tableName, FAMILIES, SPLIT_KEYS);
-
-        // get the HTable instance
-        Table table = openTable(tableName);
-        List<Put> puts = new ArrayList<>();
-        // add some data
-        puts.add(
-                putRow(
-                        1,
-                        10,
-                        "Hello-1",
-                        100L,
-                        1.01,
-                        false,
-                        "Welt-1",
-                        Timestamp.valueOf("2019-08-18 19:00:00"),
-                        Date.valueOf("2019-08-18"),
-                        Time.valueOf("19:00:00"),
-                        new BigDecimal("12345678.0001")));
-        puts.add(
-                putRow(
-                        2,
-                        20,
-                        "Hello-2",
-                        200L,
-                        2.02,
-                        true,
-                        "Welt-2",
-                        Timestamp.valueOf("2019-08-18 19:01:00"),
-                        Date.valueOf("2019-08-18"),
-                        Time.valueOf("19:01:00"),
-                        new BigDecimal("12345678.0002")));
-        puts.add(
-                putRow(
-                        3,
-                        30,
-                        "Hello-3",
-                        300L,
-                        3.03,
-                        false,
-                        "Welt-3",
-                        Timestamp.valueOf("2019-08-18 19:02:00"),
-                        Date.valueOf("2019-08-18"),
-                        Time.valueOf("19:02:00"),
-                        new BigDecimal("12345678.0003")));
-        puts.add(
-                putRow(
-                        4,
-                        40,
-                        null,
-                        400L,
-                        4.04,
-                        true,
-                        "Welt-4",
-                        Timestamp.valueOf("2019-08-18 19:03:00"),
-                        Date.valueOf("2019-08-18"),
-                        Time.valueOf("19:03:00"),
-                        new BigDecimal("12345678.0004")));
-        puts.add(
-                putRow(
-                        5,
-                        50,
-                        "Hello-5",
-                        500L,
-                        5.05,
-                        false,
-                        "Welt-5",
-                        Timestamp.valueOf("2019-08-19 19:10:00"),
-                        Date.valueOf("2019-08-19"),
-                        Time.valueOf("19:10:00"),
-                        new BigDecimal("12345678.0005")));
-        puts.add(
-                putRow(
-                        6,
-                        60,
-                        "Hello-6",
-                        600L,
-                        6.06,
-                        true,
-                        "Welt-6",
-                        Timestamp.valueOf("2019-08-19 19:20:00"),
-                        Date.valueOf("2019-08-19"),
-                        Time.valueOf("19:20:00"),
-                        new BigDecimal("12345678.0006")));
-        puts.add(
-                putRow(
-                        7,
-                        70,
-                        "Hello-7",
-                        700L,
-                        7.07,
-                        false,
-                        "Welt-7",
-                        Timestamp.valueOf("2019-08-19 19:30:00"),
-                        Date.valueOf("2019-08-19"),
-                        Time.valueOf("19:30:00"),
-                        new BigDecimal("12345678.0007")));
-        puts.add(
-                putRow(
-                        8,
-                        80,
-                        null,
-                        800L,
-                        8.08,
-                        true,
-                        "Welt-8",
-                        Timestamp.valueOf("2019-08-19 19:40:00"),
-                        Date.valueOf("2019-08-19"),
-                        Time.valueOf("19:40:00"),
-                        new BigDecimal("12345678.0008")));
-
-        // append rows to table
-        table.put(puts);
-        table.close();
-    }
-
-    private static void createHBaseTable2() {
-        // create a table
-        TableName tableName = TableName.valueOf(TEST_TABLE_2);
-        createTable(tableName, FAMILIES, SPLIT_KEYS);
-    }
-
-    private static void createHBaseTable3() {
-        // create a table
-        byte[][] families =
-                new byte[][] {
-                    Bytes.toBytes(FAMILY1),
-                    Bytes.toBytes(FAMILY2),
-                    Bytes.toBytes(FAMILY3),
-                    Bytes.toBytes(FAMILY4),
-                };
-        TableName tableName = TableName.valueOf(TEST_TABLE_3);
-        createTable(tableName, families, SPLIT_KEYS);
-    }
-
-    private static Put putRow(
-            int rowKey,
-            int f1c1,
-            String f2c1,
-            long f2c2,
-            double f3c1,
-            boolean f3c2,
-            String f3c3,
-            Timestamp f4c1,
-            Date f4c2,
-            Time f4c3,
-            BigDecimal f4c4) {
-        Put put = new Put(Bytes.toBytes(rowKey));
-        // family 1
-        put.addColumn(Bytes.toBytes(FAMILY1), Bytes.toBytes(F1COL1), Bytes.toBytes(f1c1));
-        // family 2
-        if (f2c1 != null) {
-            put.addColumn(Bytes.toBytes(FAMILY2), Bytes.toBytes(F2COL1), Bytes.toBytes(f2c1));
-        }
-        put.addColumn(Bytes.toBytes(FAMILY2), Bytes.toBytes(F2COL2), Bytes.toBytes(f2c2));
-        // family 3
-        put.addColumn(Bytes.toBytes(FAMILY3), Bytes.toBytes(F3COL1), Bytes.toBytes(f3c1));
-        put.addColumn(Bytes.toBytes(FAMILY3), Bytes.toBytes(F3COL2), Bytes.toBytes(f3c2));
-        put.addColumn(Bytes.toBytes(FAMILY3), Bytes.toBytes(F3COL3), Bytes.toBytes(f3c3));
-
-        // family 4
-        put.addColumn(
-                Bytes.toBytes(FAMILY4), Bytes.toBytes(F4COL1), Bytes.toBytes(toInternal(f4c1)));
-        put.addColumn(
-                Bytes.toBytes(FAMILY4), Bytes.toBytes(F4COL2), Bytes.toBytes(toInternal(f4c2)));
-        put.addColumn(
-                Bytes.toBytes(FAMILY4), Bytes.toBytes(F4COL3), Bytes.toBytes(toInternal(f4c3)));
-        put.addColumn(Bytes.toBytes(FAMILY4), Bytes.toBytes(F4COL4), Bytes.toBytes(f4c4));
-        return put;
-    }
-}
diff --git a/flink-connectors/flink-connector-hbase-2.2/src/test/java/org/apache/flink/connector/hbase2/util/HBaseTestingClusterAutoStarter.java b/flink-connectors/flink-connector-hbase-2.2/src/test/java/org/apache/flink/connector/hbase2/util/HBaseTestingClusterAutoStarter.java
deleted file mode 100644
index 2482f1a..0000000
--- a/flink-connectors/flink-connector-hbase-2.2/src/test/java/org/apache/flink/connector/hbase2/util/HBaseTestingClusterAutoStarter.java
+++ /dev/null
@@ -1,160 +0,0 @@
-/*
- * Copyright The Apache Software Foundation
- *
- * 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.hbase2.util;
-
-import org.apache.commons.lang3.Range;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.MasterNotRunningException;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.ZooKeeperConnectionException;
-import org.apache.hadoop.hbase.client.Admin;
-import org.apache.hadoop.hbase.client.Table;
-import org.apache.hadoop.util.VersionUtil;
-import org.junit.AfterClass;
-import org.junit.Assume;
-import org.junit.BeforeClass;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-
-/**
- * By using this class as the super class of a set of tests you will have a HBase testing cluster
- * available that is very suitable for writing tests for scanning and filtering against.
- */
-public class HBaseTestingClusterAutoStarter {
-    private static final Log LOG = LogFactory.getLog(HBaseTestingClusterAutoStarter.class);
-
-    private static final Range<String> HADOOP_VERSION_RANGE =
-            Range.between("2.8.0", "3.0.3", VersionUtil::compareVersions);
-
-    private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
-    private static Admin admin = null;
-    private static List<TableName> createdTables = new ArrayList<>();
-
-    private static Configuration conf;
-
-    protected static void createTable(
-            TableName tableName, byte[][] columnFamilyName, byte[][] splitKeys) {
-        assertNotNull("HBaseAdmin is not initialized successfully.", admin);
-        HTableDescriptor desc = new HTableDescriptor(tableName);
-        for (byte[] fam : columnFamilyName) {
-            HColumnDescriptor colDef = new HColumnDescriptor(fam);
-            desc.addFamily(colDef);
-        }
-
-        try {
-            admin.createTable(desc, splitKeys);
-            createdTables.add(tableName);
-            assertTrue("Fail to create the table", admin.tableExists(tableName));
-        } catch (IOException e) {
-            assertNull("Exception found while creating table", e);
-        }
-    }
-
-    protected static Table openTable(TableName tableName) throws IOException {
-        Table table = TEST_UTIL.getConnection().getTable(tableName);
-        assertTrue("Fail to create the table", admin.tableExists(tableName));
-        return table;
-    }
-
-    private static void deleteTables() {
-        if (admin != null) {
-            for (TableName tableName : createdTables) {
-                try {
-                    if (admin.tableExists(tableName)) {
-                        admin.disableTable(tableName);
-                        admin.deleteTable(tableName);
-                    }
-                } catch (IOException e) {
-                    assertNull("Exception found deleting the table", e);
-                }
-            }
-        }
-    }
-
-    public static Configuration getConf() {
-        return conf;
-    }
-
-    public static String getZookeeperQuorum() {
-        return "localhost:" + TEST_UTIL.getZkCluster().getClientPort();
-    }
-
-    private static void initialize(Configuration c) {
-        conf = HBaseConfiguration.create(c);
-        // the default retry number is 15 in hbase-2.2, set 15 for test
-        conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 15);
-        try {
-            admin = TEST_UTIL.getAdmin();
-        } catch (MasterNotRunningException e) {
-            assertNull("Master is not running", e);
-        } catch (ZooKeeperConnectionException e) {
-            assertNull("Cannot connect to ZooKeeper", e);
-        } catch (IOException e) {
-            assertNull("IOException", e);
-        }
-    }
-
-    @BeforeClass
-    public static void setUp() throws Exception {
-        // HBase 2.2.3 HBaseTestingUtility works with only a certain range of hadoop versions
-        String hadoopVersion = System.getProperty("hadoop.version");
-        Assume.assumeTrue(HADOOP_VERSION_RANGE.contains(hadoopVersion));
-        TEST_UTIL.startMiniCluster(1);
-
-        // https://issues.apache.org/jira/browse/HBASE-11711
-        TEST_UTIL.getConfiguration().setInt("hbase.master.info.port", -1);
-
-        // Make sure the zookeeper quorum value contains the right port number (varies per run).
-        LOG.info("Hbase minicluster client port: " + TEST_UTIL.getZkCluster().getClientPort());
-        TEST_UTIL
-                .getConfiguration()
-                .set(
-                        "hbase.zookeeper.quorum",
-                        "localhost:" + TEST_UTIL.getZkCluster().getClientPort());
-
-        initialize(TEST_UTIL.getConfiguration());
-    }
-
-    @AfterClass
-    public static void tearDown() throws Exception {
-        if (conf == null) {
-            LOG.info("Skipping Hbase tear down. It was never started");
-            return;
-        }
-        LOG.info("HBase minicluster: Shutting down");
-        deleteTables();
-        TEST_UTIL.shutdownMiniCluster();
-        LOG.info("HBase minicluster: Down");
-    }
-}
diff --git a/flink-connectors/flink-connector-hbase-2.2/src/test/java/org/slf4j/impl/Log4jLoggerAdapter.java b/flink-connectors/flink-connector-hbase-2.2/src/test/java/org/slf4j/impl/Log4jLoggerAdapter.java
deleted file mode 100644
index 0287808..0000000
--- a/flink-connectors/flink-connector-hbase-2.2/src/test/java/org/slf4j/impl/Log4jLoggerAdapter.java
+++ /dev/null
@@ -1,22 +0,0 @@
-/*
- * 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.slf4j.impl;
-
-/** Fake appender to work around HBase referring to it directly. */
-public interface Log4jLoggerAdapter {}
diff --git a/flink-connectors/flink-connector-hbase-2.2/src/test/resources/archunit.properties b/flink-connectors/flink-connector-hbase-2.2/src/test/resources/archunit.properties
deleted file mode 100644
index 15be88c..0000000
--- a/flink-connectors/flink-connector-hbase-2.2/src/test/resources/archunit.properties
+++ /dev/null
@@ -1,31 +0,0 @@
-#
-# 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.
-#
-
-# By default we allow removing existing violations, but fail when new violations are added.
-freeze.store.default.allowStoreUpdate=true
-
-# Enable this if a new (frozen) rule has been added in order to create the initial store and record the existing violations.
-#freeze.store.default.allowStoreCreation=true
-
-# Enable this to add allow new violations to be recorded.
-# NOTE: Adding new violations should be avoided when possible. If the rule was correct to flag a new
-#       violation, please try to avoid creating the violation. If the violation was created due to a
-#       shortcoming of the rule, file a JIRA issue so the rule can be improved.
-#freeze.refreeze=true
-
-freeze.store.default.path=archunit-violations
diff --git a/flink-connectors/flink-connector-hbase-2.2/src/test/resources/hbase-site.xml b/flink-connectors/flink-connector-hbase-2.2/src/test/resources/hbase-site.xml
deleted file mode 100644
index 1e58ef4..0000000
--- a/flink-connectors/flink-connector-hbase-2.2/src/test/resources/hbase-site.xml
+++ /dev/null
@@ -1,29 +0,0 @@
-<?xml version="1.0"?>
-<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
-<!--
-  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.
--->
-
-<!-- Values used when running unit tests.  Specify any values in here that
-     should override the default values. -->
-
-<configuration>
-    <property>
-        <name>hbase_conf_key</name>
-        <value>hbase_conf_value!</value>
-    </property>
-</configuration>
diff --git a/flink-connectors/flink-connector-hbase-2.2/src/test/resources/log4j2-test.properties b/flink-connectors/flink-connector-hbase-2.2/src/test/resources/log4j2-test.properties
deleted file mode 100644
index 835c2ec..0000000
--- a/flink-connectors/flink-connector-hbase-2.2/src/test/resources/log4j2-test.properties
+++ /dev/null
@@ -1,28 +0,0 @@
-################################################################################
-#  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.
-################################################################################
-
-# Set root logger level to OFF to not flood build logs
-# set manually to INFO for debugging purposes
-rootLogger.level = OFF
-rootLogger.appenderRef.test.ref = TestLogger
-
-appender.testlogger.name = TestLogger
-appender.testlogger.type = CONSOLE
-appender.testlogger.target = SYSTEM_ERR
-appender.testlogger.layout.type = PatternLayout
-appender.testlogger.layout.pattern = %-4r [%t] %-5p %c %x - %m%n
diff --git a/flink-connectors/flink-connector-hbase-2.2/src/test/resources/org/apache/flink/connector/hbase2/HBaseTablePlanTest.xml b/flink-connectors/flink-connector-hbase-2.2/src/test/resources/org/apache/flink/connector/hbase2/HBaseTablePlanTest.xml
deleted file mode 100644
index 9885ccb..0000000
--- a/flink-connectors/flink-connector-hbase-2.2/src/test/resources/org/apache/flink/connector/hbase2/HBaseTablePlanTest.xml
+++ /dev/null
@@ -1,36 +0,0 @@
-<?xml version="1.0" ?>
-<!--
-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.
--->
-<Root>
-  <TestCase name="testProjectionPushDown">
-    <Resource name="sql">
-      <![CDATA[SELECT h.family3, h.family2.col2 FROM hTable AS h]]>
-    </Resource>
-    <Resource name="ast">
-      <![CDATA[
-LogicalProject(family3=[$2], col2=[$1.col2])
-+- LogicalTableScan(table=[[default_catalog, default_database, hTable]])
-]]>
-    </Resource>
-    <Resource name="optimized exec plan">
-      <![CDATA[
-Calc(select=[family3, family2.col2 AS col2])
-+- TableSourceScan(table=[[default_catalog, default_database, hTable, project=[family3, family2]]], fields=[family3, family2])
-]]>
-    </Resource>
-  </TestCase>
-</Root>
diff --git a/flink-connectors/flink-connector-hbase-base/archunit-violations/f8f77b71-7087-4b3f-88c5-29588fadcc52 b/flink-connectors/flink-connector-hbase-base/archunit-violations/f8f77b71-7087-4b3f-88c5-29588fadcc52
deleted file mode 100644
index e69de29..0000000
--- a/flink-connectors/flink-connector-hbase-base/archunit-violations/f8f77b71-7087-4b3f-88c5-29588fadcc52
+++ /dev/null
diff --git a/flink-connectors/flink-connector-hbase-base/archunit-violations/fd028eab-0e49-4e91-9c40-477a55f378f2 b/flink-connectors/flink-connector-hbase-base/archunit-violations/fd028eab-0e49-4e91-9c40-477a55f378f2
deleted file mode 100644
index e69de29..0000000
--- a/flink-connectors/flink-connector-hbase-base/archunit-violations/fd028eab-0e49-4e91-9c40-477a55f378f2
+++ /dev/null
diff --git a/flink-connectors/flink-connector-hbase-base/archunit-violations/stored.rules b/flink-connectors/flink-connector-hbase-base/archunit-violations/stored.rules
deleted file mode 100644
index c5f72df..0000000
--- a/flink-connectors/flink-connector-hbase-base/archunit-violations/stored.rules
+++ /dev/null
@@ -1,4 +0,0 @@
-#
-#Tue Feb 22 12:17:29 CET 2022
-Tests\ inheriting\ from\ AbstractTestBase\ should\ have\ name\ ending\ with\ ITCase=f8f77b71-7087-4b3f-88c5-29588fadcc52
-ITCASE\ tests\ should\ use\ a\ MiniCluster\ resource\ or\ extension=fd028eab-0e49-4e91-9c40-477a55f378f2
diff --git a/flink-connectors/flink-connector-hbase-base/pom.xml b/flink-connectors/flink-connector-hbase-base/pom.xml
deleted file mode 100644
index 1dd3b88..0000000
--- a/flink-connectors/flink-connector-hbase-base/pom.xml
+++ /dev/null
@@ -1,235 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-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.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
-
-	<modelVersion>4.0.0</modelVersion>
-
-	<parent>
-		<groupId>org.apache.flink</groupId>
-		<artifactId>flink-connectors</artifactId>
-		<version>1.19-SNAPSHOT</version>
-	</parent>
-
-	<artifactId>flink-connector-hbase-base</artifactId>
-	<name>Flink : Connectors : HBase base</name>
-	<packaging>jar</packaging>
-
-	<properties>
-		<hbase.version>1.4.3</hbase.version>
-	</properties>
-
-	<dependencies>
-
-		<!-- Core -->
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-core</artifactId>
-			<version>${project.version}</version>
-			<scope>provided</scope>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-java</artifactId>
-			<version>${project.version}</version>
-			<scope>provided</scope>
-		</dependency>
-
-		<!-- Table ecosystem -->
-
-		<!-- Projects depending on this project won't depend on flink-table-*. -->
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-table-api-java-bridge</artifactId>
-			<version>${project.version}</version>
-			<scope>provided</scope>
-			<optional>true</optional>
-		</dependency>
-
-		<!-- HBase -->
-
-		<dependency>
-			<groupId>org.apache.hadoop</groupId>
-			<artifactId>hadoop-common</artifactId>
-			<scope>provided</scope>
-			<exclusions>
-				<exclusion>
-					<groupId>ch.qos.reload4j</groupId>
-					<artifactId>reload4j</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.slf4j</groupId>
-					<artifactId>slf4j-reload4j</artifactId>
-				</exclusion>
-			</exclusions>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.hbase</groupId>
-			<artifactId>hbase-client</artifactId>
-			<version>${hbase.version}</version>
-			<exclusions>
-				<!-- Remove unneeded dependency, which is conflicting with our jetty-util version. -->
-				<exclusion>
-					<groupId>org.mortbay.jetty</groupId>
-					<artifactId>jetty-util</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.mortbay.jetty</groupId>
-					<artifactId>jetty</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.mortbay.jetty</groupId>
-					<artifactId>jetty-sslengine</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.mortbay.jetty</groupId>
-					<artifactId>jsp-2.1</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.mortbay.jetty</groupId>
-					<artifactId>jsp-api-2.1</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.mortbay.jetty</groupId>
-					<artifactId>servlet-api-2.5</artifactId>
-				</exclusion>
-				<!-- Bug in hbase annotations, can be removed when fixed. See FLINK-2153. -->
-				<exclusion>
-					<groupId>org.apache.hbase</groupId>
-					<artifactId>hbase-annotations</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>com.sun.jersey</groupId>
-					<artifactId>jersey-core</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.apache.hadoop</groupId>
-					<artifactId>hadoop-common</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.apache.hadoop</groupId>
-					<artifactId>hadoop-auth</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.apache.hadoop</groupId>
-					<artifactId>hadoop-annotations</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.apache.hadoop</groupId>
-					<artifactId>hadoop-mapreduce-client-core</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.apache.hadoop</groupId>
-					<artifactId>hadoop-client</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.apache.hadoop</groupId>
-					<artifactId>hadoop-hdfs</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>log4j</groupId>
-					<artifactId>log4j</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.slf4j</groupId>
-					<artifactId>slf4j-log4j12</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>ch.qos.reload4j</groupId>
-					<artifactId>reload4j</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.slf4j</groupId>
-					<artifactId>slf4j-reload4j</artifactId>
-				</exclusion>
-			</exclusions>
-		</dependency>
-
-		<dependency>
-			<!-- Bump hbase netty dependency -->
-			<groupId>io.netty</groupId>
-			<artifactId>netty-all</artifactId>
-			<version>4.1.46.Final</version>
-		</dependency>
-
-		<dependency>
-			<!-- Replaces Hbase log4j dependency -->
-			<groupId>org.apache.logging.log4j</groupId>
-			<artifactId>log4j-1.2-api</artifactId>
-			<scope>provided</scope>
-		</dependency>
-
-		<!-- ArchUit test dependencies -->
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-architecture-tests-test</artifactId>
-			<scope>test</scope>
-		</dependency>
-	</dependencies>
-
-	<build>
-		<plugins>
-			<plugin>
-				<groupId>org.apache.maven.plugins</groupId>
-				<artifactId>maven-jar-plugin</artifactId>
-				<executions>
-					<execution>
-						<goals>
-							<goal>test-jar</goal>
-						</goals>
-						<configuration>
-							<excludes>
-								<!-- test-jar is still used by JUnit4 modules -->
-								<exclude>META-INF/services/org.junit.jupiter.api.extension.Extension</exclude>
-							</excludes>
-						</configuration>
-					</execution>
-				</executions>
-			</plugin>
-		</plugins>
-	</build>
-
-	<profiles>
-		<profile>
-			<id>java11</id>
-			<activation>
-				<jdk>[11,)</jdk>
-			</activation>
-
-			<build>
-				<plugins>
-					<plugin>
-						<groupId>org.apache.maven.plugins</groupId>
-						<artifactId>maven-surefire-plugin</artifactId>
-						<configuration>
-							<!-- hbase currently does not support Java 11, see HBASE-21110 -->
-							<skip>true</skip>
-						</configuration>
-					</plugin>
-				</plugins>
-			</build>
-		</profile>
-	</profiles>
-
-</project>
diff --git a/flink-connectors/flink-connector-hbase-base/src/main/java/org/apache/flink/connector/hbase/options/HBaseWriteOptions.java b/flink-connectors/flink-connector-hbase-base/src/main/java/org/apache/flink/connector/hbase/options/HBaseWriteOptions.java
deleted file mode 100644
index 0ba2f88..0000000
--- a/flink-connectors/flink-connector-hbase-base/src/main/java/org/apache/flink/connector/hbase/options/HBaseWriteOptions.java
+++ /dev/null
@@ -1,163 +0,0 @@
-/*
- * 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.hadoop.hbase.client.ConnectionConfiguration;
-
-import java.io.Serializable;
-import java.util.Objects;
-
-/** Options for HBase writing. */
-@Internal
-public class HBaseWriteOptions implements Serializable {
-
-    private static final long serialVersionUID = 1L;
-
-    private final long bufferFlushMaxSizeInBytes;
-    private final long bufferFlushMaxRows;
-    private final long bufferFlushIntervalMillis;
-    private final Integer parallelism;
-
-    private HBaseWriteOptions(
-            long bufferFlushMaxSizeInBytes,
-            long bufferFlushMaxMutations,
-            long bufferFlushIntervalMillis,
-            Integer parallelism) {
-        this.bufferFlushMaxSizeInBytes = bufferFlushMaxSizeInBytes;
-        this.bufferFlushMaxRows = bufferFlushMaxMutations;
-        this.bufferFlushIntervalMillis = bufferFlushIntervalMillis;
-        this.parallelism = parallelism;
-    }
-
-    public long getBufferFlushMaxSizeInBytes() {
-        return bufferFlushMaxSizeInBytes;
-    }
-
-    public long getBufferFlushMaxRows() {
-        return bufferFlushMaxRows;
-    }
-
-    public long getBufferFlushIntervalMillis() {
-        return bufferFlushIntervalMillis;
-    }
-
-    public Integer getParallelism() {
-        return parallelism;
-    }
-
-    @Override
-    public String toString() {
-        return "HBaseWriteOptions{"
-                + "bufferFlushMaxSizeInBytes="
-                + bufferFlushMaxSizeInBytes
-                + ", bufferFlushMaxRows="
-                + bufferFlushMaxRows
-                + ", bufferFlushIntervalMillis="
-                + bufferFlushIntervalMillis
-                + ", parallelism="
-                + parallelism
-                + '}';
-    }
-
-    @Override
-    public boolean equals(Object o) {
-        if (this == o) {
-            return true;
-        }
-        if (o == null || getClass() != o.getClass()) {
-            return false;
-        }
-        HBaseWriteOptions that = (HBaseWriteOptions) o;
-        return bufferFlushMaxSizeInBytes == that.bufferFlushMaxSizeInBytes
-                && bufferFlushMaxRows == that.bufferFlushMaxRows
-                && bufferFlushIntervalMillis == that.bufferFlushIntervalMillis
-                && parallelism == that.parallelism;
-    }
-
-    @Override
-    public int hashCode() {
-        return Objects.hash(
-                bufferFlushMaxSizeInBytes,
-                bufferFlushMaxRows,
-                bufferFlushIntervalMillis,
-                parallelism);
-    }
-
-    /** Creates a builder for {@link HBaseWriteOptions}. */
-    public static Builder builder() {
-        return new Builder();
-    }
-
-    /** Builder for {@link HBaseWriteOptions}. */
-    public static class Builder {
-
-        private long bufferFlushMaxSizeInBytes = ConnectionConfiguration.WRITE_BUFFER_SIZE_DEFAULT;
-        private long bufferFlushMaxRows = 0;
-        private long bufferFlushIntervalMillis = 0;
-        private Integer parallelism;
-
-        /**
-         * Optional. Sets when to flush a buffered request based on the memory size of rows
-         * currently added. Default to <code>2mb</code>.
-         */
-        public Builder setBufferFlushMaxSizeInBytes(long bufferFlushMaxSizeInBytes) {
-            this.bufferFlushMaxSizeInBytes = bufferFlushMaxSizeInBytes;
-            return this;
-        }
-
-        /**
-         * Optional. Sets when to flush buffered request based on the number of rows currently
-         * added. Defaults to not set, i.e. won't flush based on the number of buffered rows.
-         */
-        public Builder setBufferFlushMaxRows(long bufferFlushMaxRows) {
-            this.bufferFlushMaxRows = bufferFlushMaxRows;
-            return this;
-        }
-
-        /**
-         * Optional. Sets a flush interval flushing buffered requesting if the interval passes, in
-         * milliseconds. Defaults to not set, i.e. won't flush based on flush interval.
-         */
-        public Builder setBufferFlushIntervalMillis(long bufferFlushIntervalMillis) {
-            this.bufferFlushIntervalMillis = bufferFlushIntervalMillis;
-            return this;
-        }
-
-        /**
-         * Optional. Defines the parallelism of the HBase sink operator. By default, the parallelism
-         * is determined by the framework using the same parallelism of the upstream chained
-         * operator.
-         */
-        public Builder setParallelism(Integer parallelism) {
-            this.parallelism = parallelism;
-            return this;
-        }
-
-        /** Creates a new instance of {@link HBaseWriteOptions}. */
-        public HBaseWriteOptions build() {
-            return new HBaseWriteOptions(
-                    bufferFlushMaxSizeInBytes,
-                    bufferFlushMaxRows,
-                    bufferFlushIntervalMillis,
-                    parallelism);
-        }
-    }
-}
diff --git a/flink-connectors/flink-connector-hbase-base/src/main/java/org/apache/flink/connector/hbase/sink/HBaseMutationConverter.java b/flink-connectors/flink-connector-hbase-base/src/main/java/org/apache/flink/connector/hbase/sink/HBaseMutationConverter.java
deleted file mode 100644
index d89bd4c..0000000
--- a/flink-connectors/flink-connector-hbase-base/src/main/java/org/apache/flink/connector/hbase/sink/HBaseMutationConverter.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * 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.sink;
-
-import org.apache.flink.annotation.Internal;
-
-import org.apache.hadoop.hbase.client.Delete;
-import org.apache.hadoop.hbase.client.Mutation;
-import org.apache.hadoop.hbase.client.Put;
-
-import java.io.Serializable;
-
-/**
- * A converter used to converts the input record into HBase {@link Mutation}.
- *
- * @param <T> type of input record.
- */
-@Internal
-public interface HBaseMutationConverter<T> extends Serializable {
-
-    /** Initialization method for the function. It is called once before conversion method. */
-    void open();
-
-    /**
-     * Converts the input record into HBase {@link Mutation}. A mutation can be a {@link Put} or
-     * {@link Delete}.
-     */
-    Mutation convertToMutation(T record);
-}
diff --git a/flink-connectors/flink-connector-hbase-base/src/main/java/org/apache/flink/connector/hbase/sink/HBaseSinkFunction.java b/flink-connectors/flink-connector-hbase-base/src/main/java/org/apache/flink/connector/hbase/sink/HBaseSinkFunction.java
deleted file mode 100644
index 4dd30b3..0000000
--- a/flink-connectors/flink-connector-hbase-base/src/main/java/org/apache/flink/connector/hbase/sink/HBaseSinkFunction.java
+++ /dev/null
@@ -1,259 +0,0 @@
-/*
- * 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.sink;
-
-import org.apache.flink.annotation.Internal;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.connector.hbase.util.HBaseConfigurationUtil;
-import org.apache.flink.runtime.state.FunctionInitializationContext;
-import org.apache.flink.runtime.state.FunctionSnapshotContext;
-import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
-import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
-import org.apache.flink.util.StringUtils;
-import org.apache.flink.util.concurrent.ExecutorThreadFactory;
-
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.TableNotFoundException;
-import org.apache.hadoop.hbase.client.BufferedMutator;
-import org.apache.hadoop.hbase.client.BufferedMutatorParams;
-import org.apache.hadoop.hbase.client.Connection;
-import org.apache.hadoop.hbase.client.ConnectionFactory;
-import org.apache.hadoop.hbase.client.RetriesExhaustedWithDetailsException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.util.concurrent.Executors;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.ScheduledFuture;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.atomic.AtomicReference;
-
-/**
- * The sink function for HBase.
- *
- * <p>This class leverage {@link BufferedMutator} to buffer multiple {@link
- * org.apache.hadoop.hbase.client.Mutation Mutations} before sending the requests to cluster. The
- * buffering strategy can be configured by {@code bufferFlushMaxSizeInBytes}, {@code
- * bufferFlushMaxMutations} and {@code bufferFlushIntervalMillis}.
- */
-@Internal
-public class HBaseSinkFunction<T> extends RichSinkFunction<T>
-        implements CheckpointedFunction, BufferedMutator.ExceptionListener {
-
-    private static final long serialVersionUID = 1L;
-    private static final Logger LOG = LoggerFactory.getLogger(HBaseSinkFunction.class);
-
-    private final String hTableName;
-    private final byte[] serializedConfig;
-
-    private final long bufferFlushMaxSizeInBytes;
-    private final long bufferFlushMaxMutations;
-    private final long bufferFlushIntervalMillis;
-    private final HBaseMutationConverter<T> mutationConverter;
-
-    private transient Connection connection;
-    private transient BufferedMutator mutator;
-
-    private transient ScheduledExecutorService executor;
-    private transient ScheduledFuture scheduledFuture;
-    private transient AtomicLong numPendingRequests;
-
-    private transient volatile boolean closed = false;
-
-    /**
-     * This is set from inside the {@link BufferedMutator.ExceptionListener} if a {@link Throwable}
-     * was thrown.
-     *
-     * <p>Errors will be checked and rethrown before processing each input element, and when the
-     * sink is closed.
-     */
-    private final AtomicReference<Throwable> failureThrowable = new AtomicReference<>();
-
-    public HBaseSinkFunction(
-            String hTableName,
-            org.apache.hadoop.conf.Configuration conf,
-            HBaseMutationConverter<T> mutationConverter,
-            long bufferFlushMaxSizeInBytes,
-            long bufferFlushMaxMutations,
-            long bufferFlushIntervalMillis) {
-        this.hTableName = hTableName;
-        // Configuration is not serializable
-        this.serializedConfig = HBaseConfigurationUtil.serializeConfiguration(conf);
-        this.mutationConverter = mutationConverter;
-        this.bufferFlushMaxSizeInBytes = bufferFlushMaxSizeInBytes;
-        this.bufferFlushMaxMutations = bufferFlushMaxMutations;
-        this.bufferFlushIntervalMillis = bufferFlushIntervalMillis;
-    }
-
-    @Override
-    public void open(Configuration parameters) throws Exception {
-        LOG.info("start open ...");
-        org.apache.hadoop.conf.Configuration config = prepareRuntimeConfiguration();
-        try {
-            this.mutationConverter.open();
-            this.numPendingRequests = new AtomicLong(0);
-
-            if (null == connection) {
-                this.connection = ConnectionFactory.createConnection(config);
-            }
-            // create a parameter instance, set the table name and custom listener reference.
-            BufferedMutatorParams params =
-                    new BufferedMutatorParams(TableName.valueOf(hTableName)).listener(this);
-            if (bufferFlushMaxSizeInBytes > 0) {
-                params.writeBufferSize(bufferFlushMaxSizeInBytes);
-            }
-            this.mutator = connection.getBufferedMutator(params);
-
-            if (bufferFlushIntervalMillis > 0 && bufferFlushMaxMutations != 1) {
-                this.executor =
-                        Executors.newScheduledThreadPool(
-                                1, new ExecutorThreadFactory("hbase-upsert-sink-flusher"));
-                this.scheduledFuture =
-                        this.executor.scheduleWithFixedDelay(
-                                () -> {
-                                    if (closed) {
-                                        return;
-                                    }
-                                    try {
-                                        flush();
-                                    } catch (Exception e) {
-                                        // fail the sink and skip the rest of the items
-                                        // if the failure handler decides to throw an exception
-                                        failureThrowable.compareAndSet(null, e);
-                                    }
-                                },
-                                bufferFlushIntervalMillis,
-                                bufferFlushIntervalMillis,
-                                TimeUnit.MILLISECONDS);
-            }
-        } catch (TableNotFoundException tnfe) {
-            LOG.error("The table " + hTableName + " not found ", tnfe);
-            throw new RuntimeException("HBase table '" + hTableName + "' not found.", tnfe);
-        } catch (IOException ioe) {
-            LOG.error("Exception while creating connection to HBase.", ioe);
-            throw new RuntimeException("Cannot create connection to HBase.", ioe);
-        }
-        LOG.info("end open.");
-    }
-
-    private org.apache.hadoop.conf.Configuration prepareRuntimeConfiguration() throws IOException {
-        // create default configuration from current runtime env (`hbase-site.xml` in classpath)
-        // first,
-        // and overwrite configuration using serialized configuration from client-side env
-        // (`hbase-site.xml` in classpath).
-        // user params from client-side have the highest priority
-        org.apache.hadoop.conf.Configuration runtimeConfig =
-                HBaseConfigurationUtil.deserializeConfiguration(
-                        serializedConfig, HBaseConfigurationUtil.getHBaseConfiguration());
-
-        // do validation: check key option(s) in final runtime configuration
-        if (StringUtils.isNullOrWhitespaceOnly(runtimeConfig.get(HConstants.ZOOKEEPER_QUORUM))) {
-            LOG.error(
-                    "Can not connect to HBase without {} configuration",
-                    HConstants.ZOOKEEPER_QUORUM);
-            throw new IOException(
-                    "Check HBase configuration failed, lost: '"
-                            + HConstants.ZOOKEEPER_QUORUM
-                            + "'!");
-        }
-
-        return runtimeConfig;
-    }
-
-    private void checkErrorAndRethrow() {
-        Throwable cause = failureThrowable.get();
-        if (cause != null) {
-            throw new RuntimeException("An error occurred in HBaseSink.", cause);
-        }
-    }
-
-    @SuppressWarnings("rawtypes")
-    @Override
-    public void invoke(T value, Context context) throws Exception {
-        checkErrorAndRethrow();
-
-        mutator.mutate(mutationConverter.convertToMutation(value));
-
-        // flush when the buffer number of mutations greater than the configured max size.
-        if (bufferFlushMaxMutations > 0
-                && numPendingRequests.incrementAndGet() >= bufferFlushMaxMutations) {
-            flush();
-        }
-    }
-
-    private void flush() throws IOException {
-        // BufferedMutator is thread-safe
-        mutator.flush();
-        numPendingRequests.set(0);
-        checkErrorAndRethrow();
-    }
-
-    @Override
-    public void close() throws Exception {
-        closed = true;
-
-        if (mutator != null) {
-            try {
-                mutator.close();
-            } catch (IOException e) {
-                LOG.warn("Exception occurs while closing HBase BufferedMutator.", e);
-            }
-            this.mutator = null;
-        }
-
-        if (connection != null) {
-            try {
-                connection.close();
-            } catch (IOException e) {
-                LOG.warn("Exception occurs while closing HBase Connection.", e);
-            }
-            this.connection = null;
-        }
-
-        if (scheduledFuture != null) {
-            scheduledFuture.cancel(false);
-            if (executor != null) {
-                executor.shutdownNow();
-            }
-        }
-    }
-
-    @Override
-    public void snapshotState(FunctionSnapshotContext context) throws Exception {
-        while (numPendingRequests.get() != 0) {
-            flush();
-        }
-    }
-
-    @Override
-    public void initializeState(FunctionInitializationContext context) throws Exception {
-        // nothing to do.
-    }
-
-    @Override
-    public void onException(RetriesExhaustedWithDetailsException exception, BufferedMutator mutator)
-            throws RetriesExhaustedWithDetailsException {
-        // fail the sink and skip the rest of the items
-        // if the failure handler decides to throw an exception
-        failureThrowable.compareAndSet(null, exception);
-    }
-}
diff --git a/flink-connectors/flink-connector-hbase-base/src/main/java/org/apache/flink/connector/hbase/sink/RowDataToMutationConverter.java b/flink-connectors/flink-connector-hbase-base/src/main/java/org/apache/flink/connector/hbase/sink/RowDataToMutationConverter.java
deleted file mode 100644
index 406a996..0000000
--- a/flink-connectors/flink-connector-hbase-base/src/main/java/org/apache/flink/connector/hbase/sink/RowDataToMutationConverter.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * 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.sink;
-
-import org.apache.flink.connector.hbase.util.HBaseSerde;
-import org.apache.flink.connector.hbase.util.HBaseTableSchema;
-import org.apache.flink.table.data.RowData;
-import org.apache.flink.types.RowKind;
-
-import org.apache.hadoop.hbase.client.Mutation;
-
-/**
- * An implementation of {@link HBaseMutationConverter} which converts {@link RowData} into {@link
- * Mutation}.
- */
-public class RowDataToMutationConverter implements HBaseMutationConverter<RowData> {
-    private static final long serialVersionUID = 1L;
-
-    private final HBaseTableSchema schema;
-    private final String nullStringLiteral;
-    private transient HBaseSerde serde;
-
-    public RowDataToMutationConverter(HBaseTableSchema schema, final String nullStringLiteral) {
-        this.schema = schema;
-        this.nullStringLiteral = nullStringLiteral;
-    }
-
-    @Override
-    public void open() {
-        this.serde = new HBaseSerde(schema, nullStringLiteral);
-    }
-
-    @Override
-    public Mutation convertToMutation(RowData record) {
-        RowKind kind = record.getRowKind();
-        if (kind == RowKind.INSERT || kind == RowKind.UPDATE_AFTER) {
-            return serde.createPutMutation(record);
-        } else {
-            return serde.createDeleteMutation(record);
-        }
-    }
-}
diff --git a/flink-connectors/flink-connector-hbase-base/src/main/java/org/apache/flink/connector/hbase/source/AbstractHBaseDynamicTableSource.java b/flink-connectors/flink-connector-hbase-base/src/main/java/org/apache/flink/connector/hbase/source/AbstractHBaseDynamicTableSource.java
deleted file mode 100644
index c530cde..0000000
--- a/flink-connectors/flink-connector-hbase-base/src/main/java/org/apache/flink/connector/hbase/source/AbstractHBaseDynamicTableSource.java
+++ /dev/null
@@ -1,140 +0,0 @@
-/*
- * 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.source;
-
-import org.apache.flink.annotation.Internal;
-import org.apache.flink.annotation.VisibleForTesting;
-import org.apache.flink.api.common.io.InputFormat;
-import org.apache.flink.connector.hbase.util.HBaseTableSchema;
-import org.apache.flink.table.connector.ChangelogMode;
-import org.apache.flink.table.connector.Projection;
-import org.apache.flink.table.connector.source.InputFormatProvider;
-import org.apache.flink.table.connector.source.LookupTableSource;
-import org.apache.flink.table.connector.source.ScanTableSource;
-import org.apache.flink.table.connector.source.abilities.SupportsProjectionPushDown;
-import org.apache.flink.table.connector.source.lookup.LookupFunctionProvider;
-import org.apache.flink.table.connector.source.lookup.PartialCachingLookupProvider;
-import org.apache.flink.table.connector.source.lookup.cache.LookupCache;
-import org.apache.flink.table.data.RowData;
-import org.apache.flink.table.types.DataType;
-
-import org.apache.hadoop.conf.Configuration;
-
-import javax.annotation.Nullable;
-
-import static org.apache.flink.util.Preconditions.checkArgument;
-
-/** HBase table source implementation. */
-@Internal
-public abstract class AbstractHBaseDynamicTableSource
-        implements ScanTableSource, LookupTableSource, SupportsProjectionPushDown {
-
-    protected final Configuration conf;
-    protected final String tableName;
-    protected HBaseTableSchema hbaseSchema;
-    protected final String nullStringLiteral;
-    protected final int maxRetryTimes;
-    @Nullable protected final LookupCache cache;
-
-    public AbstractHBaseDynamicTableSource(
-            Configuration conf,
-            String tableName,
-            HBaseTableSchema hbaseSchema,
-            String nullStringLiteral,
-            int maxRetryTimes,
-            @Nullable LookupCache cache) {
-        this.conf = conf;
-        this.tableName = tableName;
-        this.hbaseSchema = hbaseSchema;
-        this.nullStringLiteral = nullStringLiteral;
-        this.maxRetryTimes = maxRetryTimes;
-        this.cache = cache;
-    }
-
-    @Override
-    public ScanRuntimeProvider getScanRuntimeProvider(ScanContext runtimeProviderContext) {
-        return InputFormatProvider.of(getInputFormat());
-    }
-
-    protected abstract InputFormat<RowData, ?> getInputFormat();
-
-    @Override
-    public LookupRuntimeProvider getLookupRuntimeProvider(LookupContext context) {
-        checkArgument(
-                context.getKeys().length == 1 && context.getKeys()[0].length == 1,
-                "Currently, HBase table can only be lookup by single rowkey.");
-        checkArgument(
-                hbaseSchema.getRowKeyName().isPresent(),
-                "HBase schema must have a row key when used in lookup mode.");
-        checkArgument(
-                DataType.getFieldNames(hbaseSchema.convertToDataType())
-                        .get(context.getKeys()[0][0])
-                        .equals(hbaseSchema.getRowKeyName().get()),
-                "Currently, HBase table only supports lookup by rowkey field.");
-        HBaseRowDataLookupFunction lookupFunction =
-                new HBaseRowDataLookupFunction(
-                        conf, tableName, hbaseSchema, nullStringLiteral, maxRetryTimes);
-        if (cache != null) {
-            return PartialCachingLookupProvider.of(lookupFunction, cache);
-        } else {
-            return LookupFunctionProvider.of(lookupFunction);
-        }
-    }
-
-    @Override
-    public boolean supportsNestedProjection() {
-        // planner doesn't support nested projection push down yet.
-        return false;
-    }
-
-    @Override
-    public void applyProjection(int[][] projectedFields, DataType producedDataType) {
-        this.hbaseSchema =
-                HBaseTableSchema.fromDataType(
-                        Projection.of(projectedFields).project(hbaseSchema.convertToDataType()));
-    }
-
-    @Override
-    public ChangelogMode getChangelogMode() {
-        return ChangelogMode.insertOnly();
-    }
-
-    @Override
-    public String asSummaryString() {
-        return "HBase";
-    }
-
-    // -------------------------------------------------------------------------------------------
-
-    @VisibleForTesting
-    public HBaseTableSchema getHBaseTableSchema() {
-        return this.hbaseSchema;
-    }
-
-    @VisibleForTesting
-    public int getMaxRetryTimes() {
-        return maxRetryTimes;
-    }
-
-    @VisibleForTesting
-    @Nullable
-    public LookupCache getCache() {
-        return cache;
-    }
-}
diff --git a/flink-connectors/flink-connector-hbase-base/src/main/java/org/apache/flink/connector/hbase/source/HBaseRowDataLookupFunction.java b/flink-connectors/flink-connector-hbase-base/src/main/java/org/apache/flink/connector/hbase/source/HBaseRowDataLookupFunction.java
deleted file mode 100644
index 0e1ba54..0000000
--- a/flink-connectors/flink-connector-hbase-base/src/main/java/org/apache/flink/connector/hbase/source/HBaseRowDataLookupFunction.java
+++ /dev/null
@@ -1,189 +0,0 @@
-/*
- * 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.source;
-
-import org.apache.flink.annotation.Internal;
-import org.apache.flink.annotation.VisibleForTesting;
-import org.apache.flink.connector.hbase.util.HBaseConfigurationUtil;
-import org.apache.flink.connector.hbase.util.HBaseSerde;
-import org.apache.flink.connector.hbase.util.HBaseTableSchema;
-import org.apache.flink.table.data.GenericRowData;
-import org.apache.flink.table.data.RowData;
-import org.apache.flink.table.functions.FunctionContext;
-import org.apache.flink.table.functions.LookupFunction;
-import org.apache.flink.util.StringUtils;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.TableNotFoundException;
-import org.apache.hadoop.hbase.client.Connection;
-import org.apache.hadoop.hbase.client.ConnectionFactory;
-import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.HTable;
-import org.apache.hadoop.hbase.client.Result;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.Collections;
-
-/**
- * The HBaseRowDataLookupFunction is a standard user-defined table function, it can be used in
- * tableAPI and also useful for temporal table join plan in SQL. It looks up the result as {@link
- * RowData}.
- */
-@Internal
-public class HBaseRowDataLookupFunction extends LookupFunction {
-
-    private static final Logger LOG = LoggerFactory.getLogger(HBaseRowDataLookupFunction.class);
-    private static final long serialVersionUID = 1L;
-
-    private final String hTableName;
-    private final byte[] serializedConfig;
-    private final HBaseTableSchema hbaseTableSchema;
-    private final String nullStringLiteral;
-
-    private transient Connection hConnection;
-    private transient HTable table;
-    private transient HBaseSerde serde;
-
-    private final int maxRetryTimes;
-
-    public HBaseRowDataLookupFunction(
-            Configuration configuration,
-            String hTableName,
-            HBaseTableSchema hbaseTableSchema,
-            String nullStringLiteral,
-            int maxRetryTimes) {
-        this.serializedConfig = HBaseConfigurationUtil.serializeConfiguration(configuration);
-        this.hTableName = hTableName;
-        this.hbaseTableSchema = hbaseTableSchema;
-        this.nullStringLiteral = nullStringLiteral;
-        this.maxRetryTimes = maxRetryTimes;
-    }
-
-    /**
-     * The invoke entry point of lookup function.
-     *
-     * @param keyRow - A {@link RowData} that wraps lookup keys. Currently only support single
-     *     rowkey.
-     */
-    @Override
-    public Collection<RowData> lookup(RowData keyRow) throws IOException {
-        for (int retry = 0; retry <= maxRetryTimes; retry++) {
-            try {
-                // TODO: The implementation of LookupFunction will pass a GenericRowData as key row
-                // and it's safe to cast for now. We need to update the logic once we improve the
-                // LookupFunction in the future.
-                Get get = serde.createGet(((GenericRowData) keyRow).getField(0));
-                if (get != null) {
-                    Result result = table.get(get);
-                    if (!result.isEmpty()) {
-                        return Collections.singletonList(serde.convertToReusedRow(result));
-                    }
-                }
-                break;
-            } catch (IOException e) {
-                LOG.error(String.format("HBase lookup error, retry times = %d", retry), e);
-                if (retry >= maxRetryTimes) {
-                    throw new RuntimeException("Execution of HBase lookup failed.", e);
-                }
-                try {
-                    Thread.sleep(1000 * retry);
-                } catch (InterruptedException e1) {
-                    throw new RuntimeException(e1);
-                }
-            }
-        }
-        return Collections.emptyList();
-    }
-
-    private Configuration prepareRuntimeConfiguration() {
-        // create default configuration from current runtime env (`hbase-site.xml` in classpath)
-        // first,
-        // and overwrite configuration using serialized configuration from client-side env
-        // (`hbase-site.xml` in classpath).
-        // user params from client-side have the highest priority
-        Configuration runtimeConfig =
-                HBaseConfigurationUtil.deserializeConfiguration(
-                        serializedConfig, HBaseConfigurationUtil.getHBaseConfiguration());
-
-        // do validation: check key option(s) in final runtime configuration
-        if (StringUtils.isNullOrWhitespaceOnly(runtimeConfig.get(HConstants.ZOOKEEPER_QUORUM))) {
-            LOG.error(
-                    "can not connect to HBase without {} configuration",
-                    HConstants.ZOOKEEPER_QUORUM);
-            throw new IllegalArgumentException(
-                    "check HBase configuration failed, lost: '"
-                            + HConstants.ZOOKEEPER_QUORUM
-                            + "'!");
-        }
-
-        return runtimeConfig;
-    }
-
-    @Override
-    public void open(FunctionContext context) {
-        LOG.info("start open ...");
-        Configuration config = prepareRuntimeConfiguration();
-        try {
-            hConnection = ConnectionFactory.createConnection(config);
-            table = (HTable) hConnection.getTable(TableName.valueOf(hTableName));
-        } catch (TableNotFoundException tnfe) {
-            LOG.error("Table '{}' not found ", hTableName, tnfe);
-            throw new RuntimeException("HBase table '" + hTableName + "' not found.", tnfe);
-        } catch (IOException ioe) {
-            LOG.error("Exception while creating connection to HBase.", ioe);
-            throw new RuntimeException("Cannot create connection to HBase.", ioe);
-        }
-        this.serde = new HBaseSerde(hbaseTableSchema, nullStringLiteral);
-        LOG.info("end open.");
-    }
-
-    @Override
-    public void close() {
-        LOG.info("start close ...");
-        if (null != table) {
-            try {
-                table.close();
-                table = null;
-            } catch (IOException e) {
-                // ignore exception when close.
-                LOG.warn("exception when close table", e);
-            }
-        }
-        if (null != hConnection) {
-            try {
-                hConnection.close();
-                hConnection = null;
-            } catch (IOException e) {
-                // ignore exception when close.
-                LOG.warn("exception when close connection", e);
-            }
-        }
-        LOG.info("end close.");
-    }
-
-    @VisibleForTesting
-    public String getHTableName() {
-        return hTableName;
-    }
-}
diff --git a/flink-connectors/flink-connector-hbase-base/src/main/java/org/apache/flink/connector/hbase/source/TableInputSplit.java b/flink-connectors/flink-connector-hbase-base/src/main/java/org/apache/flink/connector/hbase/source/TableInputSplit.java
deleted file mode 100644
index b745fff..0000000
--- a/flink-connectors/flink-connector-hbase-base/src/main/java/org/apache/flink/connector/hbase/source/TableInputSplit.java
+++ /dev/null
@@ -1,90 +0,0 @@
-/*
- * 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.source;
-
-import org.apache.flink.annotation.Internal;
-import org.apache.flink.core.io.LocatableInputSplit;
-
-/**
- * This class implements a input splits for HBase. Each table input split corresponds to a key range
- * (low, high). All references to row below refer to the key of the row.
- */
-@Internal
-public class TableInputSplit extends LocatableInputSplit {
-
-    private static final long serialVersionUID = 1L;
-
-    /** The name of the table to retrieve data from. */
-    private final byte[] tableName;
-
-    /** The start row of the split. */
-    private final byte[] startRow;
-
-    /** The end row of the split. */
-    private final byte[] endRow;
-
-    /**
-     * Creates a new table input split.
-     *
-     * @param splitNumber the number of the input split
-     * @param hostnames the names of the hosts storing the data the input split refers to
-     * @param tableName the name of the table to retrieve data from
-     * @param startRow the start row of the split
-     * @param endRow the end row of the split
-     */
-    public TableInputSplit(
-            final int splitNumber,
-            final String[] hostnames,
-            final byte[] tableName,
-            final byte[] startRow,
-            final byte[] endRow) {
-        super(splitNumber, hostnames);
-
-        this.tableName = tableName;
-        this.startRow = startRow;
-        this.endRow = endRow;
-    }
-
-    /**
-     * Returns the table name.
-     *
-     * @return The table name.
-     */
-    public byte[] getTableName() {
-        return this.tableName;
-    }
-
-    /**
-     * Returns the start row.
-     *
-     * @return The start row.
-     */
-    public byte[] getStartRow() {
-        return this.startRow;
-    }
-
-    /**
-     * Returns the end row.
-     *
-     * @return The end row.
-     */
-    public byte[] getEndRow() {
-        return this.endRow;
-    }
-}
diff --git a/flink-connectors/flink-connector-hbase-base/src/main/java/org/apache/flink/connector/hbase/table/HBaseConnectorOptions.java b/flink-connectors/flink-connector-hbase-base/src/main/java/org/apache/flink/connector/hbase/table/HBaseConnectorOptions.java
deleted file mode 100644
index 0c8dc97..0000000
--- a/flink-connectors/flink-connector-hbase-base/src/main/java/org/apache/flink/connector/hbase/table/HBaseConnectorOptions.java
+++ /dev/null
@@ -1,126 +0,0 @@
-/*
- * 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.table;
-
-import org.apache.flink.annotation.PublicEvolving;
-import org.apache.flink.configuration.ConfigOption;
-import org.apache.flink.configuration.ConfigOptions;
-import org.apache.flink.configuration.MemorySize;
-import org.apache.flink.table.connector.source.lookup.LookupOptions;
-import org.apache.flink.table.factories.FactoryUtil;
-
-import java.time.Duration;
-
-/** Options for the HBase connector. */
-@PublicEvolving
-public class HBaseConnectorOptions {
-
-    public static final ConfigOption<String> TABLE_NAME =
-            ConfigOptions.key("table-name")
-                    .stringType()
-                    .noDefaultValue()
-                    .withDescription(
-                            "The name of HBase table to connect. "
-                                    + "By default, the table is in 'default' namespace. "
-                                    + "To assign the table a specified namespace you need to use 'namespace:table'.");
-
-    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.");
-
-    public static final ConfigOption<Boolean> LOOKUP_ASYNC =
-            ConfigOptions.key("lookup.async")
-                    .booleanType()
-                    .defaultValue(false)
-                    .withDescription("whether to set async lookup.");
-
-    /** @deprecated Please use {@link LookupOptions#PARTIAL_CACHE_MAX_ROWS} instead. */
-    @Deprecated
-    public static final ConfigOption<Long> LOOKUP_CACHE_MAX_ROWS =
-            ConfigOptions.key("lookup.cache.max-rows")
-                    .longType()
-                    .defaultValue(-1L)
-                    .withDescription(
-                            "the max number of rows of lookup cache, over this value, the oldest rows will "
-                                    + "be eliminated. \"cache.max-rows\" and \"cache.ttl\" options must all be specified if any of them is "
-                                    + "specified. Cache is not enabled as default.");
-
-    /** @deprecated Please use {@link LookupOptions#PARTIAL_CACHE_EXPIRE_AFTER_WRITE} instead. */
-    @Deprecated
-    public static final ConfigOption<Duration> LOOKUP_CACHE_TTL =
-            ConfigOptions.key("lookup.cache.ttl")
-                    .durationType()
-                    .defaultValue(Duration.ofSeconds(0))
-                    .withDescription("the cache time to live.");
-
-    /** @deprecated Please used {@link LookupOptions#MAX_RETRIES} instead. */
-    public static final ConfigOption<Integer> LOOKUP_MAX_RETRIES =
-            ConfigOptions.key("lookup.max-retries")
-                    .intType()
-                    .defaultValue(3)
-                    .withDescription("the max retry times if lookup database failed.");
-
-    public static final ConfigOption<Integer> SINK_PARALLELISM = FactoryUtil.SINK_PARALLELISM;
-
-    private HBaseConnectorOptions() {}
-}
diff --git a/flink-connectors/flink-connector-hbase-base/src/main/java/org/apache/flink/connector/hbase/table/HBaseConnectorOptionsUtil.java b/flink-connectors/flink-connector-hbase-base/src/main/java/org/apache/flink/connector/hbase/table/HBaseConnectorOptionsUtil.java
deleted file mode 100644
index 2141fe1..0000000
--- a/flink-connectors/flink-connector-hbase-base/src/main/java/org/apache/flink/connector/hbase/table/HBaseConnectorOptionsUtil.java
+++ /dev/null
@@ -1,134 +0,0 @@
-/*
- * 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.table;
-
-import org.apache.flink.annotation.Internal;
-import org.apache.flink.configuration.ReadableConfig;
-import org.apache.flink.connector.hbase.options.HBaseWriteOptions;
-import org.apache.flink.connector.hbase.util.HBaseConfigurationUtil;
-import org.apache.flink.connector.hbase.util.HBaseTableSchema;
-import org.apache.flink.table.types.DataType;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HConstants;
-
-import java.util.Map;
-import java.util.Properties;
-
-import static org.apache.flink.connector.hbase.table.HBaseConnectorOptions.SINK_BUFFER_FLUSH_INTERVAL;
-import static org.apache.flink.connector.hbase.table.HBaseConnectorOptions.SINK_BUFFER_FLUSH_MAX_ROWS;
-import static org.apache.flink.connector.hbase.table.HBaseConnectorOptions.SINK_BUFFER_FLUSH_MAX_SIZE;
-import static org.apache.flink.connector.hbase.table.HBaseConnectorOptions.SINK_PARALLELISM;
-import static org.apache.flink.connector.hbase.table.HBaseConnectorOptions.ZOOKEEPER_QUORUM;
-import static org.apache.flink.connector.hbase.table.HBaseConnectorOptions.ZOOKEEPER_ZNODE_PARENT;
-
-/** Utilities for {@link HBaseConnectorOptions}. */
-@Internal
-public class HBaseConnectorOptionsUtil {
-
-    /** 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(DataType dataType, int[] primaryKeyIndexes) {
-        HBaseTableSchema hbaseSchema = HBaseTableSchema.fromDataType(dataType);
-        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.");
-        }
-        if (primaryKeyIndexes.length == 0) {
-            return;
-        }
-        if (primaryKeyIndexes.length > 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(DataType.getFieldNames(dataType).get(primaryKeyIndexes[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.");
-        }
-    }
-
-    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. */
-    public static Configuration getHBaseConfiguration(ReadableConfig tableOptions) {
-        // create default configuration from current runtime env (`hbase-site.xml` in classpath)
-        // first,
-        Configuration hbaseClientConf = HBaseConfigurationUtil.getHBaseConfiguration();
-        hbaseClientConf.set(HConstants.ZOOKEEPER_QUORUM, tableOptions.get(ZOOKEEPER_QUORUM));
-        hbaseClientConf.set(
-                HConstants.ZOOKEEPER_ZNODE_PARENT, tableOptions.get(ZOOKEEPER_ZNODE_PARENT));
-        // add HBase properties
-        final Properties properties =
-                getHBaseClientProperties(
-                        ((org.apache.flink.configuration.Configuration) tableOptions).toMap());
-        properties.forEach((k, v) -> hbaseClientConf.set(k.toString(), v.toString()));
-        return hbaseClientConf;
-    }
-
-    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 whether 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));
-    }
-
-    private HBaseConnectorOptionsUtil() {}
-}
diff --git a/flink-connectors/flink-connector-hbase-base/src/main/java/org/apache/flink/connector/hbase/util/HBaseConfigurationUtil.java b/flink-connectors/flink-connector-hbase-base/src/main/java/org/apache/flink/connector/hbase/util/HBaseConfigurationUtil.java
deleted file mode 100644
index b6679bd..0000000
--- a/flink-connectors/flink-connector-hbase-base/src/main/java/org/apache/flink/connector/hbase/util/HBaseConfigurationUtil.java
+++ /dev/null
@@ -1,215 +0,0 @@
-/*
- * 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.util;
-
-import org.apache.flink.annotation.Internal;
-import org.apache.flink.util.Preconditions;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.io.Writable;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.File;
-import java.io.IOException;
-
-/** This class helps to do serialization for hadoop Configuration and HBase-related classes. */
-@Internal
-public class HBaseConfigurationUtil {
-
-    private static final Logger LOG = LoggerFactory.getLogger(HBaseConfigurationUtil.class);
-
-    public static final String ENV_HBASE_CONF_DIR = "HBASE_CONF_DIR";
-
-    public static Configuration getHBaseConfiguration() {
-
-        // Instantiate an HBaseConfiguration to load the hbase-default.xml and hbase-site.xml from
-        // the classpath.
-        Configuration result = HBaseConfiguration.create();
-        boolean foundHBaseConfiguration = false;
-
-        // We need to load both hbase-default.xml and hbase-site.xml to the hbase configuration
-        // The properties of a newly added resource will override the ones in previous resources, so
-        // a configuration
-        // file with higher priority should be added later.
-
-        // Approach 1: HBASE_HOME environment variables
-        String possibleHBaseConfPath = null;
-
-        final String hbaseHome = System.getenv("HBASE_HOME");
-        if (hbaseHome != null) {
-            LOG.debug("Searching HBase configuration files in HBASE_HOME: {}", hbaseHome);
-            possibleHBaseConfPath = hbaseHome + "/conf";
-        }
-
-        if (possibleHBaseConfPath != null) {
-            foundHBaseConfiguration = addHBaseConfIfFound(result, possibleHBaseConfPath);
-        }
-
-        // Approach 2: HBASE_CONF_DIR environment variable
-        String hbaseConfDir = System.getenv("HBASE_CONF_DIR");
-        if (hbaseConfDir != null) {
-            LOG.debug("Searching HBase configuration files in HBASE_CONF_DIR: {}", hbaseConfDir);
-            foundHBaseConfiguration =
-                    addHBaseConfIfFound(result, hbaseConfDir) || foundHBaseConfiguration;
-        }
-
-        if (!foundHBaseConfiguration) {
-            LOG.warn(
-                    "Could not find HBase configuration via any of the supported methods "
-                            + "(Flink configuration, environment variables).");
-        }
-
-        return result;
-    }
-
-    /**
-     * Search HBase configuration files in the given path, and add them to the configuration if
-     * found.
-     */
-    private static boolean addHBaseConfIfFound(
-            Configuration configuration, String possibleHBaseConfPath) {
-        boolean foundHBaseConfiguration = false;
-        if (new File(possibleHBaseConfPath).exists()) {
-            if (new File(possibleHBaseConfPath + "/hbase-default.xml").exists()) {
-                configuration.addResource(
-                        new org.apache.hadoop.fs.Path(
-                                possibleHBaseConfPath + "/hbase-default.xml"));
-                LOG.debug(
-                        "Adding "
-                                + possibleHBaseConfPath
-                                + "/hbase-default.xml to hbase configuration");
-                foundHBaseConfiguration = true;
-            }
-            if (new File(possibleHBaseConfPath + "/hbase-site.xml").exists()) {
-                configuration.addResource(
-                        new org.apache.hadoop.fs.Path(possibleHBaseConfPath + "/hbase-site.xml"));
-                LOG.debug(
-                        "Adding "
-                                + possibleHBaseConfPath
-                                + "/hbase-site.xml to hbase configuration");
-                foundHBaseConfiguration = true;
-            }
-        }
-        return foundHBaseConfiguration;
-    }
-
-    /** Serialize a Hadoop {@link Configuration} into byte[]. */
-    public static byte[] serializeConfiguration(Configuration conf) {
-        try {
-            return serializeWritable(conf);
-        } catch (IOException e) {
-            throw new RuntimeException(
-                    "Encounter an IOException when serialize the Configuration.", e);
-        }
-    }
-
-    /**
-     * Deserialize a Hadoop {@link Configuration} from byte[]. Deserialize configs to {@code
-     * targetConfig} if it is set.
-     */
-    public static Configuration deserializeConfiguration(
-            byte[] serializedConfig, Configuration targetConfig) {
-        if (null == targetConfig) {
-            targetConfig = new Configuration();
-        }
-        try {
-            deserializeWritable(targetConfig, serializedConfig);
-        } catch (IOException e) {
-            throw new RuntimeException(
-                    "Encounter an IOException when deserialize the Configuration.", e);
-        }
-        return targetConfig;
-    }
-
-    /**
-     * Serialize writable byte[].
-     *
-     * @param <T> the type parameter
-     * @param writable the writable
-     * @return the byte [ ]
-     * @throws IOException the io exception
-     */
-    private static <T extends Writable> byte[] serializeWritable(T writable) throws IOException {
-        Preconditions.checkArgument(writable != null);
-
-        ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream();
-        DataOutputStream outputStream = new DataOutputStream(byteArrayOutputStream);
-        writable.write(outputStream);
-        return byteArrayOutputStream.toByteArray();
-    }
-
-    /**
-     * Deserialize writable.
-     *
-     * @param <T> the type parameter
-     * @param writable the writable
-     * @param bytes the bytes
-     * @throws IOException the io exception
-     */
-    private static <T extends Writable> void deserializeWritable(T writable, byte[] bytes)
-            throws IOException {
-        Preconditions.checkArgument(writable != null);
-        Preconditions.checkArgument(bytes != null);
-
-        ByteArrayInputStream byteArrayInputStream = new ByteArrayInputStream(bytes);
-        DataInputStream dataInputStream = new DataInputStream(byteArrayInputStream);
-        writable.readFields(dataInputStream);
-    }
-
-    public static org.apache.hadoop.conf.Configuration createHBaseConf() {
-        org.apache.hadoop.conf.Configuration hbaseClientConf = HBaseConfiguration.create();
-
-        String hbaseConfDir = System.getenv(ENV_HBASE_CONF_DIR);
-
-        if (hbaseConfDir != null) {
-            if (new File(hbaseConfDir).exists()) {
-                String coreSite = hbaseConfDir + "/core-site.xml";
-                String hdfsSite = hbaseConfDir + "/hdfs-site.xml";
-                String hbaseSite = hbaseConfDir + "/hbase-site.xml";
-                if (new File(coreSite).exists()) {
-                    hbaseClientConf.addResource(new org.apache.hadoop.fs.Path(coreSite));
-                    LOG.info("Adding " + coreSite + " to hbase configuration");
-                }
-                if (new File(hdfsSite).exists()) {
-                    hbaseClientConf.addResource(new org.apache.hadoop.fs.Path(hdfsSite));
-                    LOG.info("Adding " + hdfsSite + " to hbase configuration");
-                }
-                if (new File(hbaseSite).exists()) {
-                    hbaseClientConf.addResource(new org.apache.hadoop.fs.Path(hbaseSite));
-                    LOG.info("Adding " + hbaseSite + " to hbase configuration");
-                }
-            } else {
-                LOG.warn(
-                        "HBase config directory '{}' not found, cannot load HBase configuration.",
-                        hbaseConfDir);
-            }
-        } else {
-            LOG.warn(
-                    "{} env variable not found, cannot load HBase configuration.",
-                    ENV_HBASE_CONF_DIR);
-        }
-        return hbaseClientConf;
-    }
-}
diff --git a/flink-connectors/flink-connector-hbase-base/src/main/java/org/apache/flink/connector/hbase/util/HBaseSerde.java b/flink-connectors/flink-connector-hbase-base/src/main/java/org/apache/flink/connector/hbase/util/HBaseSerde.java
deleted file mode 100644
index 6680017..0000000
--- a/flink-connectors/flink-connector-hbase-base/src/main/java/org/apache/flink/connector/hbase/util/HBaseSerde.java
+++ /dev/null
@@ -1,531 +0,0 @@
-/*
- * 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.util;
-
-import org.apache.flink.table.data.DecimalData;
-import org.apache.flink.table.data.GenericRowData;
-import org.apache.flink.table.data.RowData;
-import org.apache.flink.table.data.StringData;
-import org.apache.flink.table.data.TimestampData;
-import org.apache.flink.table.types.DataType;
-import org.apache.flink.table.types.logical.DecimalType;
-import org.apache.flink.table.types.logical.LogicalType;
-import org.apache.flink.table.types.logical.LogicalTypeFamily;
-
-import org.apache.hadoop.hbase.client.Delete;
-import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.util.Bytes;
-
-import javax.annotation.Nullable;
-
-import java.io.Serializable;
-import java.math.BigDecimal;
-import java.nio.charset.StandardCharsets;
-import java.util.Arrays;
-
-import static org.apache.flink.table.types.logical.utils.LogicalTypeChecks.getPrecision;
-import static org.apache.flink.util.Preconditions.checkArgument;
-
-/** Utilities for HBase serialization and deserialization. */
-public class HBaseSerde {
-
-    private static final byte[] EMPTY_BYTES = new byte[] {};
-
-    private static final int MIN_TIMESTAMP_PRECISION = 0;
-    private static final int MAX_TIMESTAMP_PRECISION = 3;
-    private static final int MIN_TIME_PRECISION = 0;
-    private static final int MAX_TIME_PRECISION = 3;
-
-    private final byte[] nullStringBytes;
-
-    // row key index in output row
-    private final int rowkeyIndex;
-
-    // family keys
-    private final byte[][] families;
-    // qualifier keys
-    private final byte[][][] qualifiers;
-
-    private final int fieldLength;
-
-    private GenericRowData reusedRow;
-    private GenericRowData[] reusedFamilyRows;
-
-    private final @Nullable FieldEncoder keyEncoder;
-    private final @Nullable FieldDecoder keyDecoder;
-    private final FieldEncoder[][] qualifierEncoders;
-    private final FieldDecoder[][] qualifierDecoders;
-    private final GenericRowData rowWithRowKey;
-    private final HBaseStronglyIncreasingTsGenerator timestampGenerator;
-
-    public HBaseSerde(HBaseTableSchema hbaseSchema, final String nullStringLiteral) {
-        this.families = hbaseSchema.getFamilyKeys();
-        this.rowkeyIndex = hbaseSchema.getRowKeyIndex();
-        LogicalType rowkeyType =
-                hbaseSchema.getRowKeyDataType().map(DataType::getLogicalType).orElse(null);
-
-        // field length need take row key into account if it exists.
-        if (rowkeyIndex != -1 && rowkeyType != null) {
-            this.fieldLength = families.length + 1;
-            this.keyEncoder = createFieldEncoder(rowkeyType);
-            this.keyDecoder = createFieldDecoder(rowkeyType);
-        } else {
-            this.fieldLength = families.length;
-            this.keyEncoder = null;
-            this.keyDecoder = null;
-        }
-        this.nullStringBytes = nullStringLiteral.getBytes(StandardCharsets.UTF_8);
-
-        // prepare output rows
-        this.reusedRow = new GenericRowData(fieldLength);
-        this.reusedFamilyRows = new GenericRowData[families.length];
-
-        this.qualifiers = new byte[families.length][][];
-        this.qualifierEncoders = new FieldEncoder[families.length][];
-        this.qualifierDecoders = new FieldDecoder[families.length][];
-        String[] familyNames = hbaseSchema.getFamilyNames();
-        for (int f = 0; f < families.length; f++) {
-            this.qualifiers[f] = hbaseSchema.getQualifierKeys(familyNames[f]);
-            DataType[] dataTypes = hbaseSchema.getQualifierDataTypes(familyNames[f]);
-            this.qualifierEncoders[f] =
-                    Arrays.stream(dataTypes)
-                            .map(DataType::getLogicalType)
-                            .map(t -> createNullableFieldEncoder(t, nullStringBytes))
-                            .toArray(FieldEncoder[]::new);
-            this.qualifierDecoders[f] =
-                    Arrays.stream(dataTypes)
-                            .map(DataType::getLogicalType)
-                            .map(t -> createNullableFieldDecoder(t, nullStringBytes))
-                            .toArray(FieldDecoder[]::new);
-            this.reusedFamilyRows[f] = new GenericRowData(dataTypes.length);
-        }
-        this.rowWithRowKey = new GenericRowData(1);
-        this.timestampGenerator = new HBaseStronglyIncreasingTsGenerator();
-    }
-
-    /**
-     * Returns an instance of Put that writes record to HBase table.
-     *
-     * @return The appropriate instance of Put for this use case.
-     */
-    public @Nullable Put createPutMutation(RowData row) {
-        checkArgument(keyEncoder != null, "row key is not set.");
-        byte[] rowkey = keyEncoder.encode(row, rowkeyIndex);
-        if (rowkey.length == 0) {
-            // drop dirty records, rowkey shouldn't be zero length
-            return null;
-        }
-        // upsert
-        Put put = new Put(rowkey, timestampGenerator.get());
-        for (int i = 0; i < fieldLength; i++) {
-            if (i != rowkeyIndex) {
-                int f = i > rowkeyIndex ? i - 1 : i;
-                // get family key
-                byte[] familyKey = families[f];
-                RowData familyRow = row.getRow(i, qualifiers[f].length);
-                for (int q = 0; q < this.qualifiers[f].length; q++) {
-                    // get quantifier key
-                    byte[] qualifier = qualifiers[f][q];
-                    // serialize value
-                    byte[] value = qualifierEncoders[f][q].encode(familyRow, q);
-                    put.addColumn(familyKey, qualifier, value);
-                }
-            }
-        }
-        return put;
-    }
-
-    /**
-     * Returns an instance of Delete that remove record from HBase table.
-     *
-     * @return The appropriate instance of Delete for this use case.
-     */
-    public @Nullable Delete createDeleteMutation(RowData row) {
-        checkArgument(keyEncoder != null, "row key is not set.");
-        byte[] rowkey = keyEncoder.encode(row, rowkeyIndex);
-        if (rowkey.length == 0) {
-            // drop dirty records, rowkey shouldn't be zero length
-            return null;
-        }
-        // delete
-        Delete delete = new Delete(rowkey, timestampGenerator.get());
-        for (int i = 0; i < fieldLength; i++) {
-            if (i != rowkeyIndex) {
-                int f = i > rowkeyIndex ? i - 1 : i;
-                // get family key
-                byte[] familyKey = families[f];
-                for (int q = 0; q < this.qualifiers[f].length; q++) {
-                    // get quantifier key
-                    byte[] qualifier = qualifiers[f][q];
-                    delete.addColumns(familyKey, qualifier);
-                }
-            }
-        }
-        return delete;
-    }
-
-    /**
-     * Returns an instance of Scan that retrieves the required subset of records from the HBase
-     * table.
-     *
-     * @return The appropriate instance of Scan for this use case.
-     */
-    public Scan createScan() {
-        Scan scan = new Scan();
-        for (int f = 0; f < families.length; f++) {
-            byte[] family = families[f];
-            for (int q = 0; q < qualifiers[f].length; q++) {
-                byte[] quantifier = qualifiers[f][q];
-                scan.addColumn(family, quantifier);
-            }
-        }
-        return scan;
-    }
-
-    /**
-     * Returns an instance of Get that retrieves the matches records from the HBase table.
-     *
-     * @return The appropriate instance of Get for this use case.
-     */
-    public Get createGet(Object rowKey) {
-        checkArgument(keyEncoder != null, "row key is not set.");
-        rowWithRowKey.setField(0, rowKey);
-        byte[] rowkey = keyEncoder.encode(rowWithRowKey, 0);
-        if (rowkey.length == 0) {
-            // drop dirty records, rowkey shouldn't be zero length
-            return null;
-        }
-        Get get = new Get(rowkey);
-        for (int f = 0; f < families.length; f++) {
-            byte[] family = families[f];
-            for (byte[] qualifier : qualifiers[f]) {
-                get.addColumn(family, qualifier);
-            }
-        }
-        return get;
-    }
-
-    /**
-     * Converts HBase {@link Result} into a new {@link RowData} instance.
-     *
-     * <p>Note: this method is thread-safe.
-     */
-    public RowData convertToNewRow(Result result) {
-        // The output rows needs to be initialized each time
-        // to prevent the possibility of putting the output object into the cache.
-        GenericRowData resultRow = new GenericRowData(fieldLength);
-        GenericRowData[] familyRows = new GenericRowData[families.length];
-        for (int f = 0; f < families.length; f++) {
-            familyRows[f] = new GenericRowData(qualifiers[f].length);
-        }
-        return convertToRow(result, resultRow, familyRows);
-    }
-
-    /**
-     * Converts HBase {@link Result} into a reused {@link RowData} instance.
-     *
-     * <p>Note: this method is NOT thread-safe.
-     */
-    public RowData convertToReusedRow(Result result) {
-        return convertToRow(result, reusedRow, reusedFamilyRows);
-    }
-
-    private RowData convertToRow(
-            Result result, GenericRowData resultRow, GenericRowData[] familyRows) {
-        for (int i = 0; i < fieldLength; i++) {
-            if (rowkeyIndex == i) {
-                assert keyDecoder != null;
-                Object rowkey = keyDecoder.decode(result.getRow());
-                resultRow.setField(rowkeyIndex, rowkey);
-            } else {
-                int f = (rowkeyIndex != -1 && i > rowkeyIndex) ? i - 1 : i;
-                // get family key
-                byte[] familyKey = families[f];
-                GenericRowData familyRow = familyRows[f];
-                for (int q = 0; q < this.qualifiers[f].length; q++) {
-                    // get quantifier key
-                    byte[] qualifier = qualifiers[f][q];
-                    // read value
-                    byte[] value = result.getValue(familyKey, qualifier);
-                    familyRow.setField(q, qualifierDecoders[f][q].decode(value));
-                }
-                resultRow.setField(i, familyRow);
-            }
-        }
-        return resultRow;
-    }
-
-    /**
-     * Converts HBase {@link Result} into {@link RowData}.
-     *
-     * @deprecated Use {@link #convertToReusedRow(Result)} instead.
-     */
-    @Deprecated
-    public RowData convertToRow(Result result) {
-        for (int i = 0; i < fieldLength; i++) {
-            if (rowkeyIndex == i) {
-                assert keyDecoder != null;
-                Object rowkey = keyDecoder.decode(result.getRow());
-                reusedRow.setField(rowkeyIndex, rowkey);
-            } else {
-                int f = (rowkeyIndex != -1 && i > rowkeyIndex) ? i - 1 : i;
-                // get family key
-                byte[] familyKey = families[f];
-                GenericRowData familyRow = reusedFamilyRows[f];
-                for (int q = 0; q < this.qualifiers[f].length; q++) {
-                    // get quantifier key
-                    byte[] qualifier = qualifiers[f][q];
-                    // read value
-                    byte[] value = result.getValue(familyKey, qualifier);
-                    familyRow.setField(q, qualifierDecoders[f][q].decode(value));
-                }
-                reusedRow.setField(i, familyRow);
-            }
-        }
-        return reusedRow;
-    }
-
-    // ------------------------------------------------------------------------------------
-    // HBase Runtime Encoders
-    // ------------------------------------------------------------------------------------
-
-    /** Runtime encoder that encodes a specified field in {@link RowData} into byte[]. */
-    @FunctionalInterface
-    private interface FieldEncoder extends Serializable {
-        byte[] encode(RowData row, int pos);
-    }
-
-    private static FieldEncoder createNullableFieldEncoder(
-            LogicalType fieldType, final byte[] nullStringBytes) {
-        final FieldEncoder encoder = createFieldEncoder(fieldType);
-        if (fieldType.isNullable()) {
-            if (fieldType.is(LogicalTypeFamily.CHARACTER_STRING)) {
-                // special logic for null string values, because HBase can store empty bytes for
-                // string
-                return (row, pos) -> {
-                    if (row.isNullAt(pos)) {
-                        return nullStringBytes;
-                    } else {
-                        return encoder.encode(row, pos);
-                    }
-                };
-            } else {
-                // encode empty bytes for null values
-                return (row, pos) -> {
-                    if (row.isNullAt(pos)) {
-                        return EMPTY_BYTES;
-                    } else {
-                        return encoder.encode(row, pos);
-                    }
-                };
-            }
-        } else {
-            return encoder;
-        }
-    }
-
-    private static FieldEncoder createFieldEncoder(LogicalType fieldType) {
-        // ordered by type root definition
-        switch (fieldType.getTypeRoot()) {
-            case CHAR:
-            case VARCHAR:
-                // get the underlying UTF-8 bytes
-                return (row, pos) -> row.getString(pos).toBytes();
-            case BOOLEAN:
-                return (row, pos) -> Bytes.toBytes(row.getBoolean(pos));
-            case BINARY:
-            case VARBINARY:
-                return RowData::getBinary;
-            case DECIMAL:
-                return createDecimalEncoder((DecimalType) fieldType);
-            case TINYINT:
-                return (row, pos) -> new byte[] {row.getByte(pos)};
-            case SMALLINT:
-                return (row, pos) -> Bytes.toBytes(row.getShort(pos));
-            case INTEGER:
-            case DATE:
-            case INTERVAL_YEAR_MONTH:
-                return (row, pos) -> Bytes.toBytes(row.getInt(pos));
-            case TIME_WITHOUT_TIME_ZONE:
-                final int timePrecision = getPrecision(fieldType);
-                if (timePrecision < MIN_TIME_PRECISION || timePrecision > MAX_TIME_PRECISION) {
-                    throw new UnsupportedOperationException(
-                            String.format(
-                                    "The precision %s of TIME type is out of the range [%s, %s] supported by "
-                                            + "HBase connector",
-                                    timePrecision, MIN_TIME_PRECISION, MAX_TIME_PRECISION));
-                }
-                return (row, pos) -> Bytes.toBytes(row.getInt(pos));
-            case BIGINT:
-            case INTERVAL_DAY_TIME:
-                return (row, pos) -> Bytes.toBytes(row.getLong(pos));
-            case FLOAT:
-                return (row, pos) -> Bytes.toBytes(row.getFloat(pos));
-            case DOUBLE:
-                return (row, pos) -> Bytes.toBytes(row.getDouble(pos));
-            case TIMESTAMP_WITHOUT_TIME_ZONE:
-            case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
-                final int timestampPrecision = getPrecision(fieldType);
-                if (timestampPrecision < MIN_TIMESTAMP_PRECISION
-                        || timestampPrecision > MAX_TIMESTAMP_PRECISION) {
-                    throw new UnsupportedOperationException(
-                            String.format(
-                                    "The precision %s of TIMESTAMP type is out of the range [%s, %s] supported by "
-                                            + "HBase connector",
-                                    timestampPrecision,
-                                    MIN_TIMESTAMP_PRECISION,
-                                    MAX_TIMESTAMP_PRECISION));
-                }
-                return createTimestampEncoder(timestampPrecision);
-            default:
-                throw new UnsupportedOperationException("Unsupported type: " + fieldType);
-        }
-    }
-
-    private static FieldEncoder createDecimalEncoder(DecimalType decimalType) {
-        final int precision = decimalType.getPrecision();
-        final int scale = decimalType.getScale();
-        return (row, pos) -> {
-            BigDecimal decimal = row.getDecimal(pos, precision, scale).toBigDecimal();
-            return Bytes.toBytes(decimal);
-        };
-    }
-
-    private static FieldEncoder createTimestampEncoder(final int precision) {
-        return (row, pos) -> {
-            long millisecond = row.getTimestamp(pos, precision).getMillisecond();
-            return Bytes.toBytes(millisecond);
-        };
-    }
-
-    // ------------------------------------------------------------------------------------
-    // HBase Runtime Decoders
-    // ------------------------------------------------------------------------------------
-
-    /** Runtime decoder that decodes a byte[] into objects of internal data structure. */
-    @FunctionalInterface
-    private interface FieldDecoder extends Serializable {
-        @Nullable
-        Object decode(byte[] value);
-    }
-
-    private static FieldDecoder createNullableFieldDecoder(
-            LogicalType fieldType, final byte[] nullStringBytes) {
-        final FieldDecoder decoder = createFieldDecoder(fieldType);
-        if (fieldType.isNullable()) {
-            if (fieldType.is(LogicalTypeFamily.CHARACTER_STRING)) {
-                return value -> {
-                    if (value == null || Arrays.equals(value, nullStringBytes)) {
-                        return null;
-                    } else {
-                        return decoder.decode(value);
-                    }
-                };
-            } else {
-                return value -> {
-                    if (value == null || value.length == 0) {
-                        return null;
-                    } else {
-                        return decoder.decode(value);
-                    }
-                };
-            }
-        } else {
-            return decoder;
-        }
-    }
-
-    private static FieldDecoder createFieldDecoder(LogicalType fieldType) {
-        // ordered by type root definition
-        switch (fieldType.getTypeRoot()) {
-            case CHAR:
-            case VARCHAR:
-                // reuse bytes
-                return StringData::fromBytes;
-            case BOOLEAN:
-                return Bytes::toBoolean;
-            case BINARY:
-            case VARBINARY:
-                return value -> value;
-            case DECIMAL:
-                return createDecimalDecoder((DecimalType) fieldType);
-            case TINYINT:
-                return value -> value[0];
-            case SMALLINT:
-                return Bytes::toShort;
-            case INTEGER:
-            case DATE:
-            case INTERVAL_YEAR_MONTH:
-                return Bytes::toInt;
-            case TIME_WITHOUT_TIME_ZONE:
-                final int timePrecision = getPrecision(fieldType);
-                if (timePrecision < MIN_TIME_PRECISION || timePrecision > MAX_TIME_PRECISION) {
-                    throw new UnsupportedOperationException(
-                            String.format(
-                                    "The precision %s of TIME type is out of the range [%s, %s] supported by "
-                                            + "HBase connector",
-                                    timePrecision, MIN_TIME_PRECISION, MAX_TIME_PRECISION));
-                }
-                return Bytes::toInt;
-            case BIGINT:
-            case INTERVAL_DAY_TIME:
-                return Bytes::toLong;
-            case FLOAT:
-                return Bytes::toFloat;
-            case DOUBLE:
-                return Bytes::toDouble;
-            case TIMESTAMP_WITHOUT_TIME_ZONE:
-            case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
-                final int timestampPrecision = getPrecision(fieldType);
-                if (timestampPrecision < MIN_TIMESTAMP_PRECISION
-                        || timestampPrecision > MAX_TIMESTAMP_PRECISION) {
-                    throw new UnsupportedOperationException(
-                            String.format(
-                                    "The precision %s of TIMESTAMP type is out of the range [%s, %s] supported by "
-                                            + "HBase connector",
-                                    timestampPrecision,
-                                    MIN_TIMESTAMP_PRECISION,
-                                    MAX_TIMESTAMP_PRECISION));
-                }
-                return createTimestampDecoder();
-            default:
-                throw new UnsupportedOperationException("Unsupported type: " + fieldType);
-        }
-    }
-
-    private static FieldDecoder createDecimalDecoder(DecimalType decimalType) {
-        final int precision = decimalType.getPrecision();
-        final int scale = decimalType.getScale();
-        return value -> {
-            BigDecimal decimal = Bytes.toBigDecimal(value);
-            return DecimalData.fromBigDecimal(decimal, precision, scale);
-        };
-    }
-
-    private static FieldDecoder createTimestampDecoder() {
-        return value -> {
-            // TODO: support higher precision
-            long milliseconds = Bytes.toLong(value);
-            return TimestampData.fromEpochMillis(milliseconds);
-        };
-    }
-}
diff --git a/flink-connectors/flink-connector-hbase-base/src/main/java/org/apache/flink/connector/hbase/util/HBaseStronglyIncreasingTsGenerator.java b/flink-connectors/flink-connector-hbase-base/src/main/java/org/apache/flink/connector/hbase/util/HBaseStronglyIncreasingTsGenerator.java
deleted file mode 100644
index 3859738..0000000
--- a/flink-connectors/flink-connector-hbase-base/src/main/java/org/apache/flink/connector/hbase/util/HBaseStronglyIncreasingTsGenerator.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * 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.util;
-
-import org.apache.flink.annotation.VisibleForTesting;
-
-/** Generate strongly increasing timestamp in nanosecond for HBase mutation. */
-public class HBaseStronglyIncreasingTsGenerator {
-    private static final long START_SYSTEM_TIME_NANO = System.currentTimeMillis() * 1_000_000L;
-    private static final long START_JVM_TIME_NANO = System.nanoTime();
-
-    private long currentSystemTimeNano;
-
-    public HBaseStronglyIncreasingTsGenerator() {
-        currentSystemTimeNano = getCurrentSystemTimeNano();
-    }
-
-    public long get() {
-        long nowNano = getCurrentSystemTimeNano();
-        if (nowNano <= currentSystemTimeNano) {
-            nowNano = currentSystemTimeNano + 1;
-        }
-        currentSystemTimeNano = nowNano;
-        return nowNano;
-    }
-
-    @VisibleForTesting
-    protected long getCurrentSystemTimeNano() {
-        return START_SYSTEM_TIME_NANO + (System.nanoTime() - START_JVM_TIME_NANO);
-    }
-}
diff --git a/flink-connectors/flink-connector-hbase-base/src/main/java/org/apache/flink/connector/hbase/util/HBaseTableSchema.java b/flink-connectors/flink-connector-hbase-base/src/main/java/org/apache/flink/connector/hbase/util/HBaseTableSchema.java
deleted file mode 100644
index db56766..0000000
--- a/flink-connectors/flink-connector-hbase-base/src/main/java/org/apache/flink/connector/hbase/util/HBaseTableSchema.java
+++ /dev/null
@@ -1,394 +0,0 @@
-/*
- * 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.util;
-
-import org.apache.flink.annotation.Internal;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.java.typeutils.TypeExtractor;
-import org.apache.flink.table.api.DataTypes;
-import org.apache.flink.table.types.DataType;
-import org.apache.flink.table.types.logical.LogicalType;
-import org.apache.flink.table.types.logical.LogicalTypeRoot;
-import org.apache.flink.table.types.logical.RowType;
-import org.apache.flink.table.types.utils.TypeConversions;
-import org.apache.flink.util.Preconditions;
-
-import org.apache.flink.shaded.guava31.com.google.common.collect.Streams;
-
-import java.io.Serializable;
-import java.nio.charset.Charset;
-import java.util.Arrays;
-import java.util.LinkedHashMap;
-import java.util.Map;
-import java.util.Objects;
-import java.util.Optional;
-
-import static org.apache.flink.table.types.utils.TypeConversions.fromLogicalToDataType;
-
-/** Helps to specify an HBase Table's schema. */
-@Internal
-public class HBaseTableSchema implements Serializable {
-
-    private static final long serialVersionUID = 1L;
-
-    // A Map with key as column family.
-    private final Map<String, Map<String, DataType>> familyMap = new LinkedHashMap<>();
-
-    // information about rowkey
-    private RowKeyInfo rowKeyInfo;
-
-    // charset to parse HBase keys and strings. UTF-8 by default.
-    private String charset = "UTF-8";
-
-    /**
-     * Adds a column defined by family, qualifier, and type to the table schema.
-     *
-     * @param family the family name
-     * @param qualifier the qualifier name
-     * @param clazz the data type of the qualifier
-     */
-    public void addColumn(String family, String qualifier, Class<?> clazz) {
-        Preconditions.checkNotNull(clazz, "class type");
-        DataType type = TypeConversions.fromLegacyInfoToDataType(TypeExtractor.getForClass(clazz));
-        addColumn(family, qualifier, type);
-    }
-
-    public void addColumn(String family, String qualifier, DataType type) {
-        Preconditions.checkNotNull(family, "family name");
-        Preconditions.checkNotNull(qualifier, "qualifier name");
-        Preconditions.checkNotNull(type, "data type");
-        Map<String, DataType> qualifierMap = this.familyMap.get(family);
-
-        if (!HBaseTypeUtils.isSupportedType(type.getLogicalType())) {
-            // throw exception
-            throw new IllegalArgumentException(
-                    "Unsupported class type found "
-                            + type
-                            + ". "
-                            + "Better to use byte[].class and deserialize using user defined scalar functions");
-        }
-
-        if (qualifierMap == null) {
-            qualifierMap = new LinkedHashMap<>();
-        }
-        qualifierMap.put(qualifier, type);
-        familyMap.put(family, qualifierMap);
-    }
-
-    /**
-     * Sets row key information in the table schema.
-     *
-     * @param rowKeyName the row key field name
-     * @param clazz the data type of the row key
-     */
-    public void setRowKey(String rowKeyName, Class<?> clazz) {
-        Preconditions.checkNotNull(clazz, "row key class type");
-        DataType type = TypeConversions.fromLegacyInfoToDataType(TypeExtractor.getForClass(clazz));
-        setRowKey(rowKeyName, type);
-    }
-
-    public void setRowKey(String rowKeyName, DataType type) {
-        Preconditions.checkNotNull(rowKeyName, "row key field name");
-        Preconditions.checkNotNull(type, "row key data type");
-        if (!HBaseTypeUtils.isSupportedType(type.getLogicalType())) {
-            // throw exception
-            throw new IllegalArgumentException(
-                    "Unsupported class type found "
-                            + type
-                            + ". "
-                            + "Better to use byte[].class and deserialize using user defined scalar functions");
-        }
-        if (rowKeyInfo != null) {
-            throw new IllegalArgumentException("Row key can't be set multiple times.");
-        }
-        this.rowKeyInfo = new RowKeyInfo(rowKeyName, type, familyMap.size());
-    }
-
-    /**
-     * Sets the charset for value strings and HBase identifiers.
-     *
-     * @param charset the charset for value strings and HBase identifiers.
-     */
-    public void setCharset(String charset) {
-        this.charset = charset;
-    }
-
-    /**
-     * Returns the names of all registered column families.
-     *
-     * @return The names of all registered column families.
-     */
-    public String[] getFamilyNames() {
-        return this.familyMap.keySet().toArray(new String[0]);
-    }
-
-    /**
-     * Returns the HBase identifiers of all registered column families.
-     *
-     * @return The HBase identifiers of all registered column families.
-     */
-    public byte[][] getFamilyKeys() {
-        Charset c = Charset.forName(charset);
-
-        byte[][] familyKeys = new byte[this.familyMap.size()][];
-        int i = 0;
-        for (String name : this.familyMap.keySet()) {
-            familyKeys[i++] = name.getBytes(c);
-        }
-        return familyKeys;
-    }
-
-    /**
-     * Returns the names of all registered column qualifiers of a specific column family.
-     *
-     * @param family The name of the column family for which the column qualifier names are
-     *     returned.
-     * @return The names of all registered column qualifiers of a specific column family.
-     */
-    public String[] getQualifierNames(String family) {
-        Map<String, DataType> qualifierMap = familyMap.get(family);
-
-        if (qualifierMap == null) {
-            throw new IllegalArgumentException("Family " + family + " does not exist in schema.");
-        }
-
-        String[] qualifierNames = new String[qualifierMap.size()];
-        int i = 0;
-        for (String qualifier : qualifierMap.keySet()) {
-            qualifierNames[i] = qualifier;
-            i++;
-        }
-        return qualifierNames;
-    }
-
-    /**
-     * Returns the HBase identifiers of all registered column qualifiers for a specific column
-     * family.
-     *
-     * @param family The name of the column family for which the column qualifier identifiers are
-     *     returned.
-     * @return The HBase identifiers of all registered column qualifiers for a specific column
-     *     family.
-     */
-    public byte[][] getQualifierKeys(String family) {
-        Map<String, DataType> qualifierMap = familyMap.get(family);
-
-        if (qualifierMap == null) {
-            throw new IllegalArgumentException("Family " + family + " does not exist in schema.");
-        }
-        Charset c = Charset.forName(charset);
-
-        byte[][] qualifierKeys = new byte[qualifierMap.size()][];
-        int i = 0;
-        for (String name : qualifierMap.keySet()) {
-            qualifierKeys[i++] = name.getBytes(c);
-        }
-        return qualifierKeys;
-    }
-
-    public DataType[] getQualifierDataTypes(String family) {
-        Map<String, DataType> qualifierMap = familyMap.get(family);
-
-        if (qualifierMap == null) {
-            throw new IllegalArgumentException("Family " + family + " does not exist in schema.");
-        }
-
-        DataType[] dataTypes = new DataType[qualifierMap.size()];
-        int i = 0;
-        for (DataType dataType : qualifierMap.values()) {
-            dataTypes[i] = dataType;
-            i++;
-        }
-        return dataTypes;
-    }
-
-    /**
-     * Returns the names and types of all registered column qualifiers of a specific column family.
-     *
-     * @param family The name of the column family for which the column qualifier names and types
-     *     are returned.
-     * @return The names and types of all registered column qualifiers of a specific column family.
-     */
-    private Map<String, DataType> getFamilyInfo(String family) {
-        return familyMap.get(family);
-    }
-
-    /** Returns field index of row key in the table schema. Returns -1 if row key is not set. */
-    public int getRowKeyIndex() {
-        return rowKeyInfo == null ? -1 : rowKeyInfo.rowKeyIndex;
-    }
-
-    /** Returns the optional type information of row key. Returns null if row key is not set. */
-    public Optional<TypeInformation<?>> getRowKeyTypeInfo() {
-        return rowKeyInfo == null
-                ? Optional.empty()
-                : Optional.of(TypeConversions.fromDataTypeToLegacyInfo(rowKeyInfo.rowKeyType));
-    }
-
-    public Optional<DataType> getRowKeyDataType() {
-        return rowKeyInfo == null ? Optional.empty() : Optional.of(rowKeyInfo.rowKeyType);
-    }
-
-    /**
-     * Returns optional value of row key name. The row key name is the field name in hbase schema
-     * which can be queried in Flink SQL.
-     */
-    public Optional<String> getRowKeyName() {
-        return rowKeyInfo == null ? Optional.empty() : Optional.of(rowKeyInfo.rowKeyName);
-    }
-
-    /**
-     * Converts this {@link HBaseTableSchema} to {@link DataType}, the fields are consisted of
-     * families and rowkey, the order is in the definition order (i.e. calling {@link
-     * #addColumn(String, String, Class)} and {@link #setRowKey(String, Class)}). The family field
-     * is a composite type which is consisted of qualifiers.
-     *
-     * @return the {@link DataType} derived from the {@link HBaseTableSchema}.
-     */
-    public DataType convertToDataType() {
-        String[] familyNames = getFamilyNames();
-        if (rowKeyInfo != null) {
-            String[] fieldNames = new String[familyNames.length + 1];
-            DataType[] fieldTypes = new DataType[familyNames.length + 1];
-            for (int i = 0; i < fieldNames.length; i++) {
-                if (i == rowKeyInfo.rowKeyIndex) {
-                    fieldNames[i] = rowKeyInfo.rowKeyName;
-                    fieldTypes[i] = rowKeyInfo.rowKeyType;
-                } else {
-                    int familyIndex = i < rowKeyInfo.rowKeyIndex ? i : i - 1;
-                    String family = familyNames[familyIndex];
-                    fieldNames[i] = family;
-                    fieldTypes[i] =
-                            getRowDataType(
-                                    getQualifierNames(family), getQualifierDataTypes(family));
-                }
-            }
-            return DataTypes.ROW(
-                    Streams.zip(
-                                    Arrays.stream(fieldNames),
-                                    Arrays.stream(fieldTypes),
-                                    DataTypes::FIELD)
-                            .toArray(DataTypes.Field[]::new));
-        } else {
-            String[] fieldNames = new String[familyNames.length];
-            DataType[] fieldTypes = new DataType[familyNames.length];
-            for (int i = 0; i < fieldNames.length; i++) {
-                String family = familyNames[i];
-                fieldNames[i] = family;
-                fieldTypes[i] =
-                        getRowDataType(getQualifierNames(family), getQualifierDataTypes(family));
-            }
-            return DataTypes.ROW(
-                    Streams.zip(
-                                    Arrays.stream(fieldNames),
-                                    Arrays.stream(fieldTypes),
-                                    DataTypes::FIELD)
-                            .toArray(DataTypes.Field[]::new));
-        }
-    }
-
-    /**
-     * Returns row data type with given field names {@code fieldNames} and data types {@code
-     * fieldTypes}.
-     *
-     * @param fieldNames the field names
-     * @param fieldTypes the field types
-     * @return nullable row type
-     */
-    private static DataType getRowDataType(String[] fieldNames, DataType[] fieldTypes) {
-        final DataTypes.Field[] fields = new DataTypes.Field[fieldNames.length];
-        for (int j = 0; j < fieldNames.length; j++) {
-            fields[j] = DataTypes.FIELD(fieldNames[j], fieldTypes[j]);
-        }
-        return DataTypes.ROW(fields);
-    }
-
-    /** Construct a {@link HBaseTableSchema} from a {@link DataType}. */
-    public static HBaseTableSchema fromDataType(DataType physicalRowType) {
-        HBaseTableSchema hbaseSchema = new HBaseTableSchema();
-        RowType rowType = (RowType) physicalRowType.getLogicalType();
-        for (RowType.RowField field : rowType.getFields()) {
-            LogicalType fieldType = field.getType();
-            if (fieldType.getTypeRoot() == LogicalTypeRoot.ROW) {
-                RowType familyType = (RowType) fieldType;
-                String familyName = field.getName();
-                for (RowType.RowField qualifier : familyType.getFields()) {
-                    hbaseSchema.addColumn(
-                            familyName,
-                            qualifier.getName(),
-                            fromLogicalToDataType(qualifier.getType()));
-                }
-            } else if (fieldType.getChildren().size() == 0) {
-                hbaseSchema.setRowKey(field.getName(), fromLogicalToDataType(fieldType));
-            } else {
-                throw new IllegalArgumentException(
-                        "Unsupported field type '" + fieldType + "' for HBase.");
-            }
-        }
-        return hbaseSchema;
-    }
-
-    // ------------------------------------------------------------------------------------
-
-    /** A class containing information about rowKey, such as rowKeyName, rowKeyType, rowKeyIndex. */
-    private static class RowKeyInfo implements Serializable {
-        private static final long serialVersionUID = 1L;
-        final String rowKeyName;
-        final DataType rowKeyType;
-        final int rowKeyIndex;
-
-        RowKeyInfo(String rowKeyName, DataType rowKeyType, int rowKeyIndex) {
-            this.rowKeyName = rowKeyName;
-            this.rowKeyType = rowKeyType;
-            this.rowKeyIndex = rowKeyIndex;
-        }
-
-        @Override
-        public boolean equals(Object o) {
-            if (!(o instanceof RowKeyInfo)) {
-                return false;
-            }
-            RowKeyInfo that = (RowKeyInfo) o;
-            return Objects.equals(rowKeyName, that.rowKeyName)
-                    && Objects.equals(rowKeyType, that.rowKeyType)
-                    && Objects.equals(rowKeyIndex, that.rowKeyIndex);
-        }
-
-        @Override
-        public int hashCode() {
-            return Objects.hash(rowKeyName, rowKeyType, rowKeyIndex);
-        }
-    }
-
-    @Override
-    public boolean equals(Object o) {
-        if (!(o instanceof HBaseTableSchema)) {
-            return false;
-        }
-        HBaseTableSchema that = (HBaseTableSchema) o;
-        return Objects.equals(familyMap, that.familyMap)
-                && Objects.equals(rowKeyInfo, that.rowKeyInfo)
-                && Objects.equals(charset, that.charset);
-    }
-
-    @Override
-    public int hashCode() {
-        return Objects.hash(familyMap, rowKeyInfo, charset);
-    }
-}
diff --git a/flink-connectors/flink-connector-hbase-base/src/main/java/org/apache/flink/connector/hbase/util/HBaseTypeUtils.java b/flink-connectors/flink-connector-hbase-base/src/main/java/org/apache/flink/connector/hbase/util/HBaseTypeUtils.java
deleted file mode 100644
index 9dcb2e8..0000000
--- a/flink-connectors/flink-connector-hbase-base/src/main/java/org/apache/flink/connector/hbase/util/HBaseTypeUtils.java
+++ /dev/null
@@ -1,229 +0,0 @@
-/*
- * 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.util;
-
-import org.apache.flink.annotation.Internal;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.table.types.logical.LogicalType;
-
-import org.apache.hadoop.hbase.util.Bytes;
-
-import java.math.BigDecimal;
-import java.math.BigInteger;
-import java.nio.charset.Charset;
-import java.sql.Date;
-import java.sql.Time;
-import java.sql.Timestamp;
-import java.util.Arrays;
-
-import static org.apache.flink.table.types.logical.utils.LogicalTypeChecks.getPrecision;
-
-/** A utility class to process data exchange with HBase type system. */
-@Internal
-public class HBaseTypeUtils {
-
-    private static final byte[] EMPTY_BYTES = new byte[] {};
-
-    private static final int MIN_TIMESTAMP_PRECISION = 0;
-    private static final int MAX_TIMESTAMP_PRECISION = 3;
-    private static final int MIN_TIME_PRECISION = 0;
-    private static final int MAX_TIME_PRECISION = 3;
-
-    /** Deserialize byte array to Java Object with the given type. */
-    public static Object deserializeToObject(byte[] value, int typeIdx, Charset stringCharset) {
-        switch (typeIdx) {
-            case 0: // byte[]
-                return value;
-            case 1: // String
-                return Arrays.equals(EMPTY_BYTES, value) ? null : new String(value, stringCharset);
-            case 2: // byte
-                return value[0];
-            case 3:
-                return Bytes.toShort(value);
-            case 4:
-                return Bytes.toInt(value);
-            case 5:
-                return Bytes.toLong(value);
-            case 6:
-                return Bytes.toFloat(value);
-            case 7:
-                return Bytes.toDouble(value);
-            case 8:
-                return Bytes.toBoolean(value);
-            case 9: // sql.Timestamp encoded as long
-                return new Timestamp(Bytes.toLong(value));
-            case 10: // sql.Date encoded as long
-                return new Date(Bytes.toLong(value));
-            case 11: // sql.Time encoded as long
-                return new Time(Bytes.toLong(value));
-            case 12:
-                return Bytes.toBigDecimal(value);
-            case 13:
-                return new BigInteger(value);
-
-            default:
-                throw new IllegalArgumentException("unsupported type index:" + typeIdx);
-        }
-    }
-
-    /** Serialize the Java Object to byte array with the given type. */
-    public static byte[] serializeFromObject(Object value, int typeIdx, Charset stringCharset) {
-        switch (typeIdx) {
-            case 0: // byte[]
-                return (byte[]) value;
-            case 1: // external String
-                return value == null ? EMPTY_BYTES : ((String) value).getBytes(stringCharset);
-            case 2: // byte
-                return value == null ? EMPTY_BYTES : new byte[] {(byte) value};
-            case 3:
-                return Bytes.toBytes((short) value);
-            case 4:
-                return Bytes.toBytes((int) value);
-            case 5:
-                return Bytes.toBytes((long) value);
-            case 6:
-                return Bytes.toBytes((float) value);
-            case 7:
-                return Bytes.toBytes((double) value);
-            case 8:
-                return Bytes.toBytes((boolean) value);
-            case 9: // sql.Timestamp encoded to Long
-                return Bytes.toBytes(((Timestamp) value).getTime());
-            case 10: // sql.Date encoded as long
-                return Bytes.toBytes(((Date) value).getTime());
-            case 11: // sql.Time encoded as long
-                return Bytes.toBytes(((Time) value).getTime());
-            case 12:
-                return Bytes.toBytes((BigDecimal) value);
-            case 13:
-                return ((BigInteger) value).toByteArray();
-
-            default:
-                throw new IllegalArgumentException("unsupported type index:" + typeIdx);
-        }
-    }
-
-    /**
-     * Gets the type index (type representation in HBase connector) from the {@link
-     * TypeInformation}.
-     */
-    public static int getTypeIndex(TypeInformation typeInfo) {
-        return getTypeIndex(typeInfo.getTypeClass());
-    }
-
-    /** Checks whether the given Class is a supported type in HBase connector. */
-    public static boolean isSupportedType(Class<?> clazz) {
-        return getTypeIndex(clazz) != -1;
-    }
-
-    private static int getTypeIndex(Class<?> clazz) {
-        if (byte[].class.equals(clazz)) {
-            return 0;
-        } else if (String.class.equals(clazz)) {
-            return 1;
-        } else if (Byte.class.equals(clazz)) {
-            return 2;
-        } else if (Short.class.equals(clazz)) {
-            return 3;
-        } else if (Integer.class.equals(clazz)) {
-            return 4;
-        } else if (Long.class.equals(clazz)) {
-            return 5;
-        } else if (Float.class.equals(clazz)) {
-            return 6;
-        } else if (Double.class.equals(clazz)) {
-            return 7;
-        } else if (Boolean.class.equals(clazz)) {
-            return 8;
-        } else if (Timestamp.class.equals(clazz)) {
-            return 9;
-        } else if (Date.class.equals(clazz)) {
-            return 10;
-        } else if (Time.class.equals(clazz)) {
-            return 11;
-        } else if (BigDecimal.class.equals(clazz)) {
-            return 12;
-        } else if (BigInteger.class.equals(clazz)) {
-            return 13;
-        } else {
-            return -1;
-        }
-    }
-
-    /** Checks whether the given {@link LogicalType} is supported in HBase connector. */
-    public static boolean isSupportedType(LogicalType type) {
-        // ordered by type root definition
-        switch (type.getTypeRoot()) {
-            case CHAR:
-            case VARCHAR:
-            case BOOLEAN:
-            case BINARY:
-            case VARBINARY:
-            case DECIMAL:
-            case TINYINT:
-            case SMALLINT:
-            case INTEGER:
-            case DATE:
-            case INTERVAL_YEAR_MONTH:
-            case BIGINT:
-            case INTERVAL_DAY_TIME:
-            case FLOAT:
-            case DOUBLE:
-                return true;
-            case TIME_WITHOUT_TIME_ZONE:
-                final int timePrecision = getPrecision(type);
-                if (timePrecision < MIN_TIME_PRECISION || timePrecision > MAX_TIME_PRECISION) {
-                    throw new UnsupportedOperationException(
-                            String.format(
-                                    "The precision %s of TIME type is out of the range [%s, %s] supported by "
-                                            + "HBase connector",
-                                    timePrecision, MIN_TIME_PRECISION, MAX_TIME_PRECISION));
-                }
-                return true;
-            case TIMESTAMP_WITHOUT_TIME_ZONE:
-            case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
-                final int timestampPrecision = getPrecision(type);
-                if (timestampPrecision < MIN_TIMESTAMP_PRECISION
-                        || timestampPrecision > MAX_TIMESTAMP_PRECISION) {
-                    throw new UnsupportedOperationException(
-                            String.format(
-                                    "The precision %s of TIMESTAMP type is out of the range [%s, %s] supported by "
-                                            + "HBase connector",
-                                    timestampPrecision,
-                                    MIN_TIMESTAMP_PRECISION,
-                                    MAX_TIMESTAMP_PRECISION));
-                }
-                return true;
-            case TIMESTAMP_WITH_TIME_ZONE:
-            case ARRAY:
-            case MULTISET:
-            case MAP:
-            case ROW:
-            case STRUCTURED_TYPE:
-            case DISTINCT_TYPE:
-            case RAW:
-            case NULL:
-            case SYMBOL:
-            case UNRESOLVED:
-                return false;
-            default:
-                throw new IllegalArgumentException();
-        }
-    }
-}
diff --git a/flink-connectors/flink-connector-hbase-base/src/test/java/org/apache/flink/architecture/TestCodeArchitectureTest.java b/flink-connectors/flink-connector-hbase-base/src/test/java/org/apache/flink/architecture/TestCodeArchitectureTest.java
deleted file mode 100644
index 795786a..0000000
--- a/flink-connectors/flink-connector-hbase-base/src/test/java/org/apache/flink/architecture/TestCodeArchitectureTest.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/*
- * 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.architecture;
-
-import org.apache.flink.architecture.common.ImportOptions;
-
-import com.tngtech.archunit.core.importer.ImportOption;
-import com.tngtech.archunit.junit.AnalyzeClasses;
-import com.tngtech.archunit.junit.ArchTest;
-import com.tngtech.archunit.junit.ArchTests;
-
-/** Architecture tests for test code. */
-@AnalyzeClasses(
-        packages = "org.apache.flink.connector.hbase.util",
-        importOptions = {
-            ImportOption.OnlyIncludeTests.class,
-            ImportOptions.ExcludeScalaImportOption.class,
-            ImportOptions.ExcludeShadedImportOption.class
-        })
-public class TestCodeArchitectureTest {
-
-    @ArchTest
-    public static final ArchTests COMMON_TESTS = ArchTests.in(TestCodeArchitectureTestBase.class);
-}
diff --git a/flink-connectors/flink-connector-hbase-base/src/test/java/org/apache/flink/connector/hbase/util/HBaseConfigLoadingTest.java b/flink-connectors/flink-connector-hbase-base/src/test/java/org/apache/flink/connector/hbase/util/HBaseConfigLoadingTest.java
deleted file mode 100644
index 7b64ac7..0000000
--- a/flink-connectors/flink-connector-hbase-base/src/test/java/org/apache/flink/connector/hbase/util/HBaseConfigLoadingTest.java
+++ /dev/null
@@ -1,174 +0,0 @@
-/*
- * 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.util;
-
-import org.apache.flink.core.testutils.CommonTestUtils;
-
-import org.junit.jupiter.api.Test;
-import org.junit.jupiter.api.io.TempDir;
-
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.io.PrintStream;
-import java.nio.file.Files;
-import java.nio.file.Path;
-import java.util.HashMap;
-import java.util.Map;
-
-import static org.assertj.core.api.Assertions.assertThat;
-
-/**
- * Tests that validate the loading of the HBase configuration, relative to entries in the Flink
- * configuration and the environment variables.
- */
-class HBaseConfigLoadingTest {
-
-    private static final String IN_HBASE_CONFIG_KEY = "hbase_conf_key";
-    private static final String IN_HBASE_CONFIG_VALUE = "hbase_conf_value!";
-
-    @TempDir Path tmpDir;
-
-    @Test
-    void loadFromClasspathByDefault() {
-        org.apache.hadoop.conf.Configuration hbaseConf =
-                HBaseConfigurationUtil.getHBaseConfiguration();
-
-        assertThat(hbaseConf.get(IN_HBASE_CONFIG_KEY, null)).isEqualTo(IN_HBASE_CONFIG_VALUE);
-    }
-
-    @Test
-    void loadFromEnvVariables() throws Exception {
-        final String k1 = "where?";
-        final String v1 = "I'm on a boat";
-        final String k2 = "when?";
-        final String v2 = "midnight";
-        final String k3 = "why?";
-        final String v3 = "what do you think?";
-        final String k4 = "which way?";
-        final String v4 = "south, always south...";
-
-        final File hbaseConfDir = tmpDir.toFile();
-
-        final File hbaseHome = Files.createTempDirectory(tmpDir, "junit_hbaseHome_").toFile();
-
-        final File hbaseHomeConf = new File(hbaseHome, "conf");
-
-        assertThat(hbaseHomeConf.mkdirs()).isTrue();
-
-        final File file1 = new File(hbaseConfDir, "hbase-default.xml");
-        final File file2 = new File(hbaseConfDir, "hbase-site.xml");
-        final File file3 = new File(hbaseHomeConf, "hbase-default.xml");
-        final File file4 = new File(hbaseHomeConf, "hbase-site.xml");
-
-        printConfig(file1, k1, v1);
-        printConfig(file2, k2, v2);
-        printConfig(file3, k3, v3);
-        printConfig(file4, k4, v4);
-
-        final org.apache.hadoop.conf.Configuration hbaseConf;
-
-        final Map<String, String> originalEnv = System.getenv();
-        final Map<String, String> newEnv = new HashMap<>(originalEnv);
-        newEnv.put("HBASE_CONF_DIR", hbaseConfDir.getAbsolutePath());
-        newEnv.put("HBASE_HOME", hbaseHome.getAbsolutePath());
-        try {
-            CommonTestUtils.setEnv(newEnv);
-            hbaseConf = HBaseConfigurationUtil.getHBaseConfiguration();
-        } finally {
-            CommonTestUtils.setEnv(originalEnv);
-        }
-
-        // contains extra entries
-        assertThat(hbaseConf.get(k1, null)).isEqualTo(v1);
-        assertThat(hbaseConf.get(k2, null)).isEqualTo(v2);
-        assertThat(hbaseConf.get(k3, null)).isEqualTo(v3);
-        assertThat(hbaseConf.get(k4, null)).isEqualTo(v4);
-
-        // also contains classpath defaults
-        assertThat(hbaseConf.get(IN_HBASE_CONFIG_KEY, null)).isEqualTo(IN_HBASE_CONFIG_VALUE);
-    }
-
-    @Test
-    void loadOverlappingConfig() throws Exception {
-        final String k1 = "key1";
-
-        final String v1 = "from HBASE_HOME/conf";
-        final String v2 = "from HBASE_CONF_DIR";
-
-        final File hbaseHome = tmpDir.resolve("hbaseHome").toFile();
-        final File hbaseHomeConf = new File(hbaseHome, "conf");
-
-        final File hbaseConfDir = tmpDir.resolve("hbaseConfDir").toFile();
-
-        assertThat(hbaseHomeConf.mkdirs()).isTrue();
-        final File file1 = new File(hbaseHomeConf, "hbase-site.xml");
-
-        Map<String, String> properties1 = new HashMap<>();
-        properties1.put(k1, v1);
-        printConfigs(file1, properties1);
-
-        // HBASE_CONF_DIR conf will override k1 with v2
-        assertThat(hbaseConfDir.mkdirs()).isTrue();
-        final File file2 = new File(hbaseConfDir, "hbase-site.xml");
-        Map<String, String> properties2 = new HashMap<>();
-        properties2.put(k1, v2);
-        printConfigs(file2, properties2);
-
-        final org.apache.hadoop.conf.Configuration hbaseConf;
-
-        final Map<String, String> originalEnv = System.getenv();
-        final Map<String, String> newEnv = new HashMap<>(originalEnv);
-        newEnv.put("HBASE_CONF_DIR", hbaseConfDir.getAbsolutePath());
-        newEnv.put("HBASE_HOME", hbaseHome.getAbsolutePath());
-        try {
-            CommonTestUtils.setEnv(newEnv);
-            hbaseConf = HBaseConfigurationUtil.getHBaseConfiguration();
-        } finally {
-            CommonTestUtils.setEnv(originalEnv);
-        }
-
-        // contains extra entries
-        assertThat(hbaseConf.get(k1, null)).isEqualTo(v2);
-
-        // also contains classpath defaults
-        assertThat(hbaseConf.get(IN_HBASE_CONFIG_KEY, null)).isEqualTo(IN_HBASE_CONFIG_VALUE);
-    }
-
-    private static void printConfig(File file, String key, String value) throws IOException {
-        Map<String, String> map = new HashMap<>(1);
-        map.put(key, value);
-        printConfigs(file, map);
-    }
-
-    private static void printConfigs(File file, Map<String, String> properties) throws IOException {
-        try (PrintStream out = new PrintStream(new FileOutputStream(file))) {
-            out.println("<?xml version=\"1.0\"?>");
-            out.println("<?xml-stylesheet type=\"text/xsl\" href=\"configuration.xsl\"?>");
-            out.println("<configuration>");
-            for (Map.Entry<String, String> entry : properties.entrySet()) {
-                out.println("\t<property>");
-                out.println("\t\t<name>" + entry.getKey() + "</name>");
-                out.println("\t\t<value>" + entry.getValue() + "</value>");
-                out.println("\t</property>");
-            }
-            out.println("</configuration>");
-        }
-    }
-}
diff --git a/flink-connectors/flink-connector-hbase-base/src/test/java/org/apache/flink/connector/hbase/util/HBaseSerdeTest.java b/flink-connectors/flink-connector-hbase-base/src/test/java/org/apache/flink/connector/hbase/util/HBaseSerdeTest.java
deleted file mode 100644
index 3e9afb1..0000000
--- a/flink-connectors/flink-connector-hbase-base/src/test/java/org/apache/flink/connector/hbase/util/HBaseSerdeTest.java
+++ /dev/null
@@ -1,166 +0,0 @@
-/*
- * 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.util;
-
-import org.apache.flink.table.api.DataTypes;
-import org.apache.flink.table.data.RowData;
-import org.apache.flink.table.types.DataType;
-
-import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.junit.jupiter.api.Test;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import static org.apache.flink.table.api.DataTypes.BIGINT;
-import static org.apache.flink.table.api.DataTypes.DOUBLE;
-import static org.apache.flink.table.api.DataTypes.FIELD;
-import static org.apache.flink.table.api.DataTypes.INT;
-import static org.apache.flink.table.api.DataTypes.ROW;
-import static org.apache.flink.table.api.DataTypes.STRING;
-import static org.assertj.core.api.Assertions.assertThat;
-
-/** Test for {@link HBaseSerde}. */
-class HBaseSerdeTest {
-
-    private static final String ROW_KEY = "rowkey";
-
-    private static final String FAMILY1 = "family1";
-    private static final String F1COL1 = "col1";
-
-    private static final String FAMILY2 = "family2";
-    private static final String F2COL1 = "col1";
-    private static final String F2COL2 = "col2";
-
-    private static final String FAMILY3 = "family3";
-    private static final String F3COL1 = "col1";
-    private static final String F3COL2 = "col2";
-    private static final String F3COL3 = "col3";
-
-    @Test
-    void convertToNewRowTest() {
-        HBaseSerde serde = createHBaseSerde();
-        List<List<Cell>> cellsList = prepareCells();
-        List<RowData> resultRowDatas = new ArrayList<>();
-        List<String> resultRowDataStr = new ArrayList<>();
-        for (List<Cell> cells : cellsList) {
-            RowData row = serde.convertToNewRow(Result.create(cells));
-            resultRowDatas.add(row);
-            resultRowDataStr.add(row.toString());
-        }
-
-        assertThat(resultRowDatas.get(0))
-                .as("RowData should not be reused")
-                .isNotSameAs(resultRowDatas.get(1));
-        assertThat(resultRowDataStr)
-                .containsExactly(
-                        "+I(1,+I(10),+I(Hello-1,100),+I(1.01,false,Welt-1))",
-                        "+I(2,+I(20),+I(Hello-2,200),+I(2.02,true,Welt-2))");
-    }
-
-    @Test
-    void convertToReusedRowTest() {
-        HBaseSerde serde = createHBaseSerde();
-        List<List<Cell>> cellsList = prepareCells();
-        List<RowData> resultRowDatas = new ArrayList<>();
-        List<String> resultRowDataStr = new ArrayList<>();
-        for (List<Cell> cells : cellsList) {
-            RowData row = serde.convertToReusedRow(Result.create(cells));
-            resultRowDatas.add(row);
-            resultRowDataStr.add(row.toString());
-        }
-
-        assertThat(resultRowDatas.get(0))
-                .as("RowData should be reused")
-                .isSameAs(resultRowDatas.get(1));
-
-        assertThat(resultRowDataStr)
-                .containsExactly(
-                        "+I(1,+I(10),+I(Hello-1,100),+I(1.01,false,Welt-1))",
-                        "+I(2,+I(20),+I(Hello-2,200),+I(2.02,true,Welt-2))");
-    }
-
-    private HBaseSerde createHBaseSerde() {
-        DataType dataType =
-                ROW(
-                        FIELD(ROW_KEY, INT()),
-                        FIELD(FAMILY1, ROW(FIELD(F1COL1, INT()))),
-                        FIELD(FAMILY2, ROW(FIELD(F2COL1, STRING()), FIELD(F2COL2, BIGINT()))),
-                        FIELD(
-                                FAMILY3,
-                                ROW(
-                                        FIELD(F3COL1, DOUBLE()),
-                                        FIELD(F3COL2, DataTypes.BOOLEAN()),
-                                        FIELD(F3COL3, STRING()))));
-        HBaseTableSchema hbaseSchema = HBaseTableSchema.fromDataType(dataType);
-        return new HBaseSerde(hbaseSchema, "null");
-    }
-
-    private List<List<Cell>> prepareCells() {
-        List<List<Cell>> cellList = new ArrayList<>();
-        byte[] fam1 = Bytes.toBytes(FAMILY1);
-        byte[] f1c1 = Bytes.toBytes(F1COL1);
-
-        byte[] fam2 = Bytes.toBytes(FAMILY2);
-        byte[] f2c1 = Bytes.toBytes(F2COL1);
-        byte[] f2c2 = Bytes.toBytes(F2COL2);
-
-        byte[] fam3 = Bytes.toBytes(FAMILY3);
-        byte[] f3c1 = Bytes.toBytes(F3COL1);
-        byte[] f3c2 = Bytes.toBytes(F3COL2);
-        byte[] f3c3 = Bytes.toBytes(F3COL3);
-
-        byte[] row1 = Bytes.toBytes(1);
-        byte[] row2 = Bytes.toBytes(2);
-
-        Cell kv111 = new KeyValue(row1, fam1, f1c1, Bytes.toBytes(10));
-        Cell kv121 = new KeyValue(row1, fam2, f2c1, Bytes.toBytes("Hello-1"));
-        Cell kv122 = new KeyValue(row1, fam2, f2c2, Bytes.toBytes(100L));
-        Cell kv131 = new KeyValue(row1, fam3, f3c1, Bytes.toBytes(1.01));
-        Cell kv132 = new KeyValue(row1, fam3, f3c2, Bytes.toBytes(false));
-        Cell kv133 = new KeyValue(row1, fam3, f3c3, Bytes.toBytes("Welt-1"));
-
-        Cell kv211 = new KeyValue(row2, fam1, f1c1, Bytes.toBytes(20));
-        Cell kv221 = new KeyValue(row2, fam2, f2c1, Bytes.toBytes("Hello-2"));
-        Cell kv222 = new KeyValue(row2, fam2, f2c2, Bytes.toBytes(200L));
-        Cell kv231 = new KeyValue(row2, fam3, f3c1, Bytes.toBytes(2.02));
-        Cell kv232 = new KeyValue(row2, fam3, f3c2, Bytes.toBytes(true));
-        Cell kv233 = new KeyValue(row2, fam3, f3c3, Bytes.toBytes("Welt-2"));
-        List<Cell> cells1 = new ArrayList<>();
-        cells1.add(kv111);
-        cells1.add(kv121);
-        cells1.add(kv122);
-        cells1.add(kv131);
-        cells1.add(kv132);
-        cells1.add(kv133);
-        List<Cell> cells2 = new ArrayList<>();
-        cells2.add(kv211);
-        cells2.add(kv221);
-        cells2.add(kv222);
-        cells2.add(kv231);
-        cells2.add(kv232);
-        cells2.add(kv233);
-        cellList.add(cells1);
-        cellList.add(cells2);
-        return cellList;
-    }
-}
diff --git a/flink-connectors/flink-connector-hbase-base/src/test/java/org/apache/flink/connector/hbase/util/HBaseStronglyIncreasingTsGeneratorTest.java b/flink-connectors/flink-connector-hbase-base/src/test/java/org/apache/flink/connector/hbase/util/HBaseStronglyIncreasingTsGeneratorTest.java
deleted file mode 100644
index ea18266..0000000
--- a/flink-connectors/flink-connector-hbase-base/src/test/java/org/apache/flink/connector/hbase/util/HBaseStronglyIncreasingTsGeneratorTest.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * 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.util;
-
-import org.junit.Test;
-
-import static org.junit.Assert.assertTrue;
-
-/** Tests for {@link HBaseStronglyIncreasingTsGenerator}. */
-public class HBaseStronglyIncreasingTsGeneratorTest {
-    @Test
-    public void testStronglyIncreasingTimestampGenerator() {
-        HBaseStronglyIncreasingTsGenerator timestampGenerator =
-                new HBaseStronglyIncreasingTsGenerator();
-        long lastTimestamp = 0;
-        for (int i = 0; i < 10; i++) {
-            final long now = timestampGenerator.get();
-            if (lastTimestamp > 0) {
-                assertTrue(now > lastTimestamp);
-            }
-            lastTimestamp = now;
-        }
-        final long realNow = timestampGenerator.getCurrentSystemTimeNano();
-        assertTrue(
-                "The increasing timestamp should not exceed the current actual timestamp after 10 tests",
-                realNow >= lastTimestamp);
-    }
-}
diff --git a/flink-connectors/flink-connector-hbase-base/src/test/resources/META-INF/services/org.junit.jupiter.api.extension.Extension b/flink-connectors/flink-connector-hbase-base/src/test/resources/META-INF/services/org.junit.jupiter.api.extension.Extension
deleted file mode 100644
index 2899913..0000000
--- a/flink-connectors/flink-connector-hbase-base/src/test/resources/META-INF/services/org.junit.jupiter.api.extension.Extension
+++ /dev/null
@@ -1,16 +0,0 @@
-# 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.
-
-org.apache.flink.util.TestLoggerExtension
\ No newline at end of file
diff --git a/flink-connectors/flink-connector-hbase-base/src/test/resources/archunit.properties b/flink-connectors/flink-connector-hbase-base/src/test/resources/archunit.properties
deleted file mode 100644
index 15be88c..0000000
--- a/flink-connectors/flink-connector-hbase-base/src/test/resources/archunit.properties
+++ /dev/null
@@ -1,31 +0,0 @@
-#
-# 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.
-#
-
-# By default we allow removing existing violations, but fail when new violations are added.
-freeze.store.default.allowStoreUpdate=true
-
-# Enable this if a new (frozen) rule has been added in order to create the initial store and record the existing violations.
-#freeze.store.default.allowStoreCreation=true
-
-# Enable this to add allow new violations to be recorded.
-# NOTE: Adding new violations should be avoided when possible. If the rule was correct to flag a new
-#       violation, please try to avoid creating the violation. If the violation was created due to a
-#       shortcoming of the rule, file a JIRA issue so the rule can be improved.
-#freeze.refreeze=true
-
-freeze.store.default.path=archunit-violations
diff --git a/flink-connectors/flink-connector-hbase-base/src/test/resources/hbase-site.xml b/flink-connectors/flink-connector-hbase-base/src/test/resources/hbase-site.xml
deleted file mode 100644
index 1e58ef4..0000000
--- a/flink-connectors/flink-connector-hbase-base/src/test/resources/hbase-site.xml
+++ /dev/null
@@ -1,29 +0,0 @@
-<?xml version="1.0"?>
-<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
-<!--
-  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.
--->
-
-<!-- Values used when running unit tests.  Specify any values in here that
-     should override the default values. -->
-
-<configuration>
-    <property>
-        <name>hbase_conf_key</name>
-        <value>hbase_conf_value!</value>
-    </property>
-</configuration>
diff --git a/flink-connectors/flink-connector-hbase-base/src/test/resources/log4j2-test.properties b/flink-connectors/flink-connector-hbase-base/src/test/resources/log4j2-test.properties
deleted file mode 100644
index 835c2ec..0000000
--- a/flink-connectors/flink-connector-hbase-base/src/test/resources/log4j2-test.properties
+++ /dev/null
@@ -1,28 +0,0 @@
-################################################################################
-#  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.
-################################################################################
-
-# Set root logger level to OFF to not flood build logs
-# set manually to INFO for debugging purposes
-rootLogger.level = OFF
-rootLogger.appenderRef.test.ref = TestLogger
-
-appender.testlogger.name = TestLogger
-appender.testlogger.type = CONSOLE
-appender.testlogger.target = SYSTEM_ERR
-appender.testlogger.layout.type = PatternLayout
-appender.testlogger.layout.pattern = %-4r [%t] %-5p %c %x - %m%n
diff --git a/flink-connectors/flink-sql-connector-hbase-1.4/pom.xml b/flink-connectors/flink-sql-connector-hbase-1.4/pom.xml
deleted file mode 100644
index d2d6627..0000000
--- a/flink-connectors/flink-sql-connector-hbase-1.4/pom.xml
+++ /dev/null
@@ -1,166 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-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.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0"
-		 xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-		 xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-	<parent>
-		<artifactId>flink-connectors</artifactId>
-		<groupId>org.apache.flink</groupId>
-		<version>1.19-SNAPSHOT</version>
-	</parent>
-	<modelVersion>4.0.0</modelVersion>
-
-	<artifactId>flink-sql-connector-hbase-1.4</artifactId>
-	<name>Flink : Connectors : SQL : HBase 1.4</name>
-
-	<packaging>jar</packaging>
-
-	<properties>
-		<japicmp.skip>true</japicmp.skip>
-	</properties>
-
-	<dependencies>
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-connector-hbase-1.4</artifactId>
-			<version>${project.version}</version>
-			<optional>${flink.markBundledAsOptional}</optional>
-		</dependency>
-	</dependencies>
-
-	<dependencyManagement>
-		<dependencies>
-			<dependency>
-				<!-- HBase only works with Zookeper 3.4 -->
-				<groupId>org.apache.zookeeper</groupId>
-				<artifactId>zookeeper</artifactId>
-				<version>3.4.14</version>
-			</dependency>
-		</dependencies>
-	</dependencyManagement>
-
-	<build>
-		<plugins>
-			<plugin>
-				<groupId>org.apache.maven.plugins</groupId>
-				<artifactId>maven-shade-plugin</artifactId>
-				<executions>
-					<execution>
-						<id>shade-flink</id>
-						<phase>package</phase>
-						<goals>
-							<goal>shade</goal>
-						</goals>
-						<configuration>
-							<!--
-							Make the file hbase-default.xml under flink-sql-connector-hbase-1.4/src/main/resources as the
-							hbase-default.xml in the shaded target jar here, because we don't want to check the hbase
-							version at client side. Also we don't need the extra default configs keys.
-							-->
-							<transformers>
-								<transformer
-										implementation="org.apache.maven.plugins.shade.resource.IncludeResourceTransformer">
-									<resource>hbase-default.xml</resource>
-									<file>hbase-default.xml</file>
-								</transformer>
-							</transformers>
-							<artifactSet>
-								<includes>
-									<include>org.apache.flink:flink-connector-base</include>
-									<include>org.apache.flink:flink-connector-hbase-base</include>
-									<include>org.apache.flink:flink-connector-hbase-1.4</include>
-									<include>org.apache.hbase:hbase-*</include>
-									<include>org.apache.zookeeper:zookeeper</include>
-									<include>org.apache.htrace:htrace-core</include>
-									<include>com.google.guava:guava</include>
-									<include>com.google.protobuf:protobuf-java</include>
-									<include>com.yammer.metrics:metrics-core</include>
-									<include>commons-logging:commons-logging</include>
-									<include>commons-lang:commons-lang</include>
-									<include>commons-codec:commons-codec</include>
-									<include>io.netty:netty-all</include>
-								</includes>
-								<excludes>
-									<exclude>org.apache.hbase:hbase-common:jar:tests</exclude>
-								</excludes>
-							</artifactSet>
-							<filters>
-								<filter>
-									<artifact>*:*</artifact>
-									<excludes>
-										<!-- excluded all these files for a clean jar -->
-										<exclude>META-INF/services/com.fasterxml.**</exclude>
-										<exclude>META-INF/services/org.apache.hadoop.**</exclude>
-										<exclude>META-INF/services/javax.**</exclude>
- 										<exclude>digesterRules.xml</exclude>
-										<exclude>properties.dtd</exclude>
-										<exclude>PropertyList-1.0.dtd</exclude>
-										<exclude>LICENSE.txt</exclude>
-									</excludes>
-								</filter>
-							</filters>
-							<relocations>
-								<!-- Force relocation of all HBase dependencies. -->
-								<relocation>
-									<pattern>org.apache.zookeeper</pattern>
-									<shadedPattern>org.apache.flink.hbase.shaded.org.apache.zookeeper</shadedPattern>
-								</relocation>
-								<relocation>
-									<pattern>org.apache.htrace</pattern>
-									<shadedPattern>org.apache.flink.hbase.shaded.org.apache.htrace</shadedPattern>
-								</relocation>
-								<relocation>
-									<pattern>com.google</pattern>
-									<shadedPattern>org.apache.flink.hbase.shaded.com.google</shadedPattern>
-								</relocation>
-								<relocation>
-									<pattern>com.yammer.metrics</pattern>
-									<shadedPattern>org.apache.flink.hbase.shaded.com.yammer.metrics</shadedPattern>
-								</relocation>
-								<relocation>
-									<pattern>org.apache.commons</pattern>
-									<shadedPattern>org.apache.flink.hbase.shaded.org.apache.commons</shadedPattern>
-								</relocation>
-								<relocation>
-									<pattern>org.apache.jute</pattern>
-									<shadedPattern>org.apache.flink.hbase.shaded.org.apache.jute</shadedPattern>
-								</relocation>
-								<relocation>
-									<pattern>io.netty</pattern>
-									<shadedPattern>org.apache.flink.hbase.shaded.io.netty</shadedPattern>
-								</relocation>
-								<relocation>
-									<pattern>org.apache.hadoop.hbase</pattern>
-									<shadedPattern>org.apache.flink.hbase.shaded.org.apache.hadoop.hbase</shadedPattern>
-									<!-- HBase client uses shaded KeyValueCodec to encode data and put the class name in
- 										the header of request, the HBase region server can not load the shaded
-										KeyValueCodec class when decode the data, so we exclude them here. -->
-									<excludes>
-										<exclude>org.apache.hadoop.hbase.codec.*</exclude>
-									</excludes>
-								</relocation>
-							</relocations>
-						</configuration>
-					</execution>
-				</executions>
-			</plugin>
-		</plugins>
-	</build>
-</project>
diff --git a/flink-connectors/flink-sql-connector-hbase-1.4/src/main/resources/META-INF/NOTICE b/flink-connectors/flink-sql-connector-hbase-1.4/src/main/resources/META-INF/NOTICE
deleted file mode 100644
index f1842fa..0000000
--- a/flink-connectors/flink-sql-connector-hbase-1.4/src/main/resources/META-INF/NOTICE
+++ /dev/null
@@ -1,31 +0,0 @@
-flink-sql-connector-hbase-1.4
-Copyright 2014-2023 The Apache Software Foundation
-
-This product includes software developed at
-The Apache Software Foundation (http://www.apache.org/).
-
-This project bundles the following dependencies under the Apache Software License 2.0. (http://www.apache.org/licenses/LICENSE-2.0.txt)
-- com.google.guava:guava:12.0.1
-- com.yammer.metrics:metrics-core:2.2.0
-- commons-codec:commons-codec:1.15
-- commons-lang:commons-lang:2.6
-- commons-logging:commons-logging:1.1.3
-- io.netty:netty-all:4.1.70.Final
-- org.apache.hbase:hbase-client:1.4.3
-- org.apache.hbase:hbase-common:1.4.3
-- org.apache.hbase:hbase-protocol:1.4.3
-- org.apache.htrace:htrace-core:3.1.0-incubating
-- org.apache.zookeeper:zookeeper:3.4.14
-
-This project bundles the following dependencies under the BSD license.
-See bundled license files for details.
-
-- com.google.protobuf:protobuf-java:2.5.0
-
-The bundled Apache HTrace org.apache.htrace:htrace-core dependency bundles the following dependencies under
-the Apache Software License 2.0 (http://www.apache.org/licenses/LICENSE-2.0.txt)
-
-- com.fasterxml.jackson.core:jackson-annotations:2.4.0
-- com.fasterxml.jackson.core:jackson-core:2.4.0
-- com.fasterxml.jackson.core:jackson-databind:2.4.0
-- commons-logging:commons-logging:1.1.1
diff --git a/flink-connectors/flink-sql-connector-hbase-1.4/src/main/resources/META-INF/licenses/LICENSE.protobuf b/flink-connectors/flink-sql-connector-hbase-1.4/src/main/resources/META-INF/licenses/LICENSE.protobuf
deleted file mode 100644
index 19b305b..0000000
--- a/flink-connectors/flink-sql-connector-hbase-1.4/src/main/resources/META-INF/licenses/LICENSE.protobuf
+++ /dev/null
@@ -1,32 +0,0 @@
-Copyright 2008 Google Inc.  All rights reserved.
-
-Redistribution and use in source and binary forms, with or without
-modification, are permitted provided that the following conditions are
-met:
-
-    * Redistributions of source code must retain the above copyright
-notice, this list of conditions and the following disclaimer.
-    * Redistributions in binary form must reproduce the above
-copyright notice, this list of conditions and the following disclaimer
-in the documentation and/or other materials provided with the
-distribution.
-    * Neither the name of Google Inc. nor the names of its
-contributors may be used to endorse or promote products derived from
-this software without specific prior written permission.
-
-THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
-"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
-LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
-A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
-OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
-SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
-LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
-DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
-THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
-(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
-OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
-
-Code generated by the Protocol Buffer compiler is owned by the owner
-of the input file used when generating it.  This code is not
-standalone and requires a support library to be linked with it.  This
-support library is itself covered by the above license.
diff --git a/flink-connectors/flink-sql-connector-hbase-1.4/src/main/resources/hbase-default.xml b/flink-connectors/flink-sql-connector-hbase-1.4/src/main/resources/hbase-default.xml
deleted file mode 100644
index cd2c15e..0000000
--- a/flink-connectors/flink-sql-connector-hbase-1.4/src/main/resources/hbase-default.xml
+++ /dev/null
@@ -1,1558 +0,0 @@
-<?xml version="1.0"?>
-<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
-<!--
-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.
--->
-
-<!--
-OVERVIEW
-The important configs. are listed near the top.  You should change
-at least the setting for hbase.tmp.dir.  Other settings will change
-dependent on whether you are running hbase in standalone mode or
-distributed.  See the hbase reference guide for requirements and
-guidance making configuration.
-This file does not contain all possible configurations.  The file would be
-much larger if it carried everything. The absent configurations will only be
-found through source code reading.  The idea is that such configurations are
-exotic and only those who would go to the trouble of reading a particular
-section in the code would be knowledgeable or invested enough in ever wanting
-to alter such configurations, so we do not list them here.  Listing all
-possible configurations would overwhelm and obscure the important.
--->
-
-<configuration>
-	<!--Configs you will likely change are listed here at the top of the file.
-	-->
-	<property>
-		<name>hbase.tmp.dir</name>
-		<value>${java.io.tmpdir}/hbase-${user.name}</value>
-		<description>Temporary directory on the local filesystem.
-			Change this setting to point to a location more permanent
-			than '/tmp', the usual resolve for java.io.tmpdir, as the
-			'/tmp' directory is cleared on machine restart.</description>
-	</property>
-	<property>
-		<name>hbase.rootdir</name>
-		<value>${hbase.tmp.dir}/hbase</value>
-		<description>The directory shared by region servers and into
-			which HBase persists.  The URL should be 'fully-qualified'
-			to include the filesystem scheme.  For example, to specify the
-			HDFS directory '/hbase' where the HDFS instance's namenode is
-			running at namenode.example.org on port 9000, set this value to:
-			hdfs://namenode.example.org:9000/hbase.  By default, we write
-			to whatever ${hbase.tmp.dir} is set too -- usually /tmp --
-			so change this configuration or else all data will be lost on
-			machine restart.</description>
-	</property>
-	<property >
-		<name>hbase.fs.tmp.dir</name>
-		<value>/user/${user.name}/hbase-staging</value>
-		<description>A staging directory in default file system (HDFS)
-			for keeping temporary data.
-		</description>
-	</property>
-	<property >
-		<name>hbase.bulkload.staging.dir</name>
-		<value>${hbase.fs.tmp.dir}</value>
-		<description>A staging directory in default file system (HDFS)
-			for bulk loading.
-		</description>
-	</property>
-	<property >
-		<name>hbase.cluster.distributed</name>
-		<value>false</value>
-		<description>The mode the cluster will be in. Possible values are
-			false for standalone mode and true for distributed mode.  If
-			false, startup will run all HBase and ZooKeeper daemons together
-			in the one JVM.</description>
-	</property>
-	<property>
-		<name>hbase.zookeeper.quorum</name>
-		<value>localhost</value>
-		<description>Comma separated list of servers in the ZooKeeper ensemble
-			(This config. should have been named hbase.zookeeper.ensemble).
-			For example, "host1.mydomain.com,host2.mydomain.com,host3.mydomain.com".
-			By default this is set to localhost for local and pseudo-distributed modes
-			of operation. For a fully-distributed setup, this should be set to a full
-			list of ZooKeeper ensemble servers. If HBASE_MANAGES_ZK is set in hbase-env.sh
-			this is the list of servers which hbase will start/stop ZooKeeper on as
-			part of cluster start/stop.  Client-side, we will take this list of
-			ensemble members and put it together with the hbase.zookeeper.clientPort
-			config. and pass it into zookeeper constructor as the connectString
-			parameter.</description>
-	</property>
-	<!--The above are the important configurations for getting hbase up
-	  and running -->
-
-	<property>
-		<name>zookeeper.recovery.retry.maxsleeptime</name>
-		<value>60000</value>
-		<description>Max sleep time before retry zookeeper operations in milliseconds,
-			a max time is needed here so that sleep time won't grow unboundedly
-		</description>
-	</property>
-	<property>
-		<name>hbase.local.dir</name>
-		<value>${hbase.tmp.dir}/local/</value>
-		<description>Directory on the local filesystem to be used
-			as a local storage.</description>
-	</property>
-
-	<!--Master configurations-->
-	<property >
-		<name>hbase.master.port</name>
-		<value>16000</value>
-		<description>The port the HBase Master should bind to.</description>
-	</property>
-	<property>
-		<name>hbase.master.info.port</name>
-		<value>16010</value>
-		<description>The port for the HBase Master web UI.
-			Set to -1 if you do not want a UI instance run.</description>
-	</property>
-	<property>
-		<name>hbase.master.info.bindAddress</name>
-		<value>0.0.0.0</value>
-		<description>The bind address for the HBase Master web UI
-		</description>
-	</property>
-	<property>
-		<name>hbase.master.logcleaner.ttl</name>
-		<value>600000</value>
-		<description>Maximum time a WAL can stay in the .oldlogdir directory,
-			after which it will be cleaned by a Master thread.</description>
-	</property>
-	<property>
-		<name>hbase.master.infoserver.redirect</name>
-		<value>true</value>
-		<description>Whether or not the Master listens to the Master web
-			UI port (hbase.master.info.port) and redirects requests to the web
-			UI server shared by the Master and RegionServer.</description>
-	</property>
-
-	<!--RegionServer configurations-->
-	<property>
-		<name>hbase.regionserver.port</name>
-		<value>16020</value>
-		<description>The port the HBase RegionServer binds to.</description>
-	</property>
-	<property>
-		<name>hbase.regionserver.info.port</name>
-		<value>16030</value>
-		<description>The port for the HBase RegionServer web UI
-			Set to -1 if you do not want the RegionServer UI to run.</description>
-	</property>
-	<property>
-		<name>hbase.regionserver.info.bindAddress</name>
-		<value>0.0.0.0</value>
-		<description>The address for the HBase RegionServer web UI</description>
-	</property>
-	<property>
-		<name>hbase.regionserver.info.port.auto</name>
-		<value>false</value>
-		<description>Whether or not the Master or RegionServer
-			UI should search for a port to bind to. Enables automatic port
-			search if hbase.regionserver.info.port is already in use.
-			Useful for testing, turned off by default.</description>
-	</property>
-	<property>
-		<name>hbase.regionserver.handler.count</name>
-		<value>30</value>
-		<description>Count of RPC Listener instances spun up on RegionServers.
-			Same property is used by the Master for count of master handlers.</description>
-	</property>
-	<property>
-		<name>hbase.ipc.server.callqueue.handler.factor</name>
-		<value>0.1</value>
-		<description>Factor to determine the number of call queues.
-			A value of 0 means a single queue shared between all the handlers.
-			A value of 1 means that each handler has its own queue.</description>
-	</property>
-	<property>
-		<name>hbase.ipc.server.callqueue.read.ratio</name>
-		<value>0</value>
-		<description>Split the call queues into read and write queues.
-			The specified interval (which should be between 0.0 and 1.0)
-			will be multiplied by the number of call queues.
-			A value of 0 indicate to not split the call queues, meaning that both read and write
-			requests will be pushed to the same set of queues.
-			A value lower than 0.5 means that there will be less read queues than write queues.
-			A value of 0.5 means there will be the same number of read and write queues.
-			A value greater than 0.5 means that there will be more read queues than write queues.
-			A value of 1.0 means that all the queues except one are used to dispatch read requests.
-
-			Example: Given the total number of call queues being 10
-			a read.ratio of 0 means that: the 10 queues will contain both read/write requests.
-			a read.ratio of 0.3 means that: 3 queues will contain only read requests
-			and 7 queues will contain only write requests.
-			a read.ratio of 0.5 means that: 5 queues will contain only read requests
-			and 5 queues will contain only write requests.
-			a read.ratio of 0.8 means that: 8 queues will contain only read requests
-			and 2 queues will contain only write requests.
-			a read.ratio of 1 means that: 9 queues will contain only read requests
-			and 1 queues will contain only write requests.
-		</description>
-	</property>
-	<property>
-		<name>hbase.ipc.server.callqueue.scan.ratio</name>
-		<value>0</value>
-		<description>Given the number of read call queues, calculated from the total number
-			of call queues multiplied by the callqueue.read.ratio, the scan.ratio property
-			will split the read call queues into small-read and long-read queues.
-			A value lower than 0.5 means that there will be less long-read queues than short-read queues.
-			A value of 0.5 means that there will be the same number of short-read and long-read queues.
-			A value greater than 0.5 means that there will be more long-read queues than short-read queues
-			A value of 0 or 1 indicate to use the same set of queues for gets and scans.
-
-			Example: Given the total number of read call queues being 8
-			a scan.ratio of 0 or 1 means that: 8 queues will contain both long and short read requests.
-			a scan.ratio of 0.3 means that: 2 queues will contain only long-read requests
-			and 6 queues will contain only short-read requests.
-			a scan.ratio of 0.5 means that: 4 queues will contain only long-read requests
-			and 4 queues will contain only short-read requests.
-			a scan.ratio of 0.8 means that: 6 queues will contain only long-read requests
-			and 2 queues will contain only short-read requests.
-		</description>
-	</property>
-	<property>
-		<name>hbase.regionserver.msginterval</name>
-		<value>3000</value>
-		<description>Interval between messages from the RegionServer to Master
-			in milliseconds.</description>
-	</property>
-	<property>
-		<name>hbase.regionserver.logroll.period</name>
-		<value>3600000</value>
-		<description>Period at which we will roll the commit log regardless
-			of how many edits it has.</description>
-	</property>
-	<property>
-		<name>hbase.regionserver.logroll.errors.tolerated</name>
-		<value>2</value>
-		<description>The number of consecutive WAL close errors we will allow
-			before triggering a server abort.  A setting of 0 will cause the
-			region server to abort if closing the current WAL writer fails during
-			log rolling.  Even a small value (2 or 3) will allow a region server
-			to ride over transient HDFS errors.</description>
-	</property>
-	<property>
-		<name>hbase.regionserver.global.memstore.size</name>
-		<value></value>
-		<description>Maximum size of all memstores in a region server before new
-			updates are blocked and flushes are forced. Defaults to 40% of heap (0.4).
-			Updates are blocked and flushes are forced until size of all memstores
-			in a region server hits hbase.regionserver.global.memstore.size.lower.limit.
-			The default value in this configuration has been intentionally left empty in order to
-			honor the old hbase.regionserver.global.memstore.upperLimit property if present.</description>
-	</property>
-	<property>
-		<name>hbase.regionserver.global.memstore.size.lower.limit</name>
-		<value></value>
-		<description>Maximum size of all memstores in a region server before flushes are forced.
-			Defaults to 95% of hbase.regionserver.global.memstore.size (0.95).
-			A 100% value for this value causes the minimum possible flushing to occur when updates are
-			blocked due to memstore limiting.
-			The default value in this configuration has been intentionally left empty in order to
-			honor the old hbase.regionserver.global.memstore.lowerLimit property if present.</description>
-	</property>
-	<property>
-		<name>hbase.regionserver.optionalcacheflushinterval</name>
-		<value>3600000</value>
-		<description>
-			Maximum amount of time an edit lives in memory before being automatically flushed.
-			Default 1 hour. Set it to 0 to disable automatic flushing.</description>
-	</property>
-	<property>
-		<name>hbase.regionserver.dns.interface</name>
-		<value>default</value>
-		<description>The name of the Network Interface from which a region server
-			should report its IP address.</description>
-	</property>
-	<property>
-		<name>hbase.regionserver.dns.nameserver</name>
-		<value>default</value>
-		<description>The host name or IP address of the name server (DNS)
-			which a region server should use to determine the host name used by the
-			master for communication and display purposes.</description>
-	</property>
-	<property>
-		<name>hbase.regionserver.regionSplitLimit</name>
-		<value>1000</value>
-		<description>
-			Limit for the number of regions after which no more region splitting should take place.
-			This is not hard limit for the number of regions but acts as a guideline for the regionserver
-			to stop splitting after a certain limit. Default is set to 1000.
-		</description>
-	</property>
-
-	<!--ZooKeeper configuration-->
-	<property>
-		<name>zookeeper.session.timeout</name>
-		<value>90000</value>
-		<description>ZooKeeper session timeout in milliseconds. It is used in two different ways.
-			First, this value is used in the ZK client that HBase uses to connect to the ensemble.
-			It is also used by HBase when it starts a ZK server and it is passed as the 'maxSessionTimeout'. See
-			http://hadoop.apache.org/zookeeper/docs/current/zookeeperProgrammers.html#ch_zkSessions.
-			For example, if a HBase region server connects to a ZK ensemble that's also managed by HBase, then the
-			session timeout will be the one specified by this configuration. But, a region server that connects
-			to an ensemble managed with a different configuration will be subjected that ensemble's maxSessionTimeout. So,
-			even though HBase might propose using 90 seconds, the ensemble can have a max timeout lower than this and
-			it will take precedence. The current default that ZK ships with is 40 seconds, which is lower than HBase's.
-		</description>
-	</property>
-	<property>
-		<name>zookeeper.znode.parent</name>
-		<value>/hbase</value>
-		<description>Root ZNode for HBase in ZooKeeper. All of HBase's ZooKeeper
-			files that are configured with a relative path will go under this node.
-			By default, all of HBase's ZooKeeper file path are configured with a
-			relative path, so they will all go under this directory unless changed.</description>
-	</property>
-	<property>
-		<name>zookeeper.znode.acl.parent</name>
-		<value>acl</value>
-		<description>Root ZNode for access control lists.</description>
-	</property>
-	<property>
-		<name>hbase.zookeeper.dns.interface</name>
-		<value>default</value>
-		<description>The name of the Network Interface from which a ZooKeeper server
-			should report its IP address.</description>
-	</property>
-	<property>
-		<name>hbase.zookeeper.dns.nameserver</name>
-		<value>default</value>
-		<description>The host name or IP address of the name server (DNS)
-			which a ZooKeeper server should use to determine the host name used by the
-			master for communication and display purposes.</description>
-	</property>
-	<!--
-	The following three properties are used together to create the list of
-	host:peer_port:leader_port quorum servers for ZooKeeper.
-	-->
-	<property>
-		<name>hbase.zookeeper.peerport</name>
-		<value>2888</value>
-		<description>Port used by ZooKeeper peers to talk to each other.
-			See http://hadoop.apache.org/zookeeper/docs/r3.1.1/zookeeperStarted.html#sc_RunningReplicatedZooKeeper
-			for more information.</description>
-	</property>
-	<property>
-		<name>hbase.zookeeper.leaderport</name>
-		<value>3888</value>
-		<description>Port used by ZooKeeper for leader election.
-			See http://hadoop.apache.org/zookeeper/docs/r3.1.1/zookeeperStarted.html#sc_RunningReplicatedZooKeeper
-			for more information.</description>
-	</property>
-	<!-- End of properties used to generate ZooKeeper host:port quorum list. -->
-	<property>
-		<name>hbase.zookeeper.useMulti</name>
-		<value>true</value>
-		<description>Instructs HBase to make use of ZooKeeper's multi-update functionality.
-			This allows certain ZooKeeper operations to complete more quickly and prevents some issues
-			with rare Replication failure scenarios (see the release note of HBASE-2611 for an example).
-			IMPORTANT: only set this to true if all ZooKeeper servers in the cluster are on version 3.4+
-			and will not be downgraded.  ZooKeeper versions before 3.4 do not support multi-update and
-			will not fail gracefully if multi-update is invoked (see ZOOKEEPER-1495).</description>
-	</property>
-	<property>
-		<name>hbase.config.read.zookeeper.config</name>
-		<value>false</value>
-		<description>
-			Set to true to allow HBaseConfiguration to read the
-			zoo.cfg file for ZooKeeper properties. Switching this to true
-			is not recommended, since the functionality of reading ZK
-			properties from a zoo.cfg file has been deprecated.</description>
-	</property>
-	<!--
-	Beginning of properties that are directly mapped from ZooKeeper's zoo.cfg.
-	All properties with an "hbase.zookeeper.property." prefix are converted for
-	ZooKeeper's configuration. Hence, if you want to add an option from zoo.cfg,
-	e.g.  "initLimit=10" you would append the following to your configuration:
-	  <property>
-		<name>hbase.zookeeper.property.initLimit</name>
-		<value>10</value>
-	  </property>
-	-->
-	<property>
-		<name>hbase.zookeeper.property.initLimit</name>
-		<value>10</value>
-		<description>Property from ZooKeeper's config zoo.cfg.
-			The number of ticks that the initial synchronization phase can take.</description>
-	</property>
-	<property>
-		<name>hbase.zookeeper.property.syncLimit</name>
-		<value>5</value>
-		<description>Property from ZooKeeper's config zoo.cfg.
-			The number of ticks that can pass between sending a request and getting an
-			acknowledgment.</description>
-	</property>
-	<property>
-		<name>hbase.zookeeper.property.dataDir</name>
-		<value>${hbase.tmp.dir}/zookeeper</value>
-		<description>Property from ZooKeeper's config zoo.cfg.
-			The directory where the snapshot is stored.</description>
-	</property>
-	<property>
-		<name>hbase.zookeeper.property.clientPort</name>
-		<value>2181</value>
-		<description>Property from ZooKeeper's config zoo.cfg.
-			The port at which the clients will connect.</description>
-	</property>
-	<property>
-		<name>hbase.zookeeper.property.maxClientCnxns</name>
-		<value>300</value>
-		<description>Property from ZooKeeper's config zoo.cfg.
-			Limit on number of concurrent connections (at the socket level) that a
-			single client, identified by IP address, may make to a single member of
-			the ZooKeeper ensemble. Set high to avoid zk connection issues running
-			standalone and pseudo-distributed.</description>
-	</property>
-	<!-- End of properties that are directly mapped from ZooKeeper's zoo.cfg -->
-
-	<!--Client configurations-->
-	<property>
-		<name>hbase.client.write.buffer</name>
-		<value>2097152</value>
-		<description>Default size of the HTable client write buffer in bytes.
-			A bigger buffer takes more memory -- on both the client and server
-			side since server instantiates the passed write buffer to process
-			it -- but a larger buffer size reduces the number of RPCs made.
-			For an estimate of server-side memory-used, evaluate
-			hbase.client.write.buffer * hbase.regionserver.handler.count</description>
-	</property>
-	<property>
-		<name>hbase.client.pause</name>
-		<value>100</value>
-		<description>General client pause value.  Used mostly as value to wait
-			before running a retry of a failed get, region lookup, etc.
-			See hbase.client.retries.number for description of how we backoff from
-			this initial pause amount and how this pause works w/ retries.</description>
-	</property>
-	<property>
-		<name>hbase.client.pause.cqtbe</name>
-		<value></value>
-		<description>Whether or not to use a special client pause for
-			CallQueueTooBigException (cqtbe). Set this property to a higher value
-			than hbase.client.pause if you observe frequent CQTBE from the same
-			RegionServer and the call queue there keeps full</description>
-	</property>
-	<property>
-		<name>hbase.client.retries.number</name>
-		<value>35</value>
-		<description>Maximum retries.  Used as maximum for all retryable
-			operations such as the getting of a cell's value, starting a row update,
-			etc.  Retry interval is a rough function based on hbase.client.pause.  At
-			first we retry at this interval but then with backoff, we pretty quickly reach
-			retrying every ten seconds.  See HConstants#RETRY_BACKOFF for how the backup
-			ramps up.  Change this setting and hbase.client.pause to suit your workload.</description>
-	</property>
-	<property>
-		<name>hbase.client.max.total.tasks</name>
-		<value>100</value>
-		<description>The maximum number of concurrent mutation tasks a single HTable instance will
-			send to the cluster.</description>
-	</property>
-	<property>
-		<name>hbase.client.max.perserver.tasks</name>
-		<value>5</value>
-		<description>The maximum number of concurrent mutation tasks a single HTable instance will
-			send to a single region server.</description>
-	</property>
-	<property>
-		<name>hbase.client.max.perregion.tasks</name>
-		<value>1</value>
-		<description>The maximum number of concurrent mutation tasks the client will
-			maintain to a single Region. That is, if there is already
-			hbase.client.max.perregion.tasks writes in progress for this region, new puts
-			won't be sent to this region until some writes finishes.</description>
-	</property>
-	<property>
-		<name>hbase.client.perserver.requests.threshold</name>
-		<value>2147483647</value>
-		<description>The max number of concurrent pending requests for one server in all client threads
-			(process level). Exceeding requests will be thrown ServerTooBusyException immediately to prevent
-			user's threads being occupied and blocked by only one slow region server. If you use a fix
-			number of threads to access HBase in a synchronous way, set this to a suitable value which is
-			related to the number of threads will help you. See
-			https://issues.apache.org/jira/browse/HBASE-16388 for details.</description>
-	</property>
-	<property>
-		<name>hbase.client.scanner.caching</name>
-		<value>2147483647</value>
-		<description>Number of rows that we try to fetch when calling next
-			on a scanner if it is not served from (local, client) memory. This configuration
-			works together with hbase.client.scanner.max.result.size to try and use the
-			network efficiently. The default value is Integer.MAX_VALUE by default so that
-			the network will fill the chunk size defined by hbase.client.scanner.max.result.size
-			rather than be limited by a particular number of rows since the size of rows varies
-			table to table. If you know ahead of time that you will not require more than a certain
-			number of rows from a scan, this configuration should be set to that row limit via
-			Scan#setCaching. Higher caching values will enable faster scanners but will eat up more
-			memory and some calls of next may take longer and longer times when the cache is empty.
-			Do not set this value such that the time between invocations is greater than the scanner
-			timeout; i.e. hbase.client.scanner.timeout.period</description>
-	</property>
-	<property>
-		<name>hbase.client.keyvalue.maxsize</name>
-		<value>10485760</value>
-		<description>Specifies the combined maximum allowed size of a KeyValue
-			instance. This is to set an upper boundary for a single entry saved in a
-			storage file. Since they cannot be split it helps avoiding that a region
-			cannot be split any further because the data is too large. It seems wise
-			to set this to a fraction of the maximum region size. Setting it to zero
-			or less disables the check.</description>
-	</property>
-	<property>
-		<name>hbase.server.keyvalue.maxsize</name>
-		<value>10485760</value>
-		<description>Maximum allowed size of an individual cell, inclusive of value and all key
-			components. A value of 0 or less disables the check.
-			The default value is 10MB.
-			This is a safety setting to protect the server from OOM situations.
-		</description>
-	</property>
-	<property>
-		<name>hbase.client.scanner.timeout.period</name>
-		<value>60000</value>
-		<description>Client scanner lease period in milliseconds.</description>
-	</property>
-	<property>
-		<name>hbase.client.localityCheck.threadPoolSize</name>
-		<value>2</value>
-	</property>
-
-	<!--Miscellaneous configuration-->
-	<property>
-		<name>hbase.bulkload.retries.number</name>
-		<value>10</value>
-		<description>Maximum retries.  This is maximum number of iterations
-			to atomic bulk loads are attempted in the face of splitting operations
-			0 means never give up.</description>
-	</property>
-	<property>
-		<name>hbase.master.balancer.maxRitPercent</name>
-		<value>1.0</value>
-		<description>The max percent of regions in transition when balancing.
-			The default value is 1.0. So there are no balancer throttling. If set this config to 0.01,
-			It means that there are at most 1% regions in transition when balancing.
-			Then the cluster's availability is at least 99% when balancing.</description>
-	</property>
-	<property>
-		<name>hbase.balancer.period
-		</name>
-		<value>300000</value>
-		<description>Period at which the region balancer runs in the Master.</description>
-	</property>
-	<property>
-		<name>hbase.normalizer.period</name>
-		<value>1800000</value>
-		<description>Period at which the region normalizer runs in the Master.</description>
-	</property>
-	<property>
-		<name>hbase.regions.slop</name>
-		<value>0.001</value>
-		<description>Rebalance if any regionserver has average + (average * slop) regions.
-			The default value of this parameter is 0.001 in StochasticLoadBalancer (the default load balancer),
-			while the default is 0.2 in other load balancers (i.e., SimpleLoadBalancer).</description>
-	</property>
-	<property>
-		<name>hbase.server.thread.wakefrequency</name>
-		<value>10000</value>
-		<description>Time to sleep in between searches for work (in milliseconds).
-			Used as sleep interval by service threads such as log roller.</description>
-	</property>
-	<property>
-		<name>hbase.server.versionfile.writeattempts</name>
-		<value>3</value>
-		<description>
-			How many time to retry attempting to write a version file
-			before just aborting. Each attempt is seperated by the
-			hbase.server.thread.wakefrequency milliseconds.</description>
-	</property>
-	<property>
-		<name>hbase.hregion.memstore.flush.size</name>
-		<value>134217728</value>
-		<description>
-			Memstore will be flushed to disk if size of the memstore
-			exceeds this number of bytes.  Value is checked by a thread that runs
-			every hbase.server.thread.wakefrequency.</description>
-	</property>
-	<property>
-		<name>hbase.hregion.percolumnfamilyflush.size.lower.bound</name>
-		<value>16777216</value>
-		<description>
-			If FlushLargeStoresPolicy is used, then every time that we hit the
-			total memstore limit, we find out all the column families whose memstores
-			exceed this value, and only flush them, while retaining the others whose
-			memstores are lower than this limit. If none of the families have their
-			memstore size more than this, all the memstores will be flushed
-			(just as usual). This value should be less than half of the total memstore
-			threshold (hbase.hregion.memstore.flush.size).
-		</description>
-	</property>
-	<property>
-		<name>hbase.hregion.preclose.flush.size</name>
-		<value>5242880</value>
-		<description>
-			If the memstores in a region are this size or larger when we go
-			to close, run a "pre-flush" to clear out memstores before we put up
-			the region closed flag and take the region offline.  On close,
-			a flush is run under the close flag to empty memory.  During
-			this time the region is offline and we are not taking on any writes.
-			If the memstore content is large, this flush could take a long time to
-			complete.  The preflush is meant to clean out the bulk of the memstore
-			before putting up the close flag and taking the region offline so the
-			flush that runs under the close flag has little to do.</description>
-	</property>
-	<property>
-		<name>hbase.hregion.memstore.block.multiplier</name>
-		<value>4</value>
-		<description>
-			Block updates if memstore has hbase.hregion.memstore.block.multiplier
-			times hbase.hregion.memstore.flush.size bytes.  Useful preventing
-			runaway memstore during spikes in update traffic.  Without an
-			upper-bound, memstore fills such that when it flushes the
-			resultant flush files take a long time to compact or split, or
-			worse, we OOME.</description>
-	</property>
-	<property>
-		<name>hbase.hregion.memstore.mslab.enabled</name>
-		<value>true</value>
-		<description>
-			Enables the MemStore-Local Allocation Buffer,
-			a feature which works to prevent heap fragmentation under
-			heavy write loads. This can reduce the frequency of stop-the-world
-			GC pauses on large heaps.</description>
-	</property>
-	<property>
-		<name>hbase.hregion.max.filesize</name>
-		<value>10737418240</value>
-		<description>
-			Maximum HStoreFile size. If any one of a column families' HStoreFiles has
-			grown to exceed this value, the hosting HRegion is split in two.</description>
-	</property>
-	<property>
-		<name>hbase.hregion.majorcompaction</name>
-		<value>604800000</value>
-		<description>The time (in miliseconds) between 'major' compactions of all
-			HStoreFiles in a region.  Default: Set to 7 days.  Major compactions tend to
-			happen exactly when you need them least so enable them such that they run at
-			off-peak for your deploy; or, since this setting is on a periodicity that is
-			unlikely to match your loading, run the compactions via an external
-			invocation out of a cron job or some such.</description>
-	</property>
-	<property>
-		<name>hbase.hregion.majorcompaction.jitter</name>
-		<value>0.50</value>
-		<description>Jitter outer bound for major compactions.
-			On each regionserver, we multiply the hbase.region.majorcompaction
-			interval by some random fraction that is inside the bounds of this
-			maximum.  We then add this + or - product to when the next
-			major compaction is to run.  The idea is that major compaction
-			does happen on every regionserver at exactly the same time.  The
-			smaller this number, the closer the compactions come together.</description>
-	</property>
-	<property>
-		<name>hbase.hstore.compactionThreshold</name>
-		<value>3</value>
-		<description>
-			If more than this number of HStoreFiles in any one HStore
-			(one HStoreFile is written per flush of memstore) then a compaction
-			is run to rewrite all HStoreFiles files as one.  Larger numbers
-			put off compaction but when it runs, it takes longer to complete.</description>
-	</property>
-	<property>
-		<name>hbase.hstore.flusher.count</name>
-		<value>2</value>
-		<description>
-			The number of flush threads. With less threads, the memstore flushes will be queued. With
-			more threads, the flush will be executed in parallel, increasing the hdfs load. This can
-			lead as well to more compactions.
-		</description>
-	</property>
-	<property>
-		<name>hbase.hstore.blockingStoreFiles</name>
-		<value>10</value>
-		<description>
-			If more than this number of StoreFiles in any one Store
-			(one StoreFile is written per flush of MemStore) then updates are
-			blocked for this HRegion until a compaction is completed, or
-			until hbase.hstore.blockingWaitTime has been exceeded.</description>
-	</property>
-	<property>
-		<name>hbase.hstore.blockingWaitTime</name>
-		<value>90000</value>
-		<description>
-			The time an HRegion will block updates for after hitting the StoreFile
-			limit defined by hbase.hstore.blockingStoreFiles.
-			After this time has elapsed, the HRegion will stop blocking updates even
-			if a compaction has not been completed.</description>
-	</property>
-	<property>
-		<name>hbase.hstore.compaction.max</name>
-		<value>10</value>
-		<description>Max number of HStoreFiles to compact per 'minor' compaction.</description>
-	</property>
-	<property>
-		<name>hbase.hstore.compaction.kv.max</name>
-		<value>10</value>
-		<description>How many KeyValues to read and then write in a batch when flushing
-			or compacting.  Do less if big KeyValues and problems with OOME.
-			Do more if wide, small rows.</description>
-	</property>
-	<property>
-		<name>hbase.hstore.time.to.purge.deletes</name>
-		<value>0</value>
-		<description>The amount of time to delay purging of delete markers with future timestamps. If
-			unset, or set to 0, all delete markers, including those with future timestamps, are purged
-			during the next major compaction. Otherwise, a delete marker is kept until the major compaction
-			which occurs after the marker's timestamp plus the value of this setting, in milliseconds.
-		</description>
-	</property>
-	<property>
-		<name>hbase.regionserver.majorcompaction.pagecache.drop</name>
-		<value>true</value>
-		<description>Specifies whether to drop pages read/written into the system page cache by
-			major compactions. Setting it to true helps prevent major compactions from
-			polluting the page cache, which is almost always required, especially for clusters
-			with low/moderate memory to storage ratio.</description>
-	</property>
-	<property>
-		<name>hbase.regionserver.minorcompaction.pagecache.drop</name>
-		<value>true</value>
-		<description>Specifies whether to drop pages read/written into the system page cache by
-			minor compactions. Setting it to true helps prevent minor compactions from
-			polluting the page cache, which is most beneficial on clusters with low
-			memory to storage ratio or very write heavy clusters. You may want to set it to
-			false under moderate to low write workload when bulk of the reads are
-			on the most recently written data.</description>
-	</property>
-	<property>
-		<name>hbase.storescanner.parallel.seek.enable</name>
-		<value>false</value>
-		<description>
-			Enables StoreFileScanner parallel-seeking in StoreScanner,
-			a feature which can reduce response latency under special conditions.</description>
-	</property>
-	<property>
-		<name>hbase.storescanner.parallel.seek.threads</name>
-		<value>10</value>
-		<description>
-			The default thread pool size if parallel-seeking feature enabled.</description>
-	</property>
-	<property>
-		<name>hfile.block.cache.size</name>
-		<value>0.4</value>
-		<description>Percentage of maximum heap (-Xmx setting) to allocate to block cache
-			used by HFile/StoreFile. Default of 0.4 means allocate 40%.
-			Set to 0 to disable but it's not recommended; you need at least
-			enough cache to hold the storefile indices.</description>
-	</property>
-	<property>
-		<name>hfile.block.index.cacheonwrite</name>
-		<value>false</value>
-		<description>This allows to put non-root multi-level index blocks into the block
-			cache at the time the index is being written.</description>
-	</property>
-	<property>
-		<name>hfile.index.block.max.size</name>
-		<value>131072</value>
-		<description>When the size of a leaf-level, intermediate-level, or root-level
-			index block in a multi-level block index grows to this size, the
-			block is written out and a new block is started.</description>
-	</property>
-	<property>
-		<name>hbase.bucketcache.ioengine</name>
-		<value></value>
-		<description>Where to store the contents of the bucketcache. One of: heap,
-			offheap, or file. If a file, set it to file:PATH_TO_FILE. See
-			http://hbase.apache.org/book.html#offheap.blockcache for more information.
-		</description>
-	</property>
-	<property>
-		<name>hbase.bucketcache.combinedcache.enabled</name>
-		<value>true</value>
-		<description>Whether or not the bucketcache is used in league with the LRU
-			on-heap block cache. In this mode, indices and blooms are kept in the LRU
-			blockcache and the data blocks are kept in the bucketcache.</description>
-	</property>
-	<property>
-		<name>hbase.bucketcache.size</name>
-		<value></value>
-		<description>A float that EITHER represents a percentage of total heap memory
-			size to give to the cache (if &lt; 1.0) OR, it is the total capacity in
-			megabytes of BucketCache. Default: 0.0</description>
-	</property>
-	<property>
-		<name>hbase.bucketcache.bucket.sizes</name>
-		<value></value>
-		<description>A comma-separated list of sizes for buckets for the bucketcache.
-			Can be multiple sizes. List block sizes in order from smallest to largest.
-			The sizes you use will depend on your data access patterns.
-			Must be a multiple of 256 else you will run into
-			'java.io.IOException: Invalid HFile block magic' when you go to read from cache.
-			If you specify no values here, then you pick up the default bucketsizes set
-			in code (See BucketAllocator#DEFAULT_BUCKET_SIZES).
-		</description>
-	</property>
-	<property>
-		<name>hfile.format.version</name>
-		<value>3</value>
-		<description>The HFile format version to use for new files.
-			Version 3 adds support for tags in hfiles (See http://hbase.apache.org/book.html#hbase.tags).
-			Distributed Log Replay requires that tags are enabled. Also see the configuration
-			'hbase.replication.rpc.codec'.
-		</description>
-	</property>
-	<property>
-		<name>hfile.block.bloom.cacheonwrite</name>
-		<value>false</value>
-		<description>Enables cache-on-write for inline blocks of a compound Bloom filter.</description>
-	</property>
-	<property>
-		<name>io.storefile.bloom.block.size</name>
-		<value>131072</value>
-		<description>The size in bytes of a single block ("chunk") of a compound Bloom
-			filter. This size is approximate, because Bloom blocks can only be
-			inserted at data block boundaries, and the number of keys per data
-			block varies.</description>
-	</property>
-	<property>
-		<name>hbase.rs.cacheblocksonwrite</name>
-		<value>false</value>
-		<description>Whether an HFile block should be added to the block cache when the
-			block is finished.</description>
-	</property>
-	<property>
-		<name>hbase.rpc.timeout</name>
-		<value>60000</value>
-		<description>This is for the RPC layer to define how long (millisecond) HBase client applications
-			take for a remote call to time out. It uses pings to check connections
-			but will eventually throw a TimeoutException.</description>
-	</property>
-	<property>
-		<name>hbase.client.operation.timeout</name>
-		<value>1200000</value>
-		<description>Operation timeout is a top-level restriction (millisecond) that makes sure a
-			blocking operation in Table will not be blocked more than this. In each operation, if rpc
-			request fails because of timeout or other reason, it will retry until success or throw
-			RetriesExhaustedException. But if the total time being blocking reach the operation timeout
-			before retries exhausted, it will break early and throw SocketTimeoutException.</description>
-	</property>
-	<property>
-		<name>hbase.cells.scanned.per.heartbeat.check</name>
-		<value>10000</value>
-		<description>The number of cells scanned in between heartbeat checks. Heartbeat
-			checks occur during the processing of scans to determine whether or not the
-			server should stop scanning in order to send back a heartbeat message to the
-			client. Heartbeat messages are used to keep the client-server connection alive
-			during long running scans. Small values mean that the heartbeat checks will
-			occur more often and thus will provide a tighter bound on the execution time of
-			the scan. Larger values mean that the heartbeat checks occur less frequently
-		</description>
-	</property>
-	<property>
-		<name>hbase.rpc.shortoperation.timeout</name>
-		<value>10000</value>
-		<description>This is another version of "hbase.rpc.timeout". For those RPC operation
-			within cluster, we rely on this configuration to set a short timeout limitation
-			for short operation. For example, short rpc timeout for region server's trying
-			to report to active master can benefit quicker master failover process.</description>
-	</property>
-	<property>
-		<name>hbase.ipc.client.tcpnodelay</name>
-		<value>true</value>
-		<description>Set no delay on rpc socket connections.  See
-			http://docs.oracle.com/javase/1.5.0/docs/api/java/net/Socket.html#getTcpNoDelay()</description>
-	</property>
-	<property>
-		<name>hbase.regionserver.hostname</name>
-		<value></value>
-		<description>This config is for experts: don't set its value unless you really know what you are doing.
-			When set to a non-empty value, this represents the (external facing) hostname for the underlying server.
-			See https://issues.apache.org/jira/browse/HBASE-12954 for details.</description>
-	</property>
-	<property>
-		<name>hbase.regionserver.hostname.disable.master.reversedns</name>
-		<value>false</value>
-		<description>This config is for experts: don't set its value unless you really know what you are doing.
-			When set to true, regionserver will use the current node hostname for the servername and HMaster will
-			skip reverse DNS lookup and use the hostname sent by regionserver instead. Note that this config and
-			hbase.regionserver.hostname are mutually exclusive. See https://issues.apache.org/jira/browse/HBASE-18226
-			for more details.</description>
-	</property>
-	<!-- The following properties configure authentication information for
-		 HBase processes when using Kerberos security.  There are no default
-		 values, included here for documentation purposes -->
-	<property>
-		<name>hbase.master.keytab.file</name>
-		<value></value>
-		<description>Full path to the kerberos keytab file to use for logging in
-			the configured HMaster server principal.</description>
-	</property>
-	<property>
-		<name>hbase.master.kerberos.principal</name>
-		<value></value>
-		<description>Ex. "hbase/_HOST@EXAMPLE.COM".  The kerberos principal name
-			that should be used to run the HMaster process.  The principal name should
-			be in the form: user/hostname@DOMAIN.  If "_HOST" is used as the hostname
-			portion, it will be replaced with the actual hostname of the running
-			instance.</description>
-	</property>
-	<property>
-		<name>hbase.regionserver.keytab.file</name>
-		<value></value>
-		<description>Full path to the kerberos keytab file to use for logging in
-			the configured HRegionServer server principal.</description>
-	</property>
-	<property>
-		<name>hbase.regionserver.kerberos.principal</name>
-		<value></value>
-		<description>Ex. "hbase/_HOST@EXAMPLE.COM".  The kerberos principal name
-			that should be used to run the HRegionServer process.  The principal name
-			should be in the form: user/hostname@DOMAIN.  If "_HOST" is used as the
-			hostname portion, it will be replaced with the actual hostname of the
-			running instance.  An entry for this principal must exist in the file
-			specified in hbase.regionserver.keytab.file</description>
-	</property>
-	<!-- Additional configuration specific to HBase security -->
-	<property>
-		<name>hadoop.policy.file</name>
-		<value>hbase-policy.xml</value>
-		<description>The policy configuration file used by RPC servers to make
-			authorization decisions on client requests.  Only used when HBase
-			security is enabled.</description>
-	</property>
-	<property>
-		<name>hbase.superuser</name>
-		<value></value>
-		<description>List of users or groups (comma-separated), who are allowed
-			full privileges, regardless of stored ACLs, across the cluster.
-			Only used when HBase security is enabled.</description>
-	</property>
-	<property>
-		<name>hbase.auth.key.update.interval</name>
-		<value>86400000</value>
-		<description>The update interval for master key for authentication tokens
-			in servers in milliseconds.  Only used when HBase security is enabled.</description>
-	</property>
-	<property>
-		<name>hbase.auth.token.max.lifetime</name>
-		<value>604800000</value>
-		<description>The maximum lifetime in milliseconds after which an
-			authentication token expires.  Only used when HBase security is enabled.</description>
-	</property>
-	<property>
-		<name>hbase.ipc.client.fallback-to-simple-auth-allowed</name>
-		<value>false</value>
-		<description>When a client is configured to attempt a secure connection, but attempts to
-			connect to an insecure server, that server may instruct the client to
-			switch to SASL SIMPLE (unsecure) authentication. This setting controls
-			whether or not the client will accept this instruction from the server.
-			When false (the default), the client will not allow the fallback to SIMPLE
-			authentication, and will abort the connection.</description>
-	</property>
-	<property>
-		<name>hbase.ipc.server.fallback-to-simple-auth-allowed</name>
-		<value>false</value>
-		<description>When a server is configured to require secure connections, it will
-			reject connection attempts from clients using SASL SIMPLE (unsecure) authentication.
-			This setting allows secure servers to accept SASL SIMPLE connections from clients
-			when the client requests.  When false (the default), the server will not allow the fallback
-			to SIMPLE authentication, and will reject the connection.  WARNING: This setting should ONLY
-			be used as a temporary measure while converting clients over to secure authentication.  It
-			MUST BE DISABLED for secure operation.</description>
-	</property>
-	<property>
-		<name>hbase.coprocessor.enabled</name>
-		<value>true</value>
-		<description>Enables or disables coprocessor loading. If 'false'
-			(disabled), any other coprocessor related configuration will be ignored.
-		</description>
-	</property>
-	<property>
-		<name>hbase.coprocessor.user.enabled</name>
-		<value>true</value>
-		<description>Enables or disables user (aka. table) coprocessor loading.
-			If 'false' (disabled), any table coprocessor attributes in table
-			descriptors will be ignored. If "hbase.coprocessor.enabled" is 'false'
-			this setting has no effect.
-		</description>
-	</property>
-	<property>
-		<name>hbase.coprocessor.region.classes</name>
-		<value></value>
-		<description>A comma-separated list of Coprocessors that are loaded by
-			default on all tables. For any override coprocessor method, these classes
-			will be called in order. After implementing your own Coprocessor, just put
-			it in HBase's classpath and add the fully qualified class name here.
-			A coprocessor can also be loaded on demand by setting HTableDescriptor.</description>
-	</property>
-	<property>
-		<name>hbase.rest.port</name>
-		<value>8080</value>
-		<description>The port for the HBase REST server.</description>
-	</property>
-	<property>
-		<name>hbase.rest.readonly</name>
-		<value>false</value>
-		<description>Defines the mode the REST server will be started in. Possible values are:
-			false: All HTTP methods are permitted - GET/PUT/POST/DELETE.
-			true: Only the GET method is permitted.</description>
-	</property>
-	<property>
-		<name>hbase.rest.threads.max</name>
-		<value>100</value>
-		<description>The maximum number of threads of the REST server thread pool.
-			Threads in the pool are reused to process REST requests. This
-			controls the maximum number of requests processed concurrently.
-			It may help to control the memory used by the REST server to
-			avoid OOM issues. If the thread pool is full, incoming requests
-			will be queued up and wait for some free threads.</description>
-	</property>
-	<property>
-		<name>hbase.rest.threads.min</name>
-		<value>2</value>
-		<description>The minimum number of threads of the REST server thread pool.
-			The thread pool always has at least these number of threads so
-			the REST server is ready to serve incoming requests.</description>
-	</property>
-	<property>
-		<name>hbase.rest.support.proxyuser</name>
-		<value>false</value>
-		<description>Enables running the REST server to support proxy-user mode.</description>
-	</property>
-	<property>
-		<name>hbase.defaults.for.version.skip</name>
-		<value>true</value>
-		<description>Set to true to skip the 'hbase.defaults.for.version' check.
-			Setting this to true can be useful in contexts other than
-			the other side of a maven generation; i.e. running in an
-			ide.  You'll want to set this boolean to true to avoid
-			seeing the RuntimException complaint: "hbase-default.xml file
-			seems to be for and old version of HBase (\${hbase.version}), this
-			version is X.X.X-SNAPSHOT"</description>
-	</property>
-	<property>
-		<name>hbase.coprocessor.abortonerror</name>
-		<value>true</value>
-		<description>Set to true to cause the hosting server (master or regionserver)
-			to abort if a coprocessor fails to load, fails to initialize, or throws an
-			unexpected Throwable object. Setting this to false will allow the server to
-			continue execution but the system wide state of the coprocessor in question
-			will become inconsistent as it will be properly executing in only a subset
-			of servers, so this is most useful for debugging only.</description>
-	</property>
-	<property>
-		<name>hbase.online.schema.update.enable</name>
-		<value>true</value>
-		<description>Set true to enable online schema changes.</description>
-	</property>
-	<property>
-		<name>hbase.table.lock.enable</name>
-		<value>true</value>
-		<description>Set to true to enable locking the table in zookeeper for schema change operations.
-			Table locking from master prevents concurrent schema modifications to corrupt table
-			state.</description>
-	</property>
-	<property>
-		<name>hbase.table.max.rowsize</name>
-		<value>1073741824</value>
-		<description>
-			Maximum size of single row in bytes (default is 1 Gb) for Get'ting
-			or Scan'ning without in-row scan flag set. If row size exceeds this limit
-			RowTooBigException is thrown to client.
-		</description>
-	</property>
-	<property>
-		<name>hbase.thrift.minWorkerThreads</name>
-		<value>16</value>
-		<description>The "core size" of the thread pool. New threads are created on every
-			connection until this many threads are created.</description>
-	</property>
-	<property>
-		<name>hbase.thrift.maxWorkerThreads</name>
-		<value>1000</value>
-		<description>The maximum size of the thread pool. When the pending request queue
-			overflows, new threads are created until their number reaches this number.
-			After that, the server starts dropping connections.</description>
-	</property>
-	<property>
-		<name>hbase.thrift.maxQueuedRequests</name>
-		<value>1000</value>
-		<description>The maximum number of pending Thrift connections waiting in the queue. If
-			there are no idle threads in the pool, the server queues requests. Only
-			when the queue overflows, new threads are added, up to
-			hbase.thrift.maxQueuedRequests threads.</description>
-	</property>
-	<property>
-		<name>hbase.regionserver.thrift.framed</name>
-		<value>false</value>
-		<description>Use Thrift TFramedTransport on the server side.
-			This is the recommended transport for thrift servers and requires a similar setting
-			on the client side. Changing this to false will select the default transport,
-			vulnerable to DoS when malformed requests are issued due to THRIFT-601.
-		</description>
-	</property>
-	<property>
-		<name>hbase.regionserver.thrift.framed.max_frame_size_in_mb</name>
-		<value>2</value>
-		<description>Default frame size when using framed transport</description>
-	</property>
-	<property>
-		<name>hbase.regionserver.thrift.compact</name>
-		<value>false</value>
-		<description>Use Thrift TCompactProtocol binary serialization protocol.</description>
-	</property>
-	<property>
-		<name>hbase.rootdir.perms</name>
-		<value>700</value>
-		<description>FS Permissions for the root directory in a secure(kerberos) setup.
-			When master starts, it creates the rootdir with this permissions or sets the permissions
-			if it does not match.</description>
-	</property>
-	<property>
-		<name>hbase.wal.dir.perms</name>
-		<value>700</value>
-		<description>FS Permissions for the root WAL directory in a secure(kerberos) setup.
-			When master starts, it creates the WAL dir with this permissions or sets the permissions
-			if it does not match.</description>
-	</property>
-	<property>
-		<name>hbase.data.umask.enable</name>
-		<value>false</value>
-		<description>Enable, if true, that file permissions should be assigned
-			to the files written by the regionserver</description>
-	</property>
-	<property>
-		<name>hbase.data.umask</name>
-		<value>000</value>
-		<description>File permissions that should be used to write data
-			files when hbase.data.umask.enable is true</description>
-	</property>
-	<property>
-		<name>hbase.snapshot.enabled</name>
-		<value>true</value>
-		<description>Set to true to allow snapshots to be taken / restored / cloned.</description>
-	</property>
-	<property>
-		<name>hbase.snapshot.restore.take.failsafe.snapshot</name>
-		<value>true</value>
-		<description>Set to true to take a snapshot before the restore operation.
-			The snapshot taken will be used in case of failure, to restore the previous state.
-			At the end of the restore operation this snapshot will be deleted</description>
-	</property>
-	<property>
-		<name>hbase.snapshot.restore.failsafe.name</name>
-		<value>hbase-failsafe-{snapshot.name}-{restore.timestamp}</value>
-		<description>Name of the failsafe snapshot taken by the restore operation.
-			You can use the {snapshot.name}, {table.name} and {restore.timestamp} variables
-			to create a name based on what you are restoring.</description>
-	</property>
-	<property>
-		<name>hbase.server.compactchecker.interval.multiplier</name>
-		<value>1000</value>
-		<description>The number that determines how often we scan to see if compaction is necessary.
-			Normally, compactions are done after some events (such as memstore flush), but if
-			region didn't receive a lot of writes for some time, or due to different compaction
-			policies, it may be necessary to check it periodically. The interval between checks is
-			hbase.server.compactchecker.interval.multiplier multiplied by
-			hbase.server.thread.wakefrequency.</description>
-	</property>
-	<property>
-		<name>hbase.lease.recovery.timeout</name>
-		<value>900000</value>
-		<description>How long we wait on dfs lease recovery in total before giving up.</description>
-	</property>
-	<property>
-		<name>hbase.lease.recovery.dfs.timeout</name>
-		<value>64000</value>
-		<description>How long between dfs recover lease invocations. Should be larger than the sum of
-			the time it takes for the namenode to issue a block recovery command as part of
-			datanode; dfs.heartbeat.interval and the time it takes for the primary
-			datanode, performing block recovery to timeout on a dead datanode; usually
-			dfs.client.socket-timeout. See the end of HBASE-8389 for more.</description>
-	</property>
-	<property>
-		<name>hbase.column.max.version</name>
-		<value>1</value>
-		<description>New column family descriptors will use this value as the default number of versions
-			to keep.</description>
-	</property>
-	<property>
-		<name>hbase.dfs.client.read.shortcircuit.buffer.size</name>
-		<value>131072</value>
-		<description>If the DFSClient configuration
-			dfs.client.read.shortcircuit.buffer.size is unset, we will
-			use what is configured here as the short circuit read default
-			direct byte buffer size. DFSClient native default is 1MB; HBase
-			keeps its HDFS files open so number of file blocks * 1MB soon
-			starts to add up and threaten OOME because of a shortage of
-			direct memory.  So, we set it down from the default.  Make
-			it > the default hbase block size set in the HColumnDescriptor
-			which is usually 64k.
-		</description>
-	</property>
-	<property>
-		<name>hbase.regionserver.checksum.verify</name>
-		<value>true</value>
-		<description>
-			If set to true (the default), HBase verifies the checksums for hfile
-			blocks. HBase writes checksums inline with the data when it writes out
-			hfiles. HDFS (as of this writing) writes checksums to a separate file
-			than the data file necessitating extra seeks.  Setting this flag saves
-			some on i/o.  Checksum verification by HDFS will be internally disabled
-			on hfile streams when this flag is set.  If the hbase-checksum verification
-			fails, we will switch back to using HDFS checksums (so do not disable HDFS
-			checksums!  And besides this feature applies to hfiles only, not to WALs).
-			If this parameter is set to false, then hbase will not verify any checksums,
-			instead it will depend on checksum verification being done in the HDFS client.
-		</description>
-	</property>
-	<property>
-		<name>hbase.hstore.bytes.per.checksum</name>
-		<value>16384</value>
-		<description>
-			Number of bytes in a newly created checksum chunk for HBase-level
-			checksums in hfile blocks.
-		</description>
-	</property>
-	<property>
-		<name>hbase.hstore.checksum.algorithm</name>
-		<value>CRC32C</value>
-		<description>
-			Name of an algorithm that is used to compute checksums. Possible values
-			are NULL, CRC32, CRC32C.
-		</description>
-	</property>
-
-	<property>
-		<name>hbase.client.scanner.max.result.size</name>
-		<value>2097152</value>
-		<description>Maximum number of bytes returned when calling a scanner's next method.
-			Note that when a single row is larger than this limit the row is still returned completely.
-			The default value is 2MB, which is good for 1ge networks.
-			With faster and/or high latency networks this value should be increased.
-		</description>
-	</property>
-
-	<property>
-		<name>hbase.server.scanner.max.result.size</name>
-		<value>104857600</value>
-		<description>Maximum number of bytes returned when calling a scanner's next method.
-			Note that when a single row is larger than this limit the row is still returned completely.
-			The default value is 100MB.
-			This is a safety setting to protect the server from OOM situations.
-		</description>
-	</property>
-
-	<property>
-		<name>hbase.status.published</name>
-		<value>false</value>
-		<description>
-			This setting activates the publication by the master of the status of the region server.
-			When a region server dies and its recovery starts, the master will push this information
-			to the client application, to let them cut the connection immediately instead of waiting
-			for a timeout.
-		</description>
-	</property>
-	<property>
-		<name>hbase.status.multicast.address.ip</name>
-		<value>226.1.1.3</value>
-		<description>
-			Multicast address to use for the status publication by multicast.
-		</description>
-	</property>
-	<property>
-		<name>hbase.status.multicast.address.port</name>
-		<value>16100</value>
-		<description>
-			Multicast port to use for the status publication by multicast.
-		</description>
-	</property>
-
-	<property>
-		<name>hbase.dynamic.jars.dir</name>
-		<value>${hbase.rootdir}/lib</value>
-		<description>
-			The directory from which the custom filter/co-processor jars can be loaded
-			dynamically by the region server without the need to restart. However,
-			an already loaded filter/co-processor class would not be un-loaded. See
-			HBASE-1936 for more details.
-		</description>
-	</property>
-	<property>
-		<name>hbase.security.authentication</name>
-		<value>simple</value>
-		<description>
-			Controls whether or not secure authentication is enabled for HBase.
-			Possible values are 'simple' (no authentication), and 'kerberos'.
-		</description>
-	</property>
-	<property>
-		<name>hbase.rest.csrf.enabled</name>
-		<value>false</value>
-		<description>
-			Set to true to enable protection against cross-site request forgery (CSRF)
-		</description>
-	</property>
-	<property>
-		<name>hbase.rest-csrf.browser-useragents-regex</name>
-		<value>^Mozilla.*,^Opera.*</value>
-		<description>
-			A comma-separated list of regular expressions used to match against an HTTP
-			request's User-Agent header when protection against cross-site request
-			forgery (CSRF) is enabled for REST server by setting
-			hbase.rest.csrf.enabled to true.  If the incoming User-Agent matches
-			any of these regular expressions, then the request is considered to be sent
-			by a browser, and therefore CSRF prevention is enforced.  If the request's
-			User-Agent does not match any of these regular expressions, then the request
-			is considered to be sent by something other than a browser, such as scripted
-			automation.  In this case, CSRF is not a potential attack vector, so
-			the prevention is not enforced.  This helps achieve backwards-compatibility
-			with existing automation that has not been updated to send the CSRF
-			prevention header.
-		</description>
-	</property>
-	<property>
-		<name>hbase.security.exec.permission.checks</name>
-		<value>false</value>
-		<description>
-			If this setting is enabled and ACL based access control is active (the
-			AccessController coprocessor is installed either as a system coprocessor
-			or on a table as a table coprocessor) then you must grant all relevant
-			users EXEC privilege if they require the ability to execute coprocessor
-			endpoint calls. EXEC privilege, like any other permission, can be
-			granted globally to a user, or to a user on a per table or per namespace
-			basis. For more information on coprocessor endpoints, see the coprocessor
-			section of the HBase online manual. For more information on granting or
-			revoking permissions using the AccessController, see the security
-			section of the HBase online manual.
-		</description>
-	</property>
-	<property>
-		<name>hbase.regionserver.storefile.refresh.period</name>
-		<value>0</value>
-		<description>
-			The period (in milliseconds) for refreshing the store files for the secondary regions. 0
-			means this feature is disabled. Secondary regions sees new files (from flushes and
-			compactions) from primary once the secondary region refreshes the list of files in the
-			region (there is no notification mechanism). But too frequent refreshes might cause
-			extra Namenode pressure. If the files cannot be refreshed for longer than HFile TTL
-			(hbase.master.hfilecleaner.ttl) the requests are rejected. Configuring HFile TTL to a larger
-			value is also recommended with this setting.
-		</description>
-	</property>
-	<property>
-		<name>hbase.region.replica.replication.enabled</name>
-		<value>false</value>
-		<description>
-			Whether asynchronous WAL replication to the secondary region replicas is enabled or not.
-			If this is enabled, a replication peer named "region_replica_replication" will be created
-			which will tail the logs and replicate the mutatations to region replicas for tables that
-			have region replication > 1. If this is enabled once, disabling this replication also
-			requires disabling the replication peer using shell or ReplicationAdmin java class.
-			Replication to secondary region replicas works over standard inter-cluster replication.
-			So replication, if disabled explicitly, also has to be enabled by setting "hbase.replication"
-			to true for this feature to work.
-		</description>
-	</property>
-	<property>
-		<name>hbase.security.visibility.mutations.checkauths</name>
-		<value>false</value>
-		<description>
-			This property if enabled, will check whether the labels in the visibility expression are associated
-			with the user issuing the mutation
-		</description>
-	</property>
-	<property>
-		<name>hbase.http.max.threads</name>
-		<value>10</value>
-		<description>
-			The maximum number of threads that the HTTP Server will create in its
-			ThreadPool.
-		</description>
-	</property>
-	<property>
-		<name>hbase.replication.source.maxthreads</name>
-		<value>10</value>
-		<description>
-			The maximum number of threads any replication source will use for
-			shipping edits to the sinks in parallel. This also limits the number of
-			chunks each replication batch is broken into.
-			Larger values can improve the replication throughput between the master and
-			slave clusters. The default of 10 will rarely need to be changed.
-		</description>
-	</property>
-	<!-- Static Web User Filter properties. -->
-	<property>
-		<description>
-			The user name to filter as, on static web filters
-			while rendering content. An example use is the HDFS
-			web UI (user to be used for browsing files).
-		</description>
-		<name>hbase.http.staticuser.user</name>
-		<value>dr.stack</value>
-	</property>
-	<property>
-		<name>hbase.regionserver.handler.abort.on.error.percent</name>
-		<value>0.5</value>
-		<description>The percent of region server RPC threads failed to abort RS.
-			-1 Disable aborting; 0 Abort if even a single handler has died;
-			0.x Abort only when this percent of handlers have died;
-			1 Abort only all of the handers have died.</description>
-	</property>
-	<property>
-		<name>hbase.snapshot.master.timeout.millis</name>
-		<value>300000</value>
-		<description>
-			Timeout for master for the snapshot procedure execution
-		</description>
-	</property>
-	<property>
-		<name>hbase.snapshot.region.timeout</name>
-		<value>300000</value>
-		<description>
-			Timeout for regionservers to keep threads in snapshot request pool waiting
-		</description>
-	</property>
-	<property>
-		<name>hbase.rpc.rows.warning.threshold</name>
-		<value>5000</value>
-		<description>
-			Number of rows in a batch operation above which a warning will be logged.
-		</description>
-	</property>
-
-	<!--NOTE: HBase client try to load the class that configured in hbase-default.xml. -->
-	<!--But actually all these classes were already shaded and can't be loaded by those default name, -->
-	<!--so the following classes are Flink shaded classes.-->
-	<property>
-		<name>hbase.master.logcleaner.plugins</name>
-		<value>org.apache.flink.hbase.shaded.org.apache.hadoop.hbase.master.cleaner.TimeToLiveLogCleaner</value>
-		<description>A comma-separated list of BaseLogCleanerDelegate invoked by
-			the LogsCleaner service. These WAL cleaners are called in order,
-			so put the cleaner that prunes the most files in front. To
-			implement your own BaseLogCleanerDelegate, just put it in HBase's classpath
-			and add the fully qualified class name here. Always add the above
-			default log cleaners in the list.
-		</description>
-	</property>
-	<property>
-		<name>hbase.master.hfilecleaner.plugins</name>
-		<value>org.apache.flink.hbase.shaded.org.apache.hadoop.hbase.master.cleaner.TimeToLiveHFileCleaner</value>
-		<description>A comma-separated list of BaseHFileCleanerDelegate invoked by
-			the HFileCleaner service. These HFiles cleaners are called in order,
-			so put the cleaner that prunes the most files in front. To
-			implement your own BaseHFileCleanerDelegate, just put it in HBase's classpath
-			and add the fully qualified class name here. Always add the above
-			default log cleaners in the list as they will be overwritten in
-			hbase-site.xml.
-		</description>
-	</property>
-	<property>
-		<name>hbase.regionserver.hlog.reader.impl</name>
-		<value>org.apache.flink.hbase.shaded.org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader</value>
-		<description>The WAL file reader implementation.</description>
-	</property>
-	<property>
-		<name>hbase.regionserver.hlog.writer.impl</name>
-		<value>org.apache.flink.hbase.shaded.org.apache.hadoop.hbase.regionserver.wal.ProtobufLogWriter</value>
-		<description>The WAL file writer implementation.</description>
-	</property>
-	<property>
-		<name>hbase.regionserver.region.split.policy</name>
-		<value>org.apache.flink.hbase.shaded.org.apache.hadoop.hbase.regionserver.IncreasingToUpperBoundRegionSplitPolicy</value>
-		<description>
-			A split policy determines when a region should be split. The various other split policies that
-			are available currently are BusyRegionSplitPolicy, ConstantSizeRegionSplitPolicy,
-			DisabledRegionSplitPolicy, DelimitedKeyPrefixRegionSplitPolicy, KeyPrefixRegionSplitPolicy etc.
-		</description>
-	</property>
-	<property>
-		<name>hbase.status.publisher.class</name>
-		<value>org.apache.flink.hbase.shaded.org.apache.hadoop.hbase.master.ClusterStatusPublisher$MulticastPublisher</value>
-		<description>
-			Implementation of the status publication with a multicast message.
-		</description>
-	</property>
-	<property>
-		<name>hbase.status.listener.class</name>
-		<value>org.apache.flink.hbase.shaded.org.apache.hadoop.hbase.client.ClusterStatusListener$MulticastListener</value>
-		<description>
-			Implementation of the status listener with a multicast message.
-		</description>
-	</property>
-	<property>
-		<name>hbase.rest.filter.classes</name>
-		<value>org.apache.flink.hbase.shaded.org.apache.hadoop.hbase.rest.filter.GzipFilter</value>
-		<description>
-			Servlet filters for REST service.
-		</description>
-	</property>
-	<property>
-		<name>hbase.master.loadbalancer.class</name>
-		<value>org.apache.flink.hbase.shaded.org.apache.hadoop.hbase.master.balancer.StochasticLoadBalancer</value>
-		<description>
-			Class used to execute the regions balancing when the period occurs.
-			See the class comment for more on how it works
-			http://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.html
-			It replaces the DefaultLoadBalancer as the default (since renamed
-			as the SimpleLoadBalancer).
-		</description>
-	</property>
-	<property>
-		<name>hbase.coordinated.state.manager.class</name>
-		<value>org.apache.flink.hbase.shaded.org.apache.hadoop.hbase.coordination.ZkCoordinatedStateManager</value>
-		<description>Fully qualified name of class implementing coordinated state manager.</description>
-	</property>
-	<property>
-		<name>hbase.http.filter.initializers</name>
-		<value>org.apache.flink.hbase.shaded.org.apache.hadoop.hbase.http.lib.StaticUserWebFilter</value>
-		<description>
-			A comma separated list of class names. Each class in the list must extend
-			org.apache.hadoop.hbase.http.FilterInitializer. The corresponding Filter will
-			be initialized. Then, the Filter will be applied to all user facing jsp
-			and servlet web pages.
-			The ordering of the list defines the ordering of the filters.
-			The default StaticUserWebFilter add a user principal as defined by the
-			hbase.http.staticuser.user property.
-		</description>
-	</property>
-	<property>
-		<name>hbase.replication.rpc.codec</name>
-		<value>org.apache.flink.hbase.shaded.org.apache.hadoop.hbase.codec.KeyValueCodecWithTags</value>
-		<description>
-			The codec that is to be used when replication is enabled so that
-			the tags are also replicated. This is used along with HFileV3 which
-			supports tags in them. If tags are not used or if the hfile version used
-			is HFileV2 then KeyValueCodec can be used as the replication codec. Note that
-			using KeyValueCodecWithTags for replication when there are no tags causes no harm.
-		</description>
-	</property>
-	<property>
-		<name>hbase.master.normalizer.class</name>
-		<value>org.apache.flink.hbase.shaded.org.apache.hadoop.hbase.master.normalizer.SimpleRegionNormalizer</value>
-		<description>
-			Class used to execute the region normalization when the period occurs.
-			See the class comment for more on how it works
-			http://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/master/normalizer/SimpleRegionNormalizer.html
-		</description>
-	</property>
-</configuration>
diff --git a/flink-connectors/flink-sql-connector-hbase-2.2/pom.xml b/flink-connectors/flink-sql-connector-hbase-2.2/pom.xml
deleted file mode 100644
index d057e6e..0000000
--- a/flink-connectors/flink-sql-connector-hbase-2.2/pom.xml
+++ /dev/null
@@ -1,170 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-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.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0"
-		 xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-		 xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-	<parent>
-		<artifactId>flink-connectors</artifactId>
-		<groupId>org.apache.flink</groupId>
-		<version>1.19-SNAPSHOT</version>
-	</parent>
-	<modelVersion>4.0.0</modelVersion>
-
-	<artifactId>flink-sql-connector-hbase-2.2</artifactId>
-	<name>Flink : Connectors : SQL : HBase 2.2</name>
-
-	<packaging>jar</packaging>
-
-	<properties>
-		<japicmp.skip>true</japicmp.skip>
-	</properties>
-
-	<dependencies>
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-connector-hbase-2.2</artifactId>
-			<version>${project.version}</version>
-			<optional>${flink.markBundledAsOptional}</optional>
-		</dependency>
-	</dependencies>
-
-	<dependencyManagement>
-		<dependencies>
-			<dependency>
-				<!-- HBase only works with Zookeper 3.4 -->
-				<groupId>org.apache.zookeeper</groupId>
-				<artifactId>zookeeper</artifactId>
-				<version>3.4.14</version>
-			</dependency>
-		</dependencies>
-	</dependencyManagement>
-
-	<build>
-		<plugins>
-			<plugin>
-				<groupId>org.apache.maven.plugins</groupId>
-				<artifactId>maven-shade-plugin</artifactId>
-				<executions>
-					<execution>
-						<id>shade-flink</id>
-						<phase>package</phase>
-						<goals>
-							<goal>shade</goal>
-						</goals>
-						<configuration>
-							<!--
-							Make the file hbase-default.xml under flink-sql-connector-hbase-2.2/src/main/resources as the
-							hbase-default.xml in the shaded target jar here, because we don't want to check the hbase
-							version at client side. Also we don't need the extra default configs keys.
-							-->
-							<transformers>
-								<transformer
-										implementation="org.apache.maven.plugins.shade.resource.IncludeResourceTransformer">
-									<resource>hbase-default.xml</resource>
-									<file>hbase-default.xml</file>
-								</transformer>
-							</transformers>
-							<artifactSet>
-								<includes>
-									<include>org.apache.flink:flink-connector-base</include>
-									<include>org.apache.flink:flink-connector-hbase-base</include>
-									<include>org.apache.flink:flink-connector-hbase-2.2</include>
-									<include>org.apache.hbase:hbase-*</include>
-									<include>org.apache.hbase.thirdparty:hbase-shaded-*</include>
-									<include>org.apache.zookeeper:zookeeper</include>
-									<include>org.apache.htrace:htrace-core4</include>
-									<include>com.google.protobuf:protobuf-java</include>
-									<include>commons-codec:commons-codec</include>
-									<include>org.apache.commons:commons-crypto</include>
-									<include>org.apache.commons:commons-lang3</include>
-									<include>io.netty:netty-all</include>
-									<include>io.dropwizard.metrics:metrics-core</include>
-								</includes>
-								<excludes>
-									<exclude>org.apache.hbase:hbase-metrics*</exclude>
-									<exclude>org.apache.hbase:hbase-server*</exclude>
-									<exclude>org.apache.hbase:hbase-hadoop*-compat</exclude>
-								</excludes>
-							</artifactSet>
-							<filters>
-								<filter>
-									<artifact>*:*</artifact>
-									<excludes>
-										<!-- excluded all these files for a clean jar -->
-										<exclude>META-INF/services/com.fasterxml.**</exclude>
-										<exclude>META-INF/services/org.apache.hadoop.**</exclude>
-										<exclude>META-INF/services/javax.**</exclude>
- 										<exclude>digesterRules.xml</exclude>
-										<exclude>properties.dtd</exclude>
-										<exclude>PropertyList-1.0.dtd</exclude>
-										<exclude>LICENSE.txt</exclude>
-										<exclude>*.proto</exclude>
-										<exclude>protobuf/*</exclude>
-									</excludes>
-								</filter>
-							</filters>
-							<relocations>
-								<!-- Force relocation of all HBase dependencies. -->
-								<relocation>
-									<pattern>org.apache.zookeeper</pattern>
-									<shadedPattern>org.apache.flink.hbase.shaded.org.apache.zookeeper</shadedPattern>
-								</relocation>
-								<relocation>
-									<pattern>org.apache.htrace</pattern>
-									<shadedPattern>org.apache.flink.hbase.shaded.org.apache.htrace</shadedPattern>
-								</relocation>
-								<relocation>
-									<pattern>com.google</pattern>
-									<shadedPattern>org.apache.flink.hbase.shaded.com.google</shadedPattern>
-								</relocation>
-								<relocation>
-									<pattern>com.yammer.metrics</pattern>
-									<shadedPattern>org.apache.flink.hbase.shaded.com.yammer.metrics</shadedPattern>
-								</relocation>
-								<relocation>
-									<pattern>org.apache.commons</pattern>
-									<shadedPattern>org.apache.flink.hbase.shaded.org.apache.commons</shadedPattern>
-								</relocation>
-								<relocation>
-									<pattern>org.apache.jute</pattern>
-									<shadedPattern>org.apache.flink.hbase.shaded.org.apache.jute</shadedPattern>
-								</relocation>
-								<relocation>
-									<pattern>io.netty</pattern>
-									<shadedPattern>org.apache.flink.hbase.shaded.io.netty</shadedPattern>
-								</relocation>
-								<relocation>
-									<pattern>org.apache.hadoop.hbase</pattern>
-									<shadedPattern>org.apache.flink.hbase.shaded.org.apache.hadoop.hbase</shadedPattern>
-									<!-- HBase client uses shaded KeyValueCodec to encode data and put the class name in
- 										the header of request, the HBase region server can not load the shaded
-										KeyValueCodec class when decode the data, so we exclude them here. -->
-									<excludes>
-										<exclude>org.apache.hadoop.hbase.codec.*</exclude>
-									</excludes>
-								</relocation>
-							</relocations>
-						</configuration>
-					</execution>
-				</executions>
-			</plugin>
-		</plugins>
-	</build>
-</project>
diff --git a/flink-connectors/flink-sql-connector-hbase-2.2/src/main/resources/META-INF/NOTICE b/flink-connectors/flink-sql-connector-hbase-2.2/src/main/resources/META-INF/NOTICE
deleted file mode 100644
index b6225d5..0000000
--- a/flink-connectors/flink-sql-connector-hbase-2.2/src/main/resources/META-INF/NOTICE
+++ /dev/null
@@ -1,62 +0,0 @@
-flink-sql-connector-hbase-2.2
-Copyright 2014-2023 The Apache Software Foundation
-
-This product includes software developed at
-The Apache Software Foundation (http://www.apache.org/).
-
-This project bundles the following dependencies under the Apache Software License 2.0. (http://www.apache.org/licenses/LICENSE-2.0.txt)
-
- - commons-codec:commons-codec:1.15
- - io.netty:netty-all:4.1.70.Final
- - io.dropwizard.metrics:metrics-core:3.2.6
- - org.apache.commons:commons-crypto:1.0.0
- - org.apache.commons:commons-lang3:3.12.0
- - org.apache.hbase:hbase-client:2.2.3
- - org.apache.hbase:hbase-common:2.2.3
- - org.apache.hbase:hbase-protocol:2.2.3
- - org.apache.hbase:hbase-protocol-shaded:2.2.3
- - org.apache.hbase.thirdparty:hbase-shaded-protobuf:2.2.1
- - org.apache.hbase.thirdparty:hbase-shaded-miscellaneous:2.2.1
- - org.apache.hbase.thirdparty:hbase-shaded-netty:2.2.1
- - org.apache.htrace:htrace-core4:4.2.0-incubating
- - org.apache.zookeeper:zookeeper:3.4.14
-
-This project bundles the following dependencies under the BSD license.
-See bundled license files for details.
-
- - com.google.protobuf:protobuf-java:2.5.0
-
-The bundled Apache HTrace org.apache.htrace:htrace-core4 dependency bundles the following dependencies under
-the Apache Software License 2.0 (http://www.apache.org/licenses/LICENSE-2.0.txt)
-
- - com.fasterxml.jackson.core:jackson-annotations:2.4.0
- - com.fasterxml.jackson.core:jackson-core:2.4.0
- - com.fasterxml.jackson.core:jackson-databind:2.4.0
- - commons-logging:commons-logging:1.1.1
-
-The bundled Apache HBase Patched & Relocated (Shaded) Protobuf org.apache.hbase.thirdparty:hbase-shaded-protobuf
-dependency bundles the following dependencies under the BSD license. See bundled license files for details.
-
- - com.google.protobuf:protobuf-java:3.7.1
-
-The bundled Apache HBase Relocated (Shaded) Third-party Miscellaneous Libs
-org.apache.hbase.thirdparty:hbase-shaded-miscellaneous dependency bundles the following dependencies under
-the Apache Software License 2.0 (http://www.apache.org/licenses/LICENSE-2.0.txt)
-
- - com.google.code.gson:gson:2.8.5
- - com.google.guava:failureaccess:1.0.1
- - com.google.guava:guava:27.1-jre
- - commons-cli:commons-cli:1.4
- - org.apache.commons:commons-collections4:4.3
-
-The bundled Apache HBase Relocated (Shaded) Third-party Miscellaneous Libs
-org.apache.hbase.thirdparty:hbase-shaded-miscellaneous dependency bundles the following dependencies under
-the BSD license. See bundled license files for details.
-
- - com.google.protobuf:protobuf-java-util:3.7.1
-
-The bundled Apache HBase Relocated (Shaded) Netty Libs org.apache.hbase.thirdparty:hbase-shaded-netty
-dependency bundles the following dependencies under
-the Apache Software License 2.0 (http://www.apache.org/licenses/LICENSE-2.0.txt)
-
- - io.netty:netty-all:4.1.34.Final
diff --git a/flink-connectors/flink-sql-connector-hbase-2.2/src/main/resources/META-INF/licenses/LICENSE.protobuf b/flink-connectors/flink-sql-connector-hbase-2.2/src/main/resources/META-INF/licenses/LICENSE.protobuf
deleted file mode 100644
index 19b305b..0000000
--- a/flink-connectors/flink-sql-connector-hbase-2.2/src/main/resources/META-INF/licenses/LICENSE.protobuf
+++ /dev/null
@@ -1,32 +0,0 @@
-Copyright 2008 Google Inc.  All rights reserved.
-
-Redistribution and use in source and binary forms, with or without
-modification, are permitted provided that the following conditions are
-met:
-
-    * Redistributions of source code must retain the above copyright
-notice, this list of conditions and the following disclaimer.
-    * Redistributions in binary form must reproduce the above
-copyright notice, this list of conditions and the following disclaimer
-in the documentation and/or other materials provided with the
-distribution.
-    * Neither the name of Google Inc. nor the names of its
-contributors may be used to endorse or promote products derived from
-this software without specific prior written permission.
-
-THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
-"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
-LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
-A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
-OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
-SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
-LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
-DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
-THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
-(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
-OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
-
-Code generated by the Protocol Buffer compiler is owned by the owner
-of the input file used when generating it.  This code is not
-standalone and requires a support library to be linked with it.  This
-support library is itself covered by the above license.
diff --git a/flink-connectors/flink-sql-connector-hbase-2.2/src/main/resources/hbase-default.xml b/flink-connectors/flink-sql-connector-hbase-2.2/src/main/resources/hbase-default.xml
deleted file mode 100644
index 5251f12..0000000
--- a/flink-connectors/flink-sql-connector-hbase-2.2/src/main/resources/hbase-default.xml
+++ /dev/null
@@ -1,1817 +0,0 @@
-<?xml version="1.0"?>
-<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
-<!--
-/**
- * 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.
- */
--->
-
-<!--
-OVERVIEW
-
-The important configs. are listed near the top.  You should change
-at least the setting for hbase.tmp.dir.  Other settings will change
-dependent on whether you are running hbase in standalone mode or
-distributed.  See the hbase reference guide for requirements and
-guidance making configuration.
-
-This file does not contain all possible configurations.  The file would be
-much larger if it carried everything. The absent configurations will only be
-found through source code reading.  The idea is that such configurations are
-exotic and only those who would go to the trouble of reading a particular
-section in the code would be knowledgeable or invested enough in ever wanting
-to alter such configurations, so we do not list them here.  Listing all
-possible configurations would overwhelm and obscure the important.
--->
-
-<configuration>
-	<!--Configs you will likely change are listed here at the top of the file.
-    -->
-	<property >
-		<name>hbase.tmp.dir</name>
-		<value>${java.io.tmpdir}/hbase-${user.name}</value>
-		<description>Temporary directory on the local filesystem.
-			Change this setting to point to a location more permanent
-			than '/tmp', the usual resolve for java.io.tmpdir, as the
-			'/tmp' directory is cleared on machine restart.</description>
-	</property>
-	<property >
-		<name>hbase.rootdir</name>
-		<value>${hbase.tmp.dir}/hbase</value>
-		<description>The directory shared by region servers and into
-			which HBase persists.  The URL should be 'fully-qualified'
-			to include the filesystem scheme.  For example, to specify the
-			HDFS directory '/hbase' where the HDFS instance's namenode is
-			running at namenode.example.org on port 9000, set this value to:
-			hdfs://namenode.example.org:9000/hbase.  By default, we write
-			to whatever ${hbase.tmp.dir} is set too -- usually /tmp --
-			so change this configuration or else all data will be lost on
-			machine restart.</description>
-	</property>
-	<property >
-		<name>hbase.cluster.distributed</name>
-		<value>false</value>
-		<description>The mode the cluster will be in. Possible values are
-			false for standalone mode and true for distributed mode.  If
-			false, startup will run all HBase and ZooKeeper daemons together
-			in the one JVM.</description>
-	</property>
-	<property>
-		<name>hbase.zookeeper.quorum</name>
-		<value>localhost</value>
-		<description>Comma separated list of servers in the ZooKeeper ensemble
-			(This config. should have been named hbase.zookeeper.ensemble).
-			For example, "host1.mydomain.com,host2.mydomain.com,host3.mydomain.com".
-			By default this is set to localhost for local and pseudo-distributed modes
-			of operation. For a fully-distributed setup, this should be set to a full
-			list of ZooKeeper ensemble servers. If HBASE_MANAGES_ZK is set in hbase-env.sh
-			this is the list of servers which hbase will start/stop ZooKeeper on as
-			part of cluster start/stop.  Client-side, we will take this list of
-			ensemble members and put it together with the hbase.zookeeper.property.clientPort
-			config. and pass it into zookeeper constructor as the connectString
-			parameter.</description>
-	</property>
-	<!--The above are the important configurations for getting hbase up
-      and running -->
-
-	<property>
-		<name>zookeeper.recovery.retry.maxsleeptime</name>
-		<value>60000</value>
-		<description>Max sleep time before retry zookeeper operations in milliseconds,
-			a max time is needed here so that sleep time won't grow unboundedly
-		</description>
-	</property>
-	<property>
-		<name>hbase.local.dir</name>
-		<value>${hbase.tmp.dir}/local/</value>
-		<description>Directory on the local filesystem to be used
-			as a local storage.</description>
-	</property>
-
-	<!--Master configurations-->
-	<property >
-		<name>hbase.master.port</name>
-		<value>16000</value>
-		<description>The port the HBase Master should bind to.</description>
-	</property>
-	<property>
-		<name>hbase.master.info.port</name>
-		<value>16010</value>
-		<description>The port for the HBase Master web UI.
-			Set to -1 if you do not want a UI instance run.</description>
-	</property>
-	<property>
-		<name>hbase.master.info.bindAddress</name>
-		<value>0.0.0.0</value>
-		<description>The bind address for the HBase Master web UI
-		</description>
-	</property>
-	<property>
-		<name>hbase.master.logcleaner.ttl</name>
-		<value>600000</value>
-		<description>How long a WAL remain in the archive ({hbase.rootdir}/oldWALs) directory,
-			after which it will be cleaned by a Master thread. The value is in milliseconds.</description>
-	</property>
-	<property>
-		<name>hbase.master.procedurewalcleaner.ttl</name>
-		<value>604800000</value>
-		<description>How long a Procedure WAL will remain in the
-			archive directory, after which it will be cleaned
-			by a Master thread. The value is in milliseconds.</description>
-	</property>
-	<property>
-		<name>hbase.master.infoserver.redirect</name>
-		<value>true</value>
-		<description>Whether or not the Master listens to the Master web
-			UI port (hbase.master.info.port) and redirects requests to the web
-			UI server shared by the Master and RegionServer. Config. makes
-			sense when Master is serving Regions (not the default).</description>
-	</property>
-	<property>
-		<name>hbase.master.fileSplitTimeout</name>
-		<value>600000</value>
-		<description>Splitting a region, how long to wait on the file-splitting
-			step before aborting the attempt. Default: 600000. This setting used
-			to be known as hbase.regionserver.fileSplitTimeout in hbase-1.x.
-			Split is now run master-side hence the rename (If a
-			'hbase.master.fileSplitTimeout' setting found, will use it to
-			prime the current 'hbase.master.fileSplitTimeout'
-			Configuration.</description>
-	</property>
-
-	<!--RegionServer configurations-->
-	<property>
-		<name>hbase.regionserver.port</name>
-		<value>16020</value>
-		<description>The port the HBase RegionServer binds to.</description>
-	</property>
-	<property>
-		<name>hbase.regionserver.info.port</name>
-		<value>16030</value>
-		<description>The port for the HBase RegionServer web UI
-			Set to -1 if you do not want the RegionServer UI to run.</description>
-	</property>
-	<property>
-		<name>hbase.regionserver.info.bindAddress</name>
-		<value>0.0.0.0</value>
-		<description>The address for the HBase RegionServer web UI</description>
-	</property>
-	<property>
-		<name>hbase.regionserver.info.port.auto</name>
-		<value>false</value>
-		<description>Whether or not the Master or RegionServer
-			UI should search for a port to bind to. Enables automatic port
-			search if hbase.regionserver.info.port is already in use.
-			Useful for testing, turned off by default.</description>
-	</property>
-	<property>
-		<name>hbase.regionserver.handler.count</name>
-		<value>30</value>
-		<description>Count of RPC Listener instances spun up on RegionServers.
-			Same property is used by the Master for count of master handlers.
-			Too many handlers can be counter-productive. Make it a multiple of
-			CPU count. If mostly read-only, handlers count close to cpu count
-			does well. Start with twice the CPU count and tune from there.</description>
-	</property>
-	<property>
-		<name>hbase.ipc.server.callqueue.handler.factor</name>
-		<value>0.1</value>
-		<description>Factor to determine the number of call queues.
-			A value of 0 means a single queue shared between all the handlers.
-			A value of 1 means that each handler has its own queue.</description>
-	</property>
-	<property>
-		<name>hbase.ipc.server.callqueue.read.ratio</name>
-		<value>0</value>
-		<description>Split the call queues into read and write queues.
-			The specified interval (which should be between 0.0 and 1.0)
-			will be multiplied by the number of call queues.
-			A value of 0 indicate to not split the call queues, meaning that both read and write
-			requests will be pushed to the same set of queues.
-			A value lower than 0.5 means that there will be less read queues than write queues.
-			A value of 0.5 means there will be the same number of read and write queues.
-			A value greater than 0.5 means that there will be more read queues than write queues.
-			A value of 1.0 means that all the queues except one are used to dispatch read requests.
-
-			Example: Given the total number of call queues being 10
-			a read.ratio of 0 means that: the 10 queues will contain both read/write requests.
-			a read.ratio of 0.3 means that: 3 queues will contain only read requests
-			and 7 queues will contain only write requests.
-			a read.ratio of 0.5 means that: 5 queues will contain only read requests
-			and 5 queues will contain only write requests.
-			a read.ratio of 0.8 means that: 8 queues will contain only read requests
-			and 2 queues will contain only write requests.
-			a read.ratio of 1 means that: 9 queues will contain only read requests
-			and 1 queues will contain only write requests.
-		</description>
-	</property>
-	<property>
-		<name>hbase.ipc.server.callqueue.scan.ratio</name>
-		<value>0</value>
-		<description>Given the number of read call queues, calculated from the total number
-			of call queues multiplied by the callqueue.read.ratio, the scan.ratio property
-			will split the read call queues into small-read and long-read queues.
-			A value lower than 0.5 means that there will be less long-read queues than short-read queues.
-			A value of 0.5 means that there will be the same number of short-read and long-read queues.
-			A value greater than 0.5 means that there will be more long-read queues than short-read queues
-			A value of 0 or 1 indicate to use the same set of queues for gets and scans.
-
-			Example: Given the total number of read call queues being 8
-			a scan.ratio of 0 or 1 means that: 8 queues will contain both long and short read requests.
-			a scan.ratio of 0.3 means that: 2 queues will contain only long-read requests
-			and 6 queues will contain only short-read requests.
-			a scan.ratio of 0.5 means that: 4 queues will contain only long-read requests
-			and 4 queues will contain only short-read requests.
-			a scan.ratio of 0.8 means that: 6 queues will contain only long-read requests
-			and 2 queues will contain only short-read requests.
-		</description>
-	</property>
-	<property>
-		<name>hbase.regionserver.msginterval</name>
-		<value>3000</value>
-		<description>Interval between messages from the RegionServer to Master
-			in milliseconds.</description>
-	</property>
-	<property>
-		<name>hbase.regionserver.logroll.period</name>
-		<value>3600000</value>
-		<description>Period at which we will roll the commit log regardless
-			of how many edits it has.</description>
-	</property>
-	<property>
-		<name>hbase.regionserver.logroll.errors.tolerated</name>
-		<value>2</value>
-		<description>The number of consecutive WAL close errors we will allow
-			before triggering a server abort.  A setting of 0 will cause the
-			region server to abort if closing the current WAL writer fails during
-			log rolling.  Even a small value (2 or 3) will allow a region server
-			to ride over transient HDFS errors.</description>
-	</property>
-	<property>
-		<name>hbase.regionserver.global.memstore.size</name>
-		<value></value>
-		<description>Maximum size of all memstores in a region server before new
-			updates are blocked and flushes are forced. Defaults to 40% of heap (0.4).
-			Updates are blocked and flushes are forced until size of all memstores
-			in a region server hits hbase.regionserver.global.memstore.size.lower.limit.
-			The default value in this configuration has been intentionally left empty in order to
-			honor the old hbase.regionserver.global.memstore.upperLimit property if present.
-		</description>
-	</property>
-	<property>
-		<name>hbase.regionserver.global.memstore.size.lower.limit</name>
-		<value></value>
-		<description>Maximum size of all memstores in a region server before flushes
-			are forced. Defaults to 95% of hbase.regionserver.global.memstore.size
-			(0.95). A 100% value for this value causes the minimum possible flushing
-			to occur when updates are blocked due to memstore limiting. The default
-			value in this configuration has been intentionally left empty in order to
-			honor the old hbase.regionserver.global.memstore.lowerLimit property if
-			present.
-		</description>
-	</property>
-	<property>
-		<name>hbase.systemtables.compacting.memstore.type</name>
-		<value>NONE</value>
-		<description>Determines the type of memstore to be used for system tables like
-			META, namespace tables etc. By default NONE is the type and hence we use the
-			default memstore for all the system tables. If we need to use compacting
-			memstore for system tables then set this property to BASIC/EAGER
-		</description>
-	</property>
-	<property>
-		<name>hbase.regionserver.optionalcacheflushinterval</name>
-		<value>3600000</value>
-		<description>
-			Maximum amount of time an edit lives in memory before being automatically flushed.
-			Default 1 hour. Set it to 0 to disable automatic flushing.
-		</description>
-	</property>
-	<property>
-		<name>hbase.regionserver.dns.interface</name>
-		<value>default</value>
-		<description>The name of the Network Interface from which a region server
-			should report its IP address.</description>
-	</property>
-	<property>
-		<name>hbase.regionserver.dns.nameserver</name>
-		<value>default</value>
-		<description>The host name or IP address of the name server (DNS)
-			which a region server should use to determine the host name used by the
-			master for communication and display purposes.</description>
-	</property>
-	<property>
-		<name>hbase.regionserver.regionSplitLimit</name>
-		<value>1000</value>
-		<description>
-			Limit for the number of regions after which no more region splitting
-			should take place. This is not hard limit for the number of regions
-			but acts as a guideline for the regionserver to stop splitting after
-			a certain limit. Default is set to 1000.
-		</description>
-	</property>
-
-	<!--ZooKeeper configuration-->
-	<property>
-		<name>zookeeper.session.timeout</name>
-		<value>90000</value>
-		<description>ZooKeeper session timeout in milliseconds. It is used in two different ways.
-			First, this value is used in the ZK client that HBase uses to connect to the ensemble.
-			It is also used by HBase when it starts a ZK server and it is passed as the 'maxSessionTimeout'.
-			See https://zookeeper.apache.org/doc/current/zookeeperProgrammers.html#ch_zkSessions.
-			For example, if an HBase region server connects to a ZK ensemble that's also managed
-			by HBase, then the session timeout will be the one specified by this configuration.
-			But, a region server that connects to an ensemble managed with a different configuration
-			will be subjected that ensemble's maxSessionTimeout. So, even though HBase might propose
-			using 90 seconds, the ensemble can have a max timeout lower than this and it will take
-			precedence. The current default maxSessionTimeout that ZK ships with is 40 seconds, which is lower than
-			HBase's.
-		</description>
-	</property>
-	<property>
-		<name>zookeeper.znode.parent</name>
-		<value>/hbase</value>
-		<description>Root ZNode for HBase in ZooKeeper. All of HBase's ZooKeeper
-			files that are configured with a relative path will go under this node.
-			By default, all of HBase's ZooKeeper file paths are configured with a
-			relative path, so they will all go under this directory unless changed.
-		</description>
-	</property>
-	<property>
-		<name>zookeeper.znode.acl.parent</name>
-		<value>acl</value>
-		<description>Root ZNode for access control lists.</description>
-	</property>
-	<property>
-		<name>hbase.zookeeper.dns.interface</name>
-		<value>default</value>
-		<description>The name of the Network Interface from which a ZooKeeper server
-			should report its IP address.</description>
-	</property>
-	<property>
-		<name>hbase.zookeeper.dns.nameserver</name>
-		<value>default</value>
-		<description>The host name or IP address of the name server (DNS)
-			which a ZooKeeper server should use to determine the host name used by the
-			master for communication and display purposes.</description>
-	</property>
-	<!--
-    The following three properties are used together to create the list of
-    host:peer_port:leader_port quorum servers for ZooKeeper.
-    -->
-	<property>
-		<name>hbase.zookeeper.peerport</name>
-		<value>2888</value>
-		<description>Port used by ZooKeeper peers to talk to each other.
-			See https://zookeeper.apache.org/doc/r3.3.3/zookeeperStarted.html#sc_RunningReplicatedZooKeeper
-			for more information.</description>
-	</property>
-	<property>
-		<name>hbase.zookeeper.leaderport</name>
-		<value>3888</value>
-		<description>Port used by ZooKeeper for leader election.
-			See https://zookeeper.apache.org/doc/r3.3.3/zookeeperStarted.html#sc_RunningReplicatedZooKeeper
-			for more information.</description>
-	</property>
-	<!-- End of properties used to generate ZooKeeper host:port quorum list. -->
-
-	<!--
-    Beginning of properties that are directly mapped from ZooKeeper's zoo.cfg.
-    All properties with an "hbase.zookeeper.property." prefix are converted for
-    ZooKeeper's configuration. Hence, if you want to add an option from zoo.cfg,
-    e.g.  "initLimit=10" you would append the following to your configuration:
-      <property>
-        <name>hbase.zookeeper.property.initLimit</name>
-        <value>10</value>
-      </property>
-    -->
-	<property>
-		<name>hbase.zookeeper.property.initLimit</name>
-		<value>10</value>
-		<description>Property from ZooKeeper's config zoo.cfg.
-			The number of ticks that the initial synchronization phase can take.</description>
-	</property>
-	<property>
-		<name>hbase.zookeeper.property.syncLimit</name>
-		<value>5</value>
-		<description>Property from ZooKeeper's config zoo.cfg.
-			The number of ticks that can pass between sending a request and getting an
-			acknowledgment.</description>
-	</property>
-	<property>
-		<name>hbase.zookeeper.property.dataDir</name>
-		<value>${hbase.tmp.dir}/zookeeper</value>
-		<description>Property from ZooKeeper's config zoo.cfg.
-			The directory where the snapshot is stored.</description>
-	</property>
-	<property>
-		<name>hbase.zookeeper.property.clientPort</name>
-		<value>2181</value>
-		<description>Property from ZooKeeper's config zoo.cfg.
-			The port at which the clients will connect.</description>
-	</property>
-	<property>
-		<name>hbase.zookeeper.property.maxClientCnxns</name>
-		<value>300</value>
-		<description>Property from ZooKeeper's config zoo.cfg.
-			Limit on number of concurrent connections (at the socket level) that a
-			single client, identified by IP address, may make to a single member of
-			the ZooKeeper ensemble. Set high to avoid zk connection issues running
-			standalone and pseudo-distributed.</description>
-	</property>
-	<!-- End of properties that are directly mapped from ZooKeeper's zoo.cfg -->
-
-	<!--Client configurations-->
-	<property>
-		<name>hbase.client.write.buffer</name>
-		<value>2097152</value>
-		<description>Default size of the BufferedMutator write buffer in bytes.
-			A bigger buffer takes more memory -- on both the client and server
-			side since server instantiates the passed write buffer to process
-			it -- but a larger buffer size reduces the number of RPCs made.
-			For an estimate of server-side memory-used, evaluate
-			hbase.client.write.buffer * hbase.regionserver.handler.count</description>
-	</property>
-	<property>
-		<name>hbase.client.pause</name>
-		<value>100</value>
-		<description>General client pause value.  Used mostly as value to wait
-			before running a retry of a failed get, region lookup, etc.
-			See hbase.client.retries.number for description of how we backoff from
-			this initial pause amount and how this pause works w/ retries.</description>
-	</property>
-	<property>
-		<name>hbase.client.pause.cqtbe</name>
-		<value></value>
-		<description>Whether or not to use a special client pause for
-			CallQueueTooBigException (cqtbe). Set this property to a higher value
-			than hbase.client.pause if you observe frequent CQTBE from the same
-			RegionServer and the call queue there keeps full</description>
-	</property>
-	<property>
-		<name>hbase.client.retries.number</name>
-		<value>15</value>
-		<description>Maximum retries.  Used as maximum for all retryable
-			operations such as the getting of a cell's value, starting a row update,
-			etc.  Retry interval is a rough function based on hbase.client.pause.  At
-			first we retry at this interval but then with backoff, we pretty quickly reach
-			retrying every ten seconds.  See HConstants#RETRY_BACKOFF for how the backup
-			ramps up.  Change this setting and hbase.client.pause to suit your workload.</description>
-	</property>
-	<property>
-		<name>hbase.client.max.total.tasks</name>
-		<value>100</value>
-		<description>The maximum number of concurrent mutation tasks a single HTable instance will
-			send to the cluster.</description>
-	</property>
-	<property>
-		<name>hbase.client.max.perserver.tasks</name>
-		<value>2</value>
-		<description>The maximum number of concurrent mutation tasks a single HTable instance will
-			send to a single region server.</description>
-	</property>
-	<property>
-		<name>hbase.client.max.perregion.tasks</name>
-		<value>1</value>
-		<description>The maximum number of concurrent mutation tasks the client will
-			maintain to a single Region. That is, if there is already
-			hbase.client.max.perregion.tasks writes in progress for this region, new puts
-			won't be sent to this region until some writes finishes.</description>
-	</property>
-	<property>
-		<name>hbase.client.perserver.requests.threshold</name>
-		<value>2147483647</value>
-		<description>The max number of concurrent pending requests for one server in all client threads
-			(process level). Exceeding requests will be thrown ServerTooBusyException immediately to prevent
-			user's threads being occupied and blocked by only one slow region server. If you use a fix
-			number of threads to access HBase in a synchronous way, set this to a suitable value which is
-			related to the number of threads will help you. See
-			https://issues.apache.org/jira/browse/HBASE-16388 for details.</description>
-	</property>
-	<property>
-		<name>hbase.client.scanner.caching</name>
-		<value>2147483647</value>
-		<description>Number of rows that we try to fetch when calling next
-			on a scanner if it is not served from (local, client) memory. This configuration
-			works together with hbase.client.scanner.max.result.size to try and use the
-			network efficiently. The default value is Integer.MAX_VALUE by default so that
-			the network will fill the chunk size defined by hbase.client.scanner.max.result.size
-			rather than be limited by a particular number of rows since the size of rows varies
-			table to table. If you know ahead of time that you will not require more than a certain
-			number of rows from a scan, this configuration should be set to that row limit via
-			Scan#setCaching. Higher caching values will enable faster scanners but will eat up more
-			memory and some calls of next may take longer and longer times when the cache is empty.
-			Do not set this value such that the time between invocations is greater than the scanner
-			timeout; i.e. hbase.client.scanner.timeout.period</description>
-	</property>
-	<property>
-		<name>hbase.client.keyvalue.maxsize</name>
-		<value>10485760</value>
-		<description>Specifies the combined maximum allowed size of a KeyValue
-			instance. This is to set an upper boundary for a single entry saved in a
-			storage file. Since they cannot be split it helps avoiding that a region
-			cannot be split any further because the data is too large. It seems wise
-			to set this to a fraction of the maximum region size. Setting it to zero
-			or less disables the check.</description>
-	</property>
-	<property>
-		<name>hbase.server.keyvalue.maxsize</name>
-		<value>10485760</value>
-		<description>Maximum allowed size of an individual cell, inclusive of value and all key
-			components. A value of 0 or less disables the check.
-			The default value is 10MB.
-			This is a safety setting to protect the server from OOM situations.
-		</description>
-	</property>
-	<property>
-		<name>hbase.client.scanner.timeout.period</name>
-		<value>60000</value>
-		<description>Client scanner lease period in milliseconds.</description>
-	</property>
-	<property>
-		<name>hbase.client.localityCheck.threadPoolSize</name>
-		<value>2</value>
-	</property>
-
-	<!--Miscellaneous configuration-->
-	<property>
-		<name>hbase.bulkload.retries.number</name>
-		<value>10</value>
-		<description>Maximum retries.  This is maximum number of iterations
-			to atomic bulk loads are attempted in the face of splitting operations
-			0 means never give up.</description>
-	</property>
-	<property>
-		<name>hbase.master.balancer.maxRitPercent</name>
-		<value>1.0</value>
-		<description>The max percent of regions in transition when balancing.
-			The default value is 1.0. So there are no balancer throttling. If set this config to 0.01,
-			It means that there are at most 1% regions in transition when balancing.
-			Then the cluster's availability is at least 99% when balancing.</description>
-	</property>
-	<property>
-		<name>hbase.balancer.period
-		</name>
-		<value>300000</value>
-		<description>Period at which the region balancer runs in the Master.</description>
-	</property>
-	<property>
-		<name>hbase.normalizer.period</name>
-		<value>300000</value>
-		<description>Period at which the region normalizer runs in the Master.</description>
-	</property>
-	<property>
-		<name>hbase.regions.slop</name>
-		<value>0.001</value>
-		<description>Rebalance if any regionserver has average + (average * slop) regions.
-			The default value of this parameter is 0.001 in StochasticLoadBalancer (the default load balancer),
-			while the default is 0.2 in other load balancers (i.e., SimpleLoadBalancer).</description>
-	</property>
-	<property>
-		<name>hbase.server.thread.wakefrequency</name>
-		<value>10000</value>
-		<description>Time to sleep in between searches for work (in milliseconds).
-			Used as sleep interval by service threads such as log roller.</description>
-	</property>
-	<property>
-		<name>hbase.server.versionfile.writeattempts</name>
-		<value>3</value>
-		<description>
-			How many times to retry attempting to write a version file
-			before just aborting. Each attempt is separated by the
-			hbase.server.thread.wakefrequency milliseconds.</description>
-	</property>
-	<property>
-		<name>hbase.hregion.memstore.flush.size</name>
-		<value>134217728</value>
-		<description>
-			Memstore will be flushed to disk if size of the memstore
-			exceeds this number of bytes.  Value is checked by a thread that runs
-			every hbase.server.thread.wakefrequency.</description>
-	</property>
-	<property>
-		<name>hbase.hregion.percolumnfamilyflush.size.lower.bound.min</name>
-		<value>16777216</value>
-		<description>
-			If FlushLargeStoresPolicy is used and there are multiple column families,
-			then every time that we hit the total memstore limit, we find out all the
-			column families whose memstores exceed a "lower bound" and only flush them
-			while retaining the others in memory. The "lower bound" will be
-			"hbase.hregion.memstore.flush.size / column_family_number" by default
-			unless value of this property is larger than that. If none of the families
-			have their memstore size more than lower bound, all the memstores will be
-			flushed (just as usual).
-		</description>
-	</property>
-	<property>
-		<name>hbase.hregion.preclose.flush.size</name>
-		<value>5242880</value>
-		<description>
-			If the memstores in a region are this size or larger when we go
-			to close, run a "pre-flush" to clear out memstores before we put up
-			the region closed flag and take the region offline.  On close,
-			a flush is run under the close flag to empty memory.  During
-			this time the region is offline and we are not taking on any writes.
-			If the memstore content is large, this flush could take a long time to
-			complete.  The preflush is meant to clean out the bulk of the memstore
-			before putting up the close flag and taking the region offline so the
-			flush that runs under the close flag has little to do.</description>
-	</property>
-	<property>
-		<name>hbase.hregion.memstore.block.multiplier</name>
-		<value>4</value>
-		<description>
-			Block updates if memstore has hbase.hregion.memstore.block.multiplier
-			times hbase.hregion.memstore.flush.size bytes.  Useful preventing
-			runaway memstore during spikes in update traffic.  Without an
-			upper-bound, memstore fills such that when it flushes the
-			resultant flush files take a long time to compact or split, or
-			worse, we OOME.</description>
-	</property>
-	<property>
-		<name>hbase.hregion.memstore.mslab.enabled</name>
-		<value>true</value>
-		<description>
-			Enables the MemStore-Local Allocation Buffer,
-			a feature which works to prevent heap fragmentation under
-			heavy write loads. This can reduce the frequency of stop-the-world
-			GC pauses on large heaps.</description>
-	</property>
-	<property>
-		<name>hbase.hregion.max.filesize</name>
-		<value>10737418240</value>
-		<description>
-			Maximum HFile size. If the sum of the sizes of a region's HFiles has grown to exceed this
-			value, the region is split in two.</description>
-	</property>
-	<property>
-		<name>hbase.hregion.majorcompaction</name>
-		<value>604800000</value>
-		<description>Time between major compactions, expressed in milliseconds. Set to 0 to disable
-			time-based automatic major compactions. User-requested and size-based major compactions will
-			still run. This value is multiplied by hbase.hregion.majorcompaction.jitter to cause
-			compaction to start at a somewhat-random time during a given window of time. The default value
-			is 7 days, expressed in milliseconds. If major compactions are causing disruption in your
-			environment, you can configure them to run at off-peak times for your deployment, or disable
-			time-based major compactions by setting this parameter to 0, and run major compactions in a
-			cron job or by another external mechanism.</description>
-	</property>
-	<property>
-		<name>hbase.hregion.majorcompaction.jitter</name>
-		<value>0.50</value>
-		<description>A multiplier applied to hbase.hregion.majorcompaction to cause compaction to occur
-			a given amount of time either side of hbase.hregion.majorcompaction. The smaller the number,
-			the closer the compactions will happen to the hbase.hregion.majorcompaction
-			interval.</description>
-	</property>
-	<property>
-		<name>hbase.hstore.compactionThreshold</name>
-		<value>3</value>
-		<description> If more than this number of StoreFiles exist in any one Store
-			(one StoreFile is written per flush of MemStore), a compaction is run to rewrite all
-			StoreFiles into a single StoreFile. Larger values delay compaction, but when compaction does
-			occur, it takes longer to complete.</description>
-	</property>
-	<property>
-		<name>hbase.regionserver.compaction.enabled</name>
-		<value>true</value>
-		<description>Enable/disable compactions on by setting true/false.
-			We can further switch compactions dynamically with the
-			compaction_switch shell command.</description>
-	</property>
-	<property>
-		<name>hbase.hstore.flusher.count</name>
-		<value>2</value>
-		<description> The number of flush threads. With fewer threads, the MemStore flushes will be
-			queued. With more threads, the flushes will be executed in parallel, increasing the load on
-			HDFS, and potentially causing more compactions. </description>
-	</property>
-	<property>
-		<name>hbase.hstore.blockingStoreFiles</name>
-		<value>16</value>
-		<description> If more than this number of StoreFiles exist in any one Store (one StoreFile
-			is written per flush of MemStore), updates are blocked for this region until a compaction is
-			completed, or until hbase.hstore.blockingWaitTime has been exceeded.</description>
-	</property>
-	<property>
-		<name>hbase.hstore.blockingWaitTime</name>
-		<value>90000</value>
-		<description> The time for which a region will block updates after reaching the StoreFile limit
-			defined by hbase.hstore.blockingStoreFiles. After this time has elapsed, the region will stop
-			blocking updates even if a compaction has not been completed.</description>
-	</property>
-	<property>
-		<name>hbase.hstore.compaction.min</name>
-		<value>3</value>
-		<description>The minimum number of StoreFiles which must be eligible for compaction before
-			compaction can run. The goal of tuning hbase.hstore.compaction.min is to avoid ending up with
-			too many tiny StoreFiles to compact. Setting this value to 2 would cause a minor compaction
-			each time you have two StoreFiles in a Store, and this is probably not appropriate. If you
-			set this value too high, all the other values will need to be adjusted accordingly. For most
-			cases, the default value is appropriate. In previous versions of HBase, the parameter
-			hbase.hstore.compaction.min was named hbase.hstore.compactionThreshold.</description>
-	</property>
-	<property>
-		<name>hbase.hstore.compaction.max</name>
-		<value>10</value>
-		<description>The maximum number of StoreFiles which will be selected for a single minor
-			compaction, regardless of the number of eligible StoreFiles. Effectively, the value of
-			hbase.hstore.compaction.max controls the length of time it takes a single compaction to
-			complete. Setting it larger means that more StoreFiles are included in a compaction. For most
-			cases, the default value is appropriate.</description>
-	</property>
-	<property>
-		<name>hbase.hstore.compaction.min.size</name>
-		<value>134217728</value>
-		<description>A StoreFile (or a selection of StoreFiles, when using ExploringCompactionPolicy)
-			smaller than this size will always be eligible for minor compaction.
-			HFiles this size or larger are evaluated by hbase.hstore.compaction.ratio to determine if
-			they are eligible. Because this limit represents the "automatic include" limit for all
-			StoreFiles smaller than this value, this value may need to be reduced in write-heavy
-			environments where many StoreFiles in the 1-2 MB range are being flushed, because every
-			StoreFile will be targeted for compaction and the resulting StoreFiles may still be under the
-			minimum size and require further compaction. If this parameter is lowered, the ratio check is
-			triggered more quickly. This addressed some issues seen in earlier versions of HBase but
-			changing this parameter is no longer necessary in most situations. Default: 128 MB expressed
-			in bytes.</description>
-	</property>
-	<property>
-		<name>hbase.hstore.compaction.max.size</name>
-		<value>9223372036854775807</value>
-		<description>A StoreFile (or a selection of StoreFiles, when using ExploringCompactionPolicy)
-			larger than this size will be excluded from compaction. The effect of
-			raising hbase.hstore.compaction.max.size is fewer, larger StoreFiles that do not get
-			compacted often. If you feel that compaction is happening too often without much benefit, you
-			can try raising this value. Default: the value of LONG.MAX_VALUE, expressed in bytes.</description>
-	</property>
-	<property>
-		<name>hbase.hstore.compaction.ratio</name>
-		<value>1.2F</value>
-		<description>For minor compaction, this ratio is used to determine whether a given StoreFile
-			which is larger than hbase.hstore.compaction.min.size is eligible for compaction. Its
-			effect is to limit compaction of large StoreFiles. The value of hbase.hstore.compaction.ratio
-			is expressed as a floating-point decimal. A large ratio, such as 10, will produce a single
-			giant StoreFile. Conversely, a low value, such as .25, will produce behavior similar to the
-			BigTable compaction algorithm, producing four StoreFiles. A moderate value of between 1.0 and
-			1.4 is recommended. When tuning this value, you are balancing write costs with read costs.
-			Raising the value (to something like 1.4) will have more write costs, because you will
-			compact larger StoreFiles. However, during reads, HBase will need to seek through fewer
-			StoreFiles to accomplish the read. Consider this approach if you cannot take advantage of
-			Bloom filters. Otherwise, you can lower this value to something like 1.0 to reduce the
-			background cost of writes, and use Bloom filters to control the number of StoreFiles touched
-			during reads. For most cases, the default value is appropriate.</description>
-	</property>
-	<property>
-		<name>hbase.hstore.compaction.ratio.offpeak</name>
-		<value>5.0F</value>
-		<description>Allows you to set a different (by default, more aggressive) ratio for determining
-			whether larger StoreFiles are included in compactions during off-peak hours. Works in the
-			same way as hbase.hstore.compaction.ratio. Only applies if hbase.offpeak.start.hour and
-			hbase.offpeak.end.hour are also enabled.</description>
-	</property>
-	<property>
-		<name>hbase.hstore.time.to.purge.deletes</name>
-		<value>0</value>
-		<description>The amount of time to delay purging of delete markers with future timestamps. If
-			unset, or set to 0, all delete markers, including those with future timestamps, are purged
-			during the next major compaction. Otherwise, a delete marker is kept until the major compaction
-			which occurs after the marker's timestamp plus the value of this setting, in milliseconds.
-		</description>
-	</property>
-	<property>
-		<name>hbase.offpeak.start.hour</name>
-		<value>-1</value>
-		<description>The start of off-peak hours, expressed as an integer between 0 and 23, inclusive.
-			Set to -1 to disable off-peak.</description>
-	</property>
-	<property>
-		<name>hbase.offpeak.end.hour</name>
-		<value>-1</value>
-		<description>The end of off-peak hours, expressed as an integer between 0 and 23, inclusive. Set
-			to -1 to disable off-peak.</description>
-	</property>
-	<property>
-		<name>hbase.regionserver.thread.compaction.throttle</name>
-		<value>2684354560</value>
-		<description>There are two different thread pools for compactions, one for large compactions and
-			the other for small compactions. This helps to keep compaction of lean tables (such as
-			hbase:meta) fast. If a compaction is larger than this threshold, it
-			goes into the large compaction pool. In most cases, the default value is appropriate. Default:
-			2 x hbase.hstore.compaction.max x hbase.hregion.memstore.flush.size (which defaults to 128MB).
-			The value field assumes that the value of hbase.hregion.memstore.flush.size is unchanged from
-			the default.</description>
-	</property>
-	<property>
-		<name>hbase.regionserver.majorcompaction.pagecache.drop</name>
-		<value>true</value>
-		<description>Specifies whether to drop pages read/written into the system page cache by
-			major compactions. Setting it to true helps prevent major compactions from
-			polluting the page cache, which is almost always required, especially for clusters
-			with low/moderate memory to storage ratio.</description>
-	</property>
-	<property>
-		<name>hbase.regionserver.minorcompaction.pagecache.drop</name>
-		<value>true</value>
-		<description>Specifies whether to drop pages read/written into the system page cache by
-			minor compactions. Setting it to true helps prevent minor compactions from
-			polluting the page cache, which is most beneficial on clusters with low
-			memory to storage ratio or very write heavy clusters. You may want to set it to
-			false under moderate to low write workload when bulk of the reads are
-			on the most recently written data.</description>
-	</property>
-	<property>
-		<name>hbase.hstore.compaction.kv.max</name>
-		<value>10</value>
-		<description>The maximum number of KeyValues to read and then write in a batch when flushing or
-			compacting. Set this lower if you have big KeyValues and problems with Out Of Memory
-			Exceptions Set this higher if you have wide, small rows. </description>
-	</property>
-	<property>
-		<name>hbase.storescanner.parallel.seek.enable</name>
-		<value>false</value>
-		<description>
-			Enables StoreFileScanner parallel-seeking in StoreScanner,
-			a feature which can reduce response latency under special conditions.</description>
-	</property>
-	<property>
-		<name>hbase.storescanner.parallel.seek.threads</name>
-		<value>10</value>
-		<description>
-			The default thread pool size if parallel-seeking feature enabled.</description>
-	</property>
-	<property>
-		<name>hfile.block.cache.size</name>
-		<value>0.4</value>
-		<description>Percentage of maximum heap (-Xmx setting) to allocate to block cache
-			used by a StoreFile. Default of 0.4 means allocate 40%.
-			Set to 0 to disable but it's not recommended; you need at least
-			enough cache to hold the storefile indices.</description>
-	</property>
-	<property>
-		<name>hfile.block.index.cacheonwrite</name>
-		<value>false</value>
-		<description>This allows to put non-root multi-level index blocks into the block
-			cache at the time the index is being written.</description>
-	</property>
-	<property>
-		<name>hfile.index.block.max.size</name>
-		<value>131072</value>
-		<description>When the size of a leaf-level, intermediate-level, or root-level
-			index block in a multi-level block index grows to this size, the
-			block is written out and a new block is started.</description>
-	</property>
-	<property>
-		<name>hbase.bucketcache.ioengine</name>
-		<value></value>
-		<description>Where to store the contents of the bucketcache. One of: offheap,
-			file, files or mmap. If a file or files, set it to file(s):PATH_TO_FILE.
-			mmap means the content will be in an mmaped file. Use mmap:PATH_TO_FILE.
-			See http://hbase.apache.org/book.html#offheap.blockcache for more information.
-		</description>
-	</property>
-	<property>
-		<name>hbase.bucketcache.size</name>
-		<value></value>
-		<description>A float that EITHER represents a percentage of total heap memory
-			size to give to the cache (if &lt; 1.0) OR, it is the total capacity in
-			megabytes of BucketCache. Default: 0.0</description>
-	</property>
-	<property>
-		<name>hbase.bucketcache.bucket.sizes</name>
-		<value></value>
-		<description>A comma-separated list of sizes for buckets for the bucketcache.
-			Can be multiple sizes. List block sizes in order from smallest to largest.
-			The sizes you use will depend on your data access patterns.
-			Must be a multiple of 256 else you will run into
-			'java.io.IOException: Invalid HFile block magic' when you go to read from cache.
-			If you specify no values here, then you pick up the default bucketsizes set
-			in code (See BucketAllocator#DEFAULT_BUCKET_SIZES).
-		</description>
-	</property>
-	<property>
-		<name>hfile.format.version</name>
-		<value>3</value>
-		<description>The HFile format version to use for new files.
-			Version 3 adds support for tags in hfiles (See http://hbase.apache.org/book.html#hbase.tags).
-			Also see the configuration 'hbase.replication.rpc.codec'.
-		</description>
-	</property>
-	<property>
-		<name>hfile.block.bloom.cacheonwrite</name>
-		<value>false</value>
-		<description>Enables cache-on-write for inline blocks of a compound Bloom filter.</description>
-	</property>
-	<property>
-		<name>io.storefile.bloom.block.size</name>
-		<value>131072</value>
-		<description>The size in bytes of a single block ("chunk") of a compound Bloom
-			filter. This size is approximate, because Bloom blocks can only be
-			inserted at data block boundaries, and the number of keys per data
-			block varies.</description>
-	</property>
-	<property>
-		<name>hbase.rs.cacheblocksonwrite</name>
-		<value>false</value>
-		<description>Whether an HFile block should be added to the block cache when the
-			block is finished.</description>
-	</property>
-	<property>
-		<name>hbase.rpc.timeout</name>
-		<value>60000</value>
-		<description>This is for the RPC layer to define how long (millisecond) HBase client applications
-			take for a remote call to time out. It uses pings to check connections
-			but will eventually throw a TimeoutException.</description>
-	</property>
-	<property>
-		<name>hbase.client.operation.timeout</name>
-		<value>1200000</value>
-		<description>Operation timeout is a top-level restriction (millisecond) that makes sure a
-			blocking operation in Table will not be blocked more than this. In each operation, if rpc
-			request fails because of timeout or other reason, it will retry until success or throw
-			RetriesExhaustedException. But if the total time being blocking reach the operation timeout
-			before retries exhausted, it will break early and throw SocketTimeoutException.</description>
-	</property>
-	<property>
-		<name>hbase.cells.scanned.per.heartbeat.check</name>
-		<value>10000</value>
-		<description>The number of cells scanned in between heartbeat checks. Heartbeat
-			checks occur during the processing of scans to determine whether or not the
-			server should stop scanning in order to send back a heartbeat message to the
-			client. Heartbeat messages are used to keep the client-server connection alive
-			during long running scans. Small values mean that the heartbeat checks will
-			occur more often and thus will provide a tighter bound on the execution time of
-			the scan. Larger values mean that the heartbeat checks occur less frequently
-		</description>
-	</property>
-	<property>
-		<name>hbase.rpc.shortoperation.timeout</name>
-		<value>10000</value>
-		<description>This is another version of "hbase.rpc.timeout". For those RPC operation
-			within cluster, we rely on this configuration to set a short timeout limitation
-			for short operation. For example, short rpc timeout for region server's trying
-			to report to active master can benefit quicker master failover process.</description>
-	</property>
-	<property>
-		<name>hbase.ipc.client.tcpnodelay</name>
-		<value>true</value>
-		<description>Set no delay on rpc socket connections.  See
-			http://docs.oracle.com/javase/1.5.0/docs/api/java/net/Socket.html#getTcpNoDelay()</description>
-	</property>
-	<property>
-		<name>hbase.regionserver.hostname</name>
-		<value></value>
-		<description>This config is for experts: don't set its value unless you really know what you are doing.
-			When set to a non-empty value, this represents the (external facing) hostname for the underlying server.
-			See https://issues.apache.org/jira/browse/HBASE-12954 for details.</description>
-	</property>
-	<property>
-		<name>hbase.regionserver.hostname.disable.master.reversedns</name>
-		<value>false</value>
-		<description>This config is for experts: don't set its value unless you really know what you are doing.
-			When set to true, regionserver will use the current node hostname for the servername and HMaster will
-			skip reverse DNS lookup and use the hostname sent by regionserver instead. Note that this config and
-			hbase.regionserver.hostname are mutually exclusive. See https://issues.apache.org/jira/browse/HBASE-18226
-			for more details.</description>
-	</property>
-	<!-- The following properties configure authentication information for
-         HBase processes when using Kerberos security.  There are no default
-         values, included here for documentation purposes -->
-	<property>
-		<name>hbase.master.keytab.file</name>
-		<value></value>
-		<description>Full path to the kerberos keytab file to use for logging in
-			the configured HMaster server principal.</description>
-	</property>
-	<property>
-		<name>hbase.master.kerberos.principal</name>
-		<value></value>
-		<description>Ex. "hbase/_HOST@EXAMPLE.COM".  The kerberos principal name
-			that should be used to run the HMaster process.  The principal name should
-			be in the form: user/hostname@DOMAIN.  If "_HOST" is used as the hostname
-			portion, it will be replaced with the actual hostname of the running
-			instance.</description>
-	</property>
-	<property>
-		<name>hbase.regionserver.keytab.file</name>
-		<value></value>
-		<description>Full path to the kerberos keytab file to use for logging in
-			the configured HRegionServer server principal.</description>
-	</property>
-	<property>
-		<name>hbase.regionserver.kerberos.principal</name>
-		<value></value>
-		<description>Ex. "hbase/_HOST@EXAMPLE.COM".  The kerberos principal name
-			that should be used to run the HRegionServer process.  The principal name
-			should be in the form: user/hostname@DOMAIN.  If "_HOST" is used as the
-			hostname portion, it will be replaced with the actual hostname of the
-			running instance.  An entry for this principal must exist in the file
-			specified in hbase.regionserver.keytab.file</description>
-	</property>
-	<!-- Additional configuration specific to HBase security -->
-	<property>
-		<name>hadoop.policy.file</name>
-		<value>hbase-policy.xml</value>
-		<description>The policy configuration file used by RPC servers to make
-			authorization decisions on client requests.  Only used when HBase
-			security is enabled.</description>
-	</property>
-	<property>
-		<name>hbase.superuser</name>
-		<value></value>
-		<description>List of users or groups (comma-separated), who are allowed
-			full privileges, regardless of stored ACLs, across the cluster.
-			Only used when HBase security is enabled.</description>
-	</property>
-	<property>
-		<name>hbase.auth.key.update.interval</name>
-		<value>86400000</value>
-		<description>The update interval for master key for authentication tokens
-			in servers in milliseconds.  Only used when HBase security is enabled.</description>
-	</property>
-	<property>
-		<name>hbase.auth.token.max.lifetime</name>
-		<value>604800000</value>
-		<description>The maximum lifetime in milliseconds after which an
-			authentication token expires.  Only used when HBase security is enabled.</description>
-	</property>
-	<property>
-		<name>hbase.ipc.client.fallback-to-simple-auth-allowed</name>
-		<value>false</value>
-		<description>When a client is configured to attempt a secure connection, but attempts to
-			connect to an insecure server, that server may instruct the client to
-			switch to SASL SIMPLE (unsecure) authentication. This setting controls
-			whether or not the client will accept this instruction from the server.
-			When false (the default), the client will not allow the fallback to SIMPLE
-			authentication, and will abort the connection.</description>
-	</property>
-	<property>
-		<name>hbase.ipc.server.fallback-to-simple-auth-allowed</name>
-		<value>false</value>
-		<description>When a server is configured to require secure connections, it will
-			reject connection attempts from clients using SASL SIMPLE (unsecure) authentication.
-			This setting allows secure servers to accept SASL SIMPLE connections from clients
-			when the client requests.  When false (the default), the server will not allow the fallback
-			to SIMPLE authentication, and will reject the connection.  WARNING: This setting should ONLY
-			be used as a temporary measure while converting clients over to secure authentication.  It
-			MUST BE DISABLED for secure operation.</description>
-	</property>
-	<property>
-		<name>hbase.display.keys</name>
-		<value>true</value>
-		<description>When this is set to true the webUI and such will display all start/end keys
-			as part of the table details, region names, etc. When this is set to false,
-			the keys are hidden.</description>
-	</property>
-	<property>
-		<name>hbase.coprocessor.enabled</name>
-		<value>true</value>
-		<description>Enables or disables coprocessor loading. If 'false'
-			(disabled), any other coprocessor related configuration will be ignored.
-		</description>
-	</property>
-	<property>
-		<name>hbase.coprocessor.user.enabled</name>
-		<value>true</value>
-		<description>Enables or disables user (aka. table) coprocessor loading.
-			If 'false' (disabled), any table coprocessor attributes in table
-			descriptors will be ignored. If "hbase.coprocessor.enabled" is 'false'
-			this setting has no effect.
-		</description>
-	</property>
-	<property>
-		<name>hbase.coprocessor.region.classes</name>
-		<value></value>
-		<description>A comma-separated list of Coprocessors that are loaded by
-			default on all tables. For any override coprocessor method, these classes
-			will be called in order. After implementing your own Coprocessor, just put
-			it in HBase's classpath and add the fully qualified class name here.
-			A coprocessor can also be loaded on demand by setting HTableDescriptor.</description>
-	</property>
-	<property>
-		<name>hbase.coprocessor.master.classes</name>
-		<value></value>
-		<description>A comma-separated list of
-			org.apache.hadoop.hbase.coprocessor.MasterObserver coprocessors that are
-			loaded by default on the active HMaster process. For any implemented
-			coprocessor methods, the listed classes will be called in order. After
-			implementing your own MasterObserver, just put it in HBase's classpath
-			and add the fully qualified class name here.</description>
-	</property>
-	<property>
-		<name>hbase.coprocessor.abortonerror</name>
-		<value>true</value>
-		<description>Set to true to cause the hosting server (master or regionserver)
-			to abort if a coprocessor fails to load, fails to initialize, or throws an
-			unexpected Throwable object. Setting this to false will allow the server to
-			continue execution but the system wide state of the coprocessor in question
-			will become inconsistent as it will be properly executing in only a subset
-			of servers, so this is most useful for debugging only.</description>
-	</property>
-	<property>
-		<name>hbase.rest.port</name>
-		<value>8080</value>
-		<description>The port for the HBase REST server.</description>
-	</property>
-	<property>
-		<name>hbase.rest.readonly</name>
-		<value>false</value>
-		<description>Defines the mode the REST server will be started in. Possible values are:
-			false: All HTTP methods are permitted - GET/PUT/POST/DELETE.
-			true: Only the GET method is permitted.</description>
-	</property>
-	<property>
-		<name>hbase.rest.threads.max</name>
-		<value>100</value>
-		<description>The maximum number of threads of the REST server thread pool.
-			Threads in the pool are reused to process REST requests. This
-			controls the maximum number of requests processed concurrently.
-			It may help to control the memory used by the REST server to
-			avoid OOM issues. If the thread pool is full, incoming requests
-			will be queued up and wait for some free threads.</description>
-	</property>
-	<property>
-		<name>hbase.rest.threads.min</name>
-		<value>2</value>
-		<description>The minimum number of threads of the REST server thread pool.
-			The thread pool always has at least these number of threads so
-			the REST server is ready to serve incoming requests.</description>
-	</property>
-	<property>
-		<name>hbase.rest.support.proxyuser</name>
-		<value>false</value>
-		<description>Enables running the REST server to support proxy-user mode.</description>
-	</property>
-	<property skipInDoc="true">
-		<name>hbase.defaults.for.version</name>
-		<value>2.2.3</value>
-		<description>This defaults file was compiled for version ${project.version}. This variable is used
-			to make sure that a user doesn't have an old version of hbase-default.xml on the
-			classpath.</description>
-	</property>
-	<property>
-		<name>hbase.defaults.for.version.skip</name>
-		<value>false</value>
-		<description>Set to true to skip the 'hbase.defaults.for.version' check.
-			Setting this to true can be useful in contexts other than
-			the other side of a maven generation; i.e. running in an
-			IDE.  You'll want to set this boolean to true to avoid
-			seeing the RuntimeException complaint: "hbase-default.xml file
-			seems to be for and old version of HBase (\${hbase.version}), this
-			version is X.X.X-SNAPSHOT"</description>
-	</property>
-	<property>
-		<name>hbase.table.lock.enable</name>
-		<value>true</value>
-		<description>Set to true to enable locking the table in zookeeper for schema change operations.
-			Table locking from master prevents concurrent schema modifications to corrupt table
-			state.</description>
-	</property>
-	<property>
-		<name>hbase.table.max.rowsize</name>
-		<value>1073741824</value>
-		<description>
-			Maximum size of single row in bytes (default is 1 Gb) for Get'ting
-			or Scan'ning without in-row scan flag set. If row size exceeds this limit
-			RowTooBigException is thrown to client.
-		</description>
-	</property>
-	<property>
-		<name>hbase.thrift.minWorkerThreads</name>
-		<value>16</value>
-		<description>The "core size" of the thread pool. New threads are created on every
-			connection until this many threads are created.</description>
-	</property>
-	<property>
-		<name>hbase.thrift.maxWorkerThreads</name>
-		<value>1000</value>
-		<description>The maximum size of the thread pool. When the pending request queue
-			overflows, new threads are created until their number reaches this number.
-			After that, the server starts dropping connections.</description>
-	</property>
-	<property>
-		<name>hbase.thrift.maxQueuedRequests</name>
-		<value>1000</value>
-		<description>The maximum number of pending Thrift connections waiting in the queue. If
-			there are no idle threads in the pool, the server queues requests. Only
-			when the queue overflows, new threads are added, up to
-			hbase.thrift.maxQueuedRequests threads.</description>
-	</property>
-	<property>
-		<name>hbase.regionserver.thrift.framed</name>
-		<value>false</value>
-		<description>Use Thrift TFramedTransport on the server side.
-			This is the recommended transport for thrift servers and requires a similar setting
-			on the client side. Changing this to false will select the default transport,
-			vulnerable to DoS when malformed requests are issued due to THRIFT-601.
-		</description>
-	</property>
-	<property>
-		<name>hbase.regionserver.thrift.framed.max_frame_size_in_mb</name>
-		<value>2</value>
-		<description>Default frame size when using framed transport, in MB</description>
-	</property>
-	<property>
-		<name>hbase.regionserver.thrift.compact</name>
-		<value>false</value>
-		<description>Use Thrift TCompactProtocol binary serialization protocol.</description>
-	</property>
-	<property>
-		<name>hbase.rootdir.perms</name>
-		<value>700</value>
-		<description>FS Permissions for the root data subdirectory in a secure (kerberos) setup.
-			When master starts, it creates the rootdir with this permissions or sets the permissions
-			if it does not match.</description>
-	</property>
-	<property>
-		<name>hbase.wal.dir.perms</name>
-		<value>700</value>
-		<description>FS Permissions for the root WAL directory in a secure(kerberos) setup.
-			When master starts, it creates the WAL dir with this permissions or sets the permissions
-			if it does not match.</description>
-	</property>
-	<property>
-		<name>hbase.data.umask.enable</name>
-		<value>false</value>
-		<description>Enable, if true, that file permissions should be assigned
-			to the files written by the regionserver</description>
-	</property>
-	<property>
-		<name>hbase.data.umask</name>
-		<value>000</value>
-		<description>File permissions that should be used to write data
-			files when hbase.data.umask.enable is true</description>
-	</property>
-	<property>
-		<name>hbase.snapshot.enabled</name>
-		<value>true</value>
-		<description>Set to true to allow snapshots to be taken / restored / cloned.</description>
-	</property>
-	<property>
-		<name>hbase.snapshot.restore.take.failsafe.snapshot</name>
-		<value>true</value>
-		<description>Set to true to take a snapshot before the restore operation.
-			The snapshot taken will be used in case of failure, to restore the previous state.
-			At the end of the restore operation this snapshot will be deleted</description>
-	</property>
-	<property>
-		<name>hbase.snapshot.restore.failsafe.name</name>
-		<value>hbase-failsafe-{snapshot.name}-{restore.timestamp}</value>
-		<description>Name of the failsafe snapshot taken by the restore operation.
-			You can use the {snapshot.name}, {table.name} and {restore.timestamp} variables
-			to create a name based on what you are restoring.</description>
-	</property>
-	<property>
-		<name>hbase.snapshot.working.dir</name>
-		<value></value>
-		<description>Location where the snapshotting process will occur. The location of the
-			completed snapshots will not change, but the temporary directory where the snapshot
-			process occurs will be set to this location. This can be a separate filesystem than
-			the root directory, for performance increase purposes. See HBASE-21098 for more
-			information</description>
-	</property>
-	<property>
-		<name>hbase.server.compactchecker.interval.multiplier</name>
-		<value>1000</value>
-		<description>The number that determines how often we scan to see if compaction is necessary.
-			Normally, compactions are done after some events (such as memstore flush), but if
-			region didn't receive a lot of writes for some time, or due to different compaction
-			policies, it may be necessary to check it periodically. The interval between checks is
-			hbase.server.compactchecker.interval.multiplier multiplied by
-			hbase.server.thread.wakefrequency.</description>
-	</property>
-	<property>
-		<name>hbase.lease.recovery.timeout</name>
-		<value>900000</value>
-		<description>How long we wait on dfs lease recovery in total before giving up.</description>
-	</property>
-	<property>
-		<name>hbase.lease.recovery.dfs.timeout</name>
-		<value>64000</value>
-		<description>How long between dfs recover lease invocations. Should be larger than the sum of
-			the time it takes for the namenode to issue a block recovery command as part of
-			datanode; dfs.heartbeat.interval and the time it takes for the primary
-			datanode, performing block recovery to timeout on a dead datanode; usually
-			dfs.client.socket-timeout. See the end of HBASE-8389 for more.</description>
-	</property>
-	<property>
-		<name>hbase.column.max.version</name>
-		<value>1</value>
-		<description>New column family descriptors will use this value as the default number of versions
-			to keep.</description>
-	</property>
-	<property>
-		<name>dfs.client.read.shortcircuit</name>
-		<value>false</value>
-		<description>
-			If set to true, this configuration parameter enables short-circuit local
-			reads.
-		</description>
-	</property>
-	<property>
-		<name>dfs.domain.socket.path</name>
-		<value>none</value>
-		<description>
-			This is a path to a UNIX domain socket that will be used for
-			communication between the DataNode and local HDFS clients, if
-			dfs.client.read.shortcircuit is set to true. If the string "_PORT" is
-			present in this path, it will be replaced by the TCP port of the DataNode.
-			Be careful about permissions for the directory that hosts the shared
-			domain socket; dfsclient will complain if open to other users than the HBase user.
-		</description>
-	</property>
-	<property>
-		<name>hbase.dfs.client.read.shortcircuit.buffer.size</name>
-		<value>131072</value>
-		<description>If the DFSClient configuration
-			dfs.client.read.shortcircuit.buffer.size is unset, we will
-			use what is configured here as the short circuit read default
-			direct byte buffer size. DFSClient native default is 1MB; HBase
-			keeps its HDFS files open so number of file blocks * 1MB soon
-			starts to add up and threaten OOME because of a shortage of
-			direct memory.  So, we set it down from the default.  Make
-			it > the default hbase block size set in the HColumnDescriptor
-			which is usually 64k.
-		</description>
-	</property>
-	<property>
-		<name>hbase.regionserver.checksum.verify</name>
-		<value>true</value>
-		<description>
-			If set to true (the default), HBase verifies the checksums for hfile
-			blocks. HBase writes checksums inline with the data when it writes out
-			hfiles. HDFS (as of this writing) writes checksums to a separate file
-			than the data file necessitating extra seeks.  Setting this flag saves
-			some on i/o.  Checksum verification by HDFS will be internally disabled
-			on hfile streams when this flag is set.  If the hbase-checksum verification
-			fails, we will switch back to using HDFS checksums (so do not disable HDFS
-			checksums!  And besides this feature applies to hfiles only, not to WALs).
-			If this parameter is set to false, then hbase will not verify any checksums,
-			instead it will depend on checksum verification being done in the HDFS client.
-		</description>
-	</property>
-	<property>
-		<name>hbase.hstore.bytes.per.checksum</name>
-		<value>16384</value>
-		<description>
-			Number of bytes in a newly created checksum chunk for HBase-level
-			checksums in hfile blocks.
-		</description>
-	</property>
-	<property>
-		<name>hbase.hstore.checksum.algorithm</name>
-		<value>CRC32C</value>
-		<description>
-			Name of an algorithm that is used to compute checksums. Possible values
-			are NULL, CRC32, CRC32C.
-		</description>
-	</property>
-	<property>
-		<name>hbase.client.scanner.max.result.size</name>
-		<value>2097152</value>
-		<description>Maximum number of bytes returned when calling a scanner's next method.
-			Note that when a single row is larger than this limit the row is still returned completely.
-			The default value is 2MB, which is good for 1ge networks.
-			With faster and/or high latency networks this value should be increased.
-		</description>
-	</property>
-	<property>
-		<name>hbase.server.scanner.max.result.size</name>
-		<value>104857600</value>
-		<description>Maximum number of bytes returned when calling a scanner's next method.
-			Note that when a single row is larger than this limit the row is still returned completely.
-			The default value is 100MB.
-			This is a safety setting to protect the server from OOM situations.
-		</description>
-	</property>
-	<property>
-		<name>hbase.status.published</name>
-		<value>false</value>
-		<description>
-			This setting activates the publication by the master of the status of the region server.
-			When a region server dies and its recovery starts, the master will push this information
-			to the client application, to let them cut the connection immediately instead of waiting
-			for a timeout.
-		</description>
-	</property>
-	<property>
-		<name>hbase.status.multicast.address.ip</name>
-		<value>226.1.1.3</value>
-		<description>
-			Multicast address to use for the status publication by multicast.
-		</description>
-	</property>
-	<property>
-		<name>hbase.status.multicast.address.port</name>
-		<value>16100</value>
-		<description>
-			Multicast port to use for the status publication by multicast.
-		</description>
-	</property>
-	<property>
-		<name>hbase.dynamic.jars.dir</name>
-		<value>${hbase.rootdir}/lib</value>
-		<description>
-			The directory from which the custom filter JARs can be loaded
-			dynamically by the region server without the need to restart. However,
-			an already loaded filter/co-processor class would not be un-loaded. See
-			HBASE-1936 for more details.
-
-			Does not apply to coprocessors.
-		</description>
-	</property>
-	<property>
-		<name>hbase.security.authentication</name>
-		<value>simple</value>
-		<description>
-			Controls whether or not secure authentication is enabled for HBase.
-			Possible values are 'simple' (no authentication), and 'kerberos'.
-		</description>
-	</property>
-	<property>
-		<name>hbase.master.loadbalance.bytable</name>
-		<value>false</value>
-		<description>Factor Table name when the balancer runs.
-			Default: false.
-		</description>
-	</property>
-	<property>
-		<name>hbase.rest.csrf.enabled</name>
-		<value>false</value>
-		<description>
-			Set to true to enable protection against cross-site request forgery (CSRF)
-		</description>
-	</property>
-	<property>
-		<name>hbase.rest-csrf.browser-useragents-regex</name>
-		<value>^Mozilla.*,^Opera.*</value>
-		<description>
-			A comma-separated list of regular expressions used to match against an HTTP
-			request's User-Agent header when protection against cross-site request
-			forgery (CSRF) is enabled for REST server by setting
-			hbase.rest.csrf.enabled to true.  If the incoming User-Agent matches
-			any of these regular expressions, then the request is considered to be sent
-			by a browser, and therefore CSRF prevention is enforced.  If the request's
-			User-Agent does not match any of these regular expressions, then the request
-			is considered to be sent by something other than a browser, such as scripted
-			automation.  In this case, CSRF is not a potential attack vector, so
-			the prevention is not enforced.  This helps achieve backwards-compatibility
-			with existing automation that has not been updated to send the CSRF
-			prevention header.
-		</description>
-	</property>
-	<property>
-		<name>hbase.security.exec.permission.checks</name>
-		<value>false</value>
-		<description>
-			If this setting is enabled and ACL based access control is active (the
-			AccessController coprocessor is installed either as a system coprocessor
-			or on a table as a table coprocessor) then you must grant all relevant
-			users EXEC privilege if they require the ability to execute coprocessor
-			endpoint calls. EXEC privilege, like any other permission, can be
-			granted globally to a user, or to a user on a per table or per namespace
-			basis. For more information on coprocessor endpoints, see the coprocessor
-			section of the HBase online manual. For more information on granting or
-			revoking permissions using the AccessController, see the security
-			section of the HBase online manual.
-		</description>
-	</property>
-	<property>
-		<name>hbase.procedure.regionserver.classes</name>
-		<value></value>
-		<description>A comma-separated list of
-			org.apache.hadoop.hbase.procedure.RegionServerProcedureManager procedure managers that are
-			loaded by default on the active HRegionServer process. The lifecycle methods (init/start/stop)
-			will be called by the active HRegionServer process to perform the specific globally barriered
-			procedure. After implementing your own RegionServerProcedureManager, just put it in
-			HBase's classpath and add the fully qualified class name here.
-		</description>
-	</property>
-	<property>
-		<name>hbase.procedure.master.classes</name>
-		<value></value>
-		<description>A comma-separated list of
-			org.apache.hadoop.hbase.procedure.MasterProcedureManager procedure managers that are
-			loaded by default on the active HMaster process. A procedure is identified by its signature and
-			users can use the signature and an instant name to trigger an execution of a globally barriered
-			procedure. After implementing your own MasterProcedureManager, just put it in HBase's classpath
-			and add the fully qualified class name here.</description>
-	</property>
-	<property>
-		<name>hbase.regionserver.storefile.refresh.period</name>
-		<value>0</value>
-		<description>
-			The period (in milliseconds) for refreshing the store files for the secondary regions. 0
-			means this feature is disabled. Secondary regions sees new files (from flushes and
-			compactions) from primary once the secondary region refreshes the list of files in the
-			region (there is no notification mechanism). But too frequent refreshes might cause
-			extra Namenode pressure. If the files cannot be refreshed for longer than HFile TTL
-			(hbase.master.hfilecleaner.ttl) the requests are rejected. Configuring HFile TTL to a larger
-			value is also recommended with this setting.
-		</description>
-	</property>
-	<property>
-		<name>hbase.region.replica.replication.enabled</name>
-		<value>false</value>
-		<description>
-			Whether asynchronous WAL replication to the secondary region replicas is enabled or not.
-			If this is enabled, a replication peer named "region_replica_replication" will be created
-			which will tail the logs and replicate the mutations to region replicas for tables that
-			have region replication > 1. If this is enabled once, disabling this replication also
-			requires disabling the replication peer using shell or Admin java class.
-			Replication to secondary region replicas works over standard inter-cluster replication.
-		</description>
-	</property>
-	<property>
-		<name>hbase.security.visibility.mutations.checkauths</name>
-		<value>false</value>
-		<description>
-			This property if enabled, will check whether the labels in the visibility
-			expression are associated with the user issuing the mutation
-		</description>
-	</property>
-	<property>
-		<name>hbase.http.max.threads</name>
-		<value>16</value>
-		<description>
-			The maximum number of threads that the HTTP Server will create in its
-			ThreadPool.
-		</description>
-	</property>
-	<property>
-		<name>hbase.replication.source.maxthreads</name>
-		<value>10</value>
-		<description>
-			The maximum number of threads any replication source will use for
-			shipping edits to the sinks in parallel. This also limits the number of
-			chunks each replication batch is broken into. Larger values can improve
-			the replication throughput between the master and slave clusters. The
-			default of 10 will rarely need to be changed.
-		</description>
-	</property>
-	<!-- Static Web User Filter properties. -->
-	<property>
-		<name>hbase.http.staticuser.user</name>
-		<value>dr.stack</value>
-		<description>
-			The user name to filter as, on static web filters
-			while rendering content. An example use is the HDFS
-			web UI (user to be used for browsing files).
-		</description>
-	</property>
-	<property>
-		<name>hbase.regionserver.handler.abort.on.error.percent</name>
-		<value>0.5</value>
-		<description>The percent of region server RPC threads failed to abort RS.
-			-1 Disable aborting; 0 Abort if even a single handler has died;
-			0.x Abort only when this percent of handlers have died;
-			1 Abort only all of the handers have died.</description>
-	</property>
-	<!-- Mob properties. -->
-	<property>
-		<name>hbase.mob.file.cache.size</name>
-		<value>1000</value>
-		<description>
-			Number of opened file handlers to cache.
-			A larger value will benefit reads by providing more file handlers per mob
-			file cache and would reduce frequent file opening and closing.
-			However, if this is set too high, this could lead to a "too many opened file handlers"
-			The default value is 1000.
-		</description>
-	</property>
-	<property>
-		<name>hbase.mob.cache.evict.period</name>
-		<value>3600</value>
-		<description>
-			The amount of time in seconds before the mob cache evicts cached mob files.
-			The default value is 3600 seconds.
-		</description>
-	</property>
-	<property>
-		<name>hbase.mob.cache.evict.remain.ratio</name>
-		<value>0.5f</value>
-		<description>
-			The ratio (between 0.0 and 1.0) of files that remains cached after an eviction
-			is triggered when the number of cached mob files exceeds the hbase.mob.file.cache.size.
-			The default value is 0.5f.
-		</description>
-	</property>
-	<property>
-		<name>hbase.master.mob.ttl.cleaner.period</name>
-		<value>86400</value>
-		<description>
-			The period that ExpiredMobFileCleanerChore runs. The unit is second.
-			The default value is one day. The MOB file name uses only the date part of
-			the file creation time in it. We use this time for deciding TTL expiry of
-			the files. So the removal of TTL expired files might be delayed. The max
-			delay might be 24 hrs.
-		</description>
-	</property>
-	<property>
-		<name>hbase.mob.compaction.mergeable.threshold</name>
-		<value>1342177280</value>
-		<description>
-			If the size of a mob file is less than this value, it's regarded as a small
-			file and needs to be merged in mob compaction. The default value is 1280MB.
-		</description>
-	</property>
-	<property>
-		<name>hbase.mob.delfile.max.count</name>
-		<value>3</value>
-		<description>
-			The max number of del files that is allowed in the mob compaction.
-			In the mob compaction, when the number of existing del files is larger than
-			this value, they are merged until number of del files is not larger this value.
-			The default value is 3.
-		</description>
-	</property>
-	<property>
-		<name>hbase.mob.compaction.batch.size</name>
-		<value>100</value>
-		<description>
-			The max number of the mob files that is allowed in a batch of the mob compaction.
-			The mob compaction merges the small mob files to bigger ones. If the number of the
-			small files is very large, it could lead to a "too many opened file handlers" in the merge.
-			And the merge has to be split into batches. This value limits the number of mob files
-			that are selected in a batch of the mob compaction. The default value is 100.
-		</description>
-	</property>
-	<property>
-		<name>hbase.mob.compaction.chore.period</name>
-		<value>604800</value>
-		<description>
-			The period that MobCompactionChore runs. The unit is second.
-			The default value is one week.
-		</description>
-	</property>
-	<property>
-		<name>hbase.mob.compaction.threads.max</name>
-		<value>1</value>
-		<description>
-			The max number of threads used in MobCompactor.
-		</description>
-	</property>
-	<property>
-		<name>hbase.snapshot.master.timeout.millis</name>
-		<value>300000</value>
-		<description>
-			Timeout for master for the snapshot procedure execution.
-		</description>
-	</property>
-	<property>
-		<name>hbase.snapshot.region.timeout</name>
-		<value>300000</value>
-		<description>
-			Timeout for regionservers to keep threads in snapshot request pool waiting.
-		</description>
-	</property>
-	<property>
-		<name>hbase.rpc.rows.warning.threshold</name>
-		<value>5000</value>
-		<description>
-			Number of rows in a batch operation above which a warning will be logged.
-		</description>
-	</property>
-	<property>
-		<name>hbase.master.wait.on.service.seconds</name>
-		<value>30</value>
-		<description>Default is 5 minutes. Make it 30 seconds for tests. See
-			HBASE-19794 for some context.</description>
-	</property>
-
-	<!--NOTE: HBase client try to load the class that configured in hbase-default.xml. -->
-	<!--But actually all these classes were already shaded and can't be loaded by those default name, -->
-	<!--so the following classes are Flink shaded classes.-->
-	<property>
-		<name>hbase.master.logcleaner.plugins</name>
-		<value>org.apache.flink.hbase.shaded.org.apache.hadoop.hbase.master.cleaner.TimeToLiveLogCleaner,org.apache.flink.hbase.shaded.org.apache.hadoop.hbase.master.cleaner.TimeToLiveProcedureWALCleaner</value>
-		<description>A comma-separated list of BaseLogCleanerDelegate invoked by
-			the LogsCleaner service. These WAL cleaners are called in order,
-			so put the cleaner that prunes the most files in front. To
-			implement your own BaseLogCleanerDelegate, just put it in HBase's classpath
-			and add the fully qualified class name here. Always add the above
-			default log cleaners in the list.</description>
-	</property>
-	<property>
-		<name>hbase.master.hfilecleaner.plugins</name>
-		<value>org.apache.flink.hbase.shaded.org.apache.hadoop.hbase.master.cleaner.TimeToLiveHFileCleaner</value>
-		<description>A comma-separated list of BaseHFileCleanerDelegate invoked by
-			the HFileCleaner service. These HFiles cleaners are called in order,
-			so put the cleaner that prunes the most files in front. To
-			implement your own BaseHFileCleanerDelegate, just put it in HBase's classpath
-			and add the fully qualified class name here. Always add the above
-			default log cleaners in the list as they will be overwritten in
-			hbase-site.xml.</description>
-	</property>
-	<property>
-		<name>hbase.regionserver.hlog.reader.impl</name>
-		<value>org.apache.flink.hbase.shaded.org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader</value>
-		<description>The WAL file reader implementation.</description>
-	</property>
-	<property>
-		<name>hbase.regionserver.hlog.writer.impl</name>
-		<value>org.apache.flink.hbase.shaded.org.apache.hadoop.hbase.regionserver.wal.ProtobufLogWriter</value>
-		<description>The WAL file writer implementation.</description>
-	</property>
-	<property>
-		<name>hbase.regionserver.region.split.policy</name>
-		<value>org.apache.flink.hbase.shaded.org.apache.hadoop.hbase.regionserver.SteppingSplitPolicy</value>
-		<description>
-			A split policy determines when a region should be split. The various
-			other split policies that are available currently are BusyRegionSplitPolicy,
-			ConstantSizeRegionSplitPolicy, DisabledRegionSplitPolicy,
-			DelimitedKeyPrefixRegionSplitPolicy, KeyPrefixRegionSplitPolicy, and
-			SteppingSplitPolicy. DisabledRegionSplitPolicy blocks manual region splitting.
-		</description>
-	</property>
-	<property>
-		<name>hbase.status.publisher.class</name>
-		<value>org.apache.flink.hbase.shaded.org.apache.hadoop.hbase.master.ClusterStatusPublisher$MulticastPublisher</value>
-		<description>
-			Implementation of the status publication with a multicast message.
-		</description>
-	</property>
-	<property>
-		<name>hbase.status.listener.class</name>
-		<value>org.apache.flink.hbase.shaded.org.apache.hadoop.hbase.client.ClusterStatusListener$MulticastListener</value>
-		<description>
-			Implementation of the status listener with a multicast message.
-		</description>
-	</property>
-	<property>
-		<name>hbase.rest.filter.classes</name>
-		<value>org.apache.flink.hbase.shaded.org.apache.hadoop.hbase.rest.filter.GzipFilter</value>
-		<description>
-			Servlet filters for REST service.
-		</description>
-	</property>
-	<property>
-		<name>hbase.master.loadbalancer.class</name>
-		<value>org.apache.flink.hbase.shaded.org.apache.hadoop.hbase.master.balancer.StochasticLoadBalancer</value>
-		<description>
-			Class used to execute the regions balancing when the period occurs.
-			See the class comment for more on how it works
-			http://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.html
-			It replaces the DefaultLoadBalancer as the default (since renamed
-			as the SimpleLoadBalancer).
-		</description>
-	</property>
-	<property>
-		<name>hbase.coordinated.state.manager.class</name>
-		<value>org.apache.flink.hbase.shaded.org.apache.hadoop.hbase.coordination.ZkCoordinatedStateManager</value>
-		<description>Fully qualified name of class implementing coordinated state manager.</description>
-	</property>
-	<property>
-		<name>hbase.http.filter.initializers</name>
-		<value>org.apache.flink.hbase.shaded.org.apache.hadoop.hbase.http.lib.StaticUserWebFilter</value>
-		<description>
-			A comma separated list of class names. Each class in the list must extend
-			org.apache.hadoop.hbase.http.FilterInitializer. The corresponding Filter will
-			be initialized. Then, the Filter will be applied to all user facing jsp
-			and servlet web pages.
-			The ordering of the list defines the ordering of the filters.
-			The default StaticUserWebFilter add a user principal as defined by the
-			hbase.http.staticuser.user property.
-		</description>
-	</property>
-	<property>
-		<name>hbase.replication.rpc.codec</name>
-		<value>org.apache.flink.hbase.shaded.org.apache.hadoop.hbase.codec.KeyValueCodecWithTags</value>
-		<description>
-			The codec that is to be used when replication is enabled so that
-			the tags are also replicated. This is used along with HFileV3 which
-			supports tags in them.  If tags are not used or if the hfile version used
-			is HFileV2 then KeyValueCodec can be used as the replication codec. Note that
-			using KeyValueCodecWithTags for replication when there are no tags causes no harm.
-		</description>
-	</property>
-	<property>
-		<name>hbase.master.normalizer.class</name>
-		<value>org.apache.flink.hbase.shaded.org.apache.hadoop.hbase.master.normalizer.SimpleRegionNormalizer</value>
-		<description>
-			Class used to execute the region normalization when the period occurs.
-			See the class comment for more on how it works
-			http://hbase.apache.org/devapidocs/org/apache/hadoop/hbase/master/normalizer/SimpleRegionNormalizer.html
-		</description>
-	</property>
-	<property>
-		<name>hbase.mob.compactor.class</name>
-		<value>org.apache.flink.hbase.shaded.org.apache.hadoop.hbase.mob.compactions.PartitionedMobCompactor</value>
-		<description>
-			Implementation of mob compactor, the default one is PartitionedMobCompactor.
-		</description>
-	</property>
-</configuration>
\ No newline at end of file
diff --git a/flink-connectors/pom.xml b/flink-connectors/pom.xml
index 850c577..dfb64fa 100644
--- a/flink-connectors/pom.xml
+++ b/flink-connectors/pom.xml
@@ -36,9 +36,6 @@
 
 	<modules>
 		<module>flink-hadoop-compatibility</module>
-		<module>flink-connector-hbase-base</module>
-		<module>flink-connector-hbase-1.4</module>
-		<module>flink-connector-hbase-2.2</module>
 		<module>flink-connector-hive</module>
 		<module>flink-connector-base</module>
 		<module>flink-file-sink-common</module>
@@ -80,8 +77,6 @@
 				</property>
 			</activation>
 			<modules>
-				<module>flink-sql-connector-hbase-1.4</module>
-				<module>flink-sql-connector-hbase-2.2</module>
 				<module>flink-sql-connector-hive-2.3.9</module>
 				<module>flink-sql-connector-hive-3.1.3</module>
 			</modules>
diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-hbase/pom.xml b/flink-end-to-end-tests/flink-end-to-end-tests-hbase/pom.xml
deleted file mode 100644
index c1a6877..0000000
--- a/flink-end-to-end-tests/flink-end-to-end-tests-hbase/pom.xml
+++ /dev/null
@@ -1,264 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-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.
--->
-<project xmlns="http://maven.apache.org/POM/4.0.0"
-		 xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-		 xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-	<parent>
-		<artifactId>flink-end-to-end-tests</artifactId>
-		<groupId>org.apache.flink</groupId>
-		<version>1.19-SNAPSHOT</version>
-	</parent>
-	<modelVersion>4.0.0</modelVersion>
-
-	<artifactId>flink-end-to-end-tests-hbase</artifactId>
-	<name>Flink : E2E Tests : HBase</name>
-
-	<dependencies>
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-end-to-end-tests-common</artifactId>
-			<version>${project.version}</version>
-		</dependency>
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-test-utils-junit</artifactId>
-			<scope>compile</scope>
-		</dependency>
-		<dependency>
-			<groupId>org.junit.jupiter</groupId>
-			<artifactId>junit-jupiter</artifactId>
-			<scope>compile</scope>
-		</dependency>
-
-		<!--using hbase shade jar to execute end-to-end test-->
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-sql-connector-hbase-1.4</artifactId>
-			<version>${project.version}</version>
-			<scope>test</scope>
-		</dependency>
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-sql-connector-hbase-2.2</artifactId>
-			<version>${project.version}</version>
-			<scope>test</scope>
-		</dependency>
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-sql-client-test</artifactId>
-			<version>${project.version}</version>
-			<scope>test</scope>
-		</dependency>
-
-		<!-- hadoop dependencies for hbase end-to-end test -->
-		<dependency>
-			<groupId>org.apache.hadoop</groupId>
-			<artifactId>hadoop-common</artifactId>
-			<version>${flink.hadoop.version}</version>
-			<scope>test</scope>
-			<exclusions>
-				<exclusion>
-					<groupId>log4j</groupId>
-					<artifactId>log4j</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.slf4j</groupId>
-					<artifactId>slf4j-log4j12</artifactId>
-				</exclusion>
-				<exclusion>
-					<!-- This dependency is no longer shipped with the JDK since Java 9.-->
-					<groupId>jdk.tools</groupId>
-					<artifactId>jdk.tools</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>ch.qos.reload4j</groupId>
-					<artifactId>reload4j</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.slf4j</groupId>
-					<artifactId>slf4j-reload4j</artifactId>
-				</exclusion>
-			</exclusions>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.hadoop</groupId>
-			<artifactId>hadoop-yarn-client</artifactId>
-			<version>${flink.hadoop.version}</version>
-			<scope>test</scope>
-			<exclusions>
-				<exclusion>
-					<groupId>log4j</groupId>
-					<artifactId>log4j</artifactId>
-				</exclusion>
-				<exclusion>
-					<!-- This dependency is no longer shipped with the JDK since Java 9.-->
-					<groupId>jdk.tools</groupId>
-					<artifactId>jdk.tools</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>ch.qos.reload4j</groupId>
-					<artifactId>reload4j</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.slf4j</groupId>
-					<artifactId>slf4j-reload4j</artifactId>
-				</exclusion>
-			</exclusions>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.hadoop</groupId>
-			<artifactId>hadoop-yarn-api</artifactId>
-			<version>${flink.hadoop.version}</version>
-			<scope>test</scope>
-			<exclusions>
-				<exclusion>
-					<!-- This dependency is no longer shipped with the JDK since Java 9.-->
-					<groupId>jdk.tools</groupId>
-					<artifactId>jdk.tools</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>ch.qos.reload4j</groupId>
-					<artifactId>reload4j</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.slf4j</groupId>
-					<artifactId>slf4j-reload4j</artifactId>
-				</exclusion>
-			</exclusions>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.hadoop</groupId>
-			<artifactId>hadoop-minicluster</artifactId>
-			<version>${flink.hadoop.version}</version>
-			<scope>test</scope>
-			<exclusions>
-				<exclusion>
-					<groupId>log4j</groupId>
-					<artifactId>log4j</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.slf4j</groupId>
-					<artifactId>slf4j-log4j12</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>ch.qos.reload4j</groupId>
-					<artifactId>reload4j</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.slf4j</groupId>
-					<artifactId>slf4j-reload4j</artifactId>
-				</exclusion>
-			</exclusions>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.hadoop</groupId>
-			<artifactId>hadoop-minikdc</artifactId>
-			<version>${minikdc.version}</version>
-			<scope>test</scope>
-			<exclusions>
-				<exclusion>
-					<groupId>org.slf4j</groupId>
-					<artifactId>slf4j-log4j12</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.slf4j</groupId>
-					<artifactId>slf4j-reload4j</artifactId>
-				</exclusion>
-			</exclusions>
-		</dependency>
-
-	</dependencies>
-
-	<build>
-		<plugins>
-			<plugin>
-				<groupId>org.apache.maven.plugins</groupId>
-				<artifactId>maven-enforcer-plugin</artifactId>
-				<executions>
-					<execution>
-						<id>dependency-convergence</id>
-						<goals>
-							<goal>enforce</goal>
-						</goals>
-						<configuration>
-							<skip>true</skip>
-						</configuration>
-					</execution>
-				</executions>
-			</plugin>
-			<plugin>
-				<groupId>org.apache.maven.plugins</groupId>
-				<artifactId>maven-dependency-plugin</artifactId>
-				<executions>
-					<execution>
-						<id>copy</id>
-						<phase>pre-integration-test</phase>
-						<goals>
-							<goal>copy</goal>
-						</goals>
-						<configuration>
-							<artifactItems>
-								<artifactItem>
-									<groupId>org.apache.flink</groupId>
-									<artifactId>flink-sql-client-test</artifactId>
-									<version>${project.version}</version>
-									<destFileName>SqlToolbox.jar</destFileName>
-									<type>jar</type>
-									<outputDirectory>${project.build.directory}/dependencies</outputDirectory>
-								</artifactItem>
-								<artifactItem>
-									<groupId>org.apache.flink</groupId>
-									<artifactId>flink-sql-connector-hbase-1.4</artifactId>
-									<version>${project.version}</version>
-									<destFileName>sql-hbase-1.4.jar</destFileName>
-									<type>jar</type>
-									<outputDirectory>${project.build.directory}/dependencies</outputDirectory>
-								</artifactItem>
-								<artifactItem>
-									<groupId>org.apache.flink</groupId>
-									<artifactId>flink-sql-connector-hbase-2.2</artifactId>
-									<version>${project.version}</version>
-									<destFileName>sql-hbase-2.2.jar</destFileName>
-									<type>jar</type>
-									<outputDirectory>${project.build.directory}/dependencies</outputDirectory>
-								</artifactItem>
-							</artifactItems>
-						</configuration>
-					</execution>
-					<execution>
-						<id>store-classpath-in-target-for-tests</id>
-						<phase>package</phase>
-						<goals>
-							<goal>build-classpath</goal>
-						</goals>
-						<configuration>
-							<outputFile>${project.build.directory}/hadoop.classpath</outputFile>
-							<excludeGroupIds>org.apache.flink</excludeGroupIds>
-						</configuration>
-					</execution>
-				</executions>
-			</plugin>
-		</plugins>
-	</build>
-</project>
diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-hbase/src/main/java/org/apache/flink/tests/util/hbase/HBaseResource.java b/flink-end-to-end-tests/flink-end-to-end-tests-hbase/src/main/java/org/apache/flink/tests/util/hbase/HBaseResource.java
deleted file mode 100644
index a085b1d..0000000
--- a/flink-end-to-end-tests/flink-end-to-end-tests-hbase/src/main/java/org/apache/flink/tests/util/hbase/HBaseResource.java
+++ /dev/null
@@ -1,79 +0,0 @@
-/*
- * 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.tests.util.hbase;
-
-import org.apache.flink.tests.util.util.FactoryUtils;
-import org.apache.flink.util.ExternalResource;
-
-import java.io.IOException;
-import java.util.List;
-
-/** Generic interface for interacting with HBase. */
-public interface HBaseResource extends ExternalResource {
-
-    /**
-     * Creates a table with the given name and column families.
-     *
-     * @param tableName desired table name
-     * @param columnFamilies column family to create
-     * @throws IOException
-     */
-    void createTable(String tableName, String... columnFamilies) throws IOException;
-
-    /**
-     * Scan the given HBase table.
-     *
-     * @param tableName table desired to scan
-     * @throws IOException
-     */
-    List<String> scanTable(String tableName) throws IOException;
-
-    /**
-     * Put the given data to the given table.
-     *
-     * @param tableName table to put data
-     * @param rowKey row key of the given data
-     * @param columnFamily column family of the given data
-     * @param columnQualifier column qualifier of the given data
-     * @param value value of the given data
-     * @throws IOException
-     */
-    void putData(
-            String tableName,
-            String rowKey,
-            String columnFamily,
-            String columnQualifier,
-            String value)
-            throws IOException;
-
-    /**
-     * Returns the configured HBaseResource implementation, or a {@link
-     * LocalStandaloneHBaseResource} if none is configured.
-     *
-     * @param version The hbase version
-     * @return configured HbaseResource, or {@link LocalStandaloneHBaseResource} if none is
-     *     configured
-     */
-    static HBaseResource get(String version) {
-        return FactoryUtils.loadAndInvokeFactory(
-                HBaseResourceFactory.class,
-                factory -> factory.create(version),
-                LocalStandaloneHBaseResourceFactory::new);
-    }
-}
diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-hbase/src/main/java/org/apache/flink/tests/util/hbase/HBaseResourceFactory.java b/flink-end-to-end-tests/flink-end-to-end-tests-hbase/src/main/java/org/apache/flink/tests/util/hbase/HBaseResourceFactory.java
deleted file mode 100644
index aec789f..0000000
--- a/flink-end-to-end-tests/flink-end-to-end-tests-hbase/src/main/java/org/apache/flink/tests/util/hbase/HBaseResourceFactory.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * 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.tests.util.hbase;
-
-import java.util.Optional;
-
-/** A factory for {@link HBaseResource} implementations. */
-@FunctionalInterface
-public interface HBaseResourceFactory {
-
-    /**
-     * Returns a {@link HBaseResource} instance. If the instance could not be instantiated (for
-     * example, because a mandatory parameter was missing), then an empty {@link Optional} should be
-     * returned.
-     *
-     * @param version The hbase version
-     * @return HBaseResource instance
-     * @throws Exception if the instance could not be instantiated
-     */
-    HBaseResource create(String version) throws Exception;
-}
diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-hbase/src/main/java/org/apache/flink/tests/util/hbase/LocalStandaloneHBaseResource.java b/flink-end-to-end-tests/flink-end-to-end-tests-hbase/src/main/java/org/apache/flink/tests/util/hbase/LocalStandaloneHBaseResource.java
deleted file mode 100644
index 43df763..0000000
--- a/flink-end-to-end-tests/flink-end-to-end-tests-hbase/src/main/java/org/apache/flink/tests/util/hbase/LocalStandaloneHBaseResource.java
+++ /dev/null
@@ -1,246 +0,0 @@
-/*
- * 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.tests.util.hbase;
-
-import org.apache.flink.tests.util.AutoClosableProcess;
-import org.apache.flink.tests.util.CommandLineWrapper;
-import org.apache.flink.tests.util.activation.OperatingSystemRestriction;
-import org.apache.flink.tests.util.cache.DownloadCache;
-import org.apache.flink.util.OperatingSystem;
-
-import org.junit.rules.TemporaryFolder;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.nio.file.Files;
-import java.nio.file.Path;
-import java.nio.file.Paths;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.function.Consumer;
-import java.util.function.Supplier;
-import java.util.stream.Collectors;
-
-/** {@link HBaseResource} that downloads hbase and set up a local hbase cluster. */
-public class LocalStandaloneHBaseResource implements HBaseResource {
-
-    private static final Logger LOG = LoggerFactory.getLogger(LocalStandaloneHBaseResource.class);
-
-    private static final int MAX_RETRIES = 3;
-    private static final int RETRY_INTERVAL_SECONDS = 30;
-    private final TemporaryFolder tmp = new TemporaryFolder();
-
-    private final DownloadCache downloadCache = DownloadCache.get();
-    private final String hbaseVersion;
-    private Path hbaseDir;
-
-    LocalStandaloneHBaseResource(String hbaseVersion) {
-        OperatingSystemRestriction.forbid(
-                String.format(
-                        "The %s relies on UNIX utils and shell scripts.",
-                        getClass().getSimpleName()),
-                OperatingSystem.WINDOWS);
-        this.hbaseVersion = hbaseVersion;
-    }
-
-    private String getHBaseDownloadUrl() {
-        return String.format(
-                "https://archive.apache.org/dist/hbase/%1$s/hbase-%1$s-bin.tar.gz", hbaseVersion);
-    }
-
-    @Override
-    public void before() throws Exception {
-        tmp.create();
-        downloadCache.before();
-
-        this.hbaseDir = tmp.newFolder("hbase-" + hbaseVersion).toPath().toAbsolutePath();
-        setupHBaseDist();
-        setupHBaseCluster();
-    }
-
-    private void setupHBaseDist() throws IOException {
-        final Path downloadDirectory = tmp.newFolder("getOrDownload").toPath();
-        final Path hbaseArchive =
-                downloadCache.getOrDownload(getHBaseDownloadUrl(), downloadDirectory);
-
-        LOG.info("HBase location: {}", hbaseDir.toAbsolutePath());
-        AutoClosableProcess.runBlocking(
-                CommandLineWrapper.tar(hbaseArchive)
-                        .extract()
-                        .zipped()
-                        .strip(1)
-                        .targetDir(hbaseDir)
-                        .build());
-
-        LOG.info("Configure {} as hbase.tmp.dir", hbaseDir.toAbsolutePath());
-        final String tmpDirConfig =
-                "<configuration><property><name>hbase.tmp.dir</name><value>"
-                        + hbaseDir
-                        + "</value></property></configuration>";
-        Files.write(hbaseDir.resolve(Paths.get("conf", "hbase-site.xml")), tmpDirConfig.getBytes());
-    }
-
-    private void setupHBaseCluster() throws IOException {
-        LOG.info("Starting HBase cluster...");
-        runHBaseProcessWithRetry("start-hbase.sh", () -> !isHMasterRunning());
-        LOG.info("Start HBase cluster success");
-    }
-
-    @Override
-    public void afterTestSuccess() {
-        shutdownResource();
-        downloadCache.afterTestSuccess();
-        tmp.delete();
-    }
-
-    private void shutdownResource() {
-        LOG.info("Stopping HBase Cluster...");
-        try {
-            runHBaseProcessWithRetry("stop-hbase.sh", () -> isHMasterAlive());
-        } catch (IOException ioe) {
-            LOG.warn("Error when shutting down HBase Cluster.", ioe);
-        }
-        LOG.info("Stop HBase Cluster success");
-    }
-
-    private void runHBaseProcessWithRetry(String command, Supplier<Boolean> processStatusChecker)
-            throws IOException {
-        LOG.info("Execute {} for HBase Cluster", command);
-
-        for (int i = 1; i <= MAX_RETRIES; i++) {
-            try {
-                AutoClosableProcess.runBlocking(
-                        hbaseDir.resolve(Paths.get("bin", command)).toString());
-            } catch (IOException ioe) {
-                LOG.warn("Get exception when execute {} ", command, ioe);
-            }
-
-            int waitSecond = 0;
-            while (processStatusChecker.get()) {
-                try {
-                    LOG.info("Waiting for HBase {} works", command);
-                    Thread.sleep(1000L);
-                } catch (InterruptedException e) {
-                    LOG.warn("sleep interrupted", e);
-                }
-                waitSecond++;
-                if (waitSecond > RETRY_INTERVAL_SECONDS) {
-                    break;
-                }
-            }
-
-            if (waitSecond < RETRY_INTERVAL_SECONDS) {
-                break;
-            } else {
-                if (i == MAX_RETRIES) {
-                    LOG.error("Execute {} failed, retry times {}", command, i);
-                    throw new IllegalArgumentException(
-                            String.format("Execute %s failed aftert retry %s times", command, i));
-                } else {
-                    LOG.warn("Execute {} failed, retry times {}", command, i);
-                }
-            }
-        }
-    }
-
-    private boolean isHMasterRunning() {
-        try {
-            final AtomicBoolean atomicHMasterStarted = new AtomicBoolean(false);
-            queryHBaseStatus(
-                    line ->
-                            atomicHMasterStarted.compareAndSet(
-                                    false, line.contains("hbase:namespace")));
-            return atomicHMasterStarted.get();
-        } catch (IOException ioe) {
-            return false;
-        }
-    }
-
-    private void queryHBaseStatus(final Consumer<String> stdoutProcessor) throws IOException {
-        executeHBaseShell("scan 'hbase:meta'", stdoutProcessor);
-    }
-
-    private boolean isHMasterAlive() {
-        try {
-            final AtomicBoolean atomicHMasterStarted = new AtomicBoolean(false);
-            queryHBaseProcess(
-                    line -> atomicHMasterStarted.compareAndSet(false, line.contains("HMaster")));
-            return atomicHMasterStarted.get();
-        } catch (IOException ioe) {
-            return false;
-        }
-    }
-
-    private void queryHBaseProcess(final Consumer<String> stdoutProcessor) throws IOException {
-        AutoClosableProcess.create("jps").setStdoutProcessor(stdoutProcessor).runBlocking();
-    }
-
-    @Override
-    public void createTable(String tableName, String... columnFamilies) throws IOException {
-        final String createTable =
-                String.format("create '%s',", tableName)
-                        + Arrays.stream(columnFamilies)
-                                .map(cf -> String.format("{NAME=>'%s'}", cf))
-                                .collect(Collectors.joining(","));
-
-        executeHBaseShell(createTable);
-    }
-
-    @Override
-    public List<String> scanTable(String tableName) throws IOException {
-        final List<String> result = new ArrayList<>();
-        executeHBaseShell(
-                String.format("scan '%s'", tableName),
-                line -> {
-                    if (line.contains("value=")) {
-                        result.add(line);
-                    }
-                });
-        return result;
-    }
-
-    @Override
-    public void putData(
-            String tableName,
-            String rowKey,
-            String columnFamily,
-            String columnQualifier,
-            String value)
-            throws IOException {
-        executeHBaseShell(
-                String.format(
-                        "put '%s','%s','%s:%s','%s'",
-                        tableName, rowKey, columnFamily, columnQualifier, value));
-    }
-
-    private void executeHBaseShell(String cmd) throws IOException {
-        executeHBaseShell(cmd, line -> {});
-    }
-
-    private void executeHBaseShell(String cmd, Consumer<String> stdoutProcessor)
-            throws IOException {
-        AutoClosableProcess.create(hbaseDir.resolve(Paths.get("bin", "hbase")).toString(), "shell")
-                .setStdoutProcessor(stdoutProcessor)
-                .setStdInputs(cmd)
-                .runBlocking();
-    }
-}
diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-hbase/src/main/java/org/apache/flink/tests/util/hbase/LocalStandaloneHBaseResourceFactory.java b/flink-end-to-end-tests/flink-end-to-end-tests-hbase/src/main/java/org/apache/flink/tests/util/hbase/LocalStandaloneHBaseResourceFactory.java
deleted file mode 100644
index d624668..0000000
--- a/flink-end-to-end-tests/flink-end-to-end-tests-hbase/src/main/java/org/apache/flink/tests/util/hbase/LocalStandaloneHBaseResourceFactory.java
+++ /dev/null
@@ -1,28 +0,0 @@
-/*
- * 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.tests.util.hbase;
-
-/** A {@link HBaseResourceFactory} for the {@link LocalStandaloneHBaseResourceFactory}. */
-public class LocalStandaloneHBaseResourceFactory implements HBaseResourceFactory {
-
-    @Override
-    public HBaseResource create(String version) {
-        return new LocalStandaloneHBaseResource(version);
-    }
-}
diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-hbase/src/test/java/org/apache/flink/tests/util/hbase/SQLClientHBaseITCase.java b/flink-end-to-end-tests/flink-end-to-end-tests-hbase/src/test/java/org/apache/flink/tests/util/hbase/SQLClientHBaseITCase.java
deleted file mode 100644
index aaf1579..0000000
--- a/flink-end-to-end-tests/flink-end-to-end-tests-hbase/src/test/java/org/apache/flink/tests/util/hbase/SQLClientHBaseITCase.java
+++ /dev/null
@@ -1,243 +0,0 @@
-/*
- * 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.tests.util.hbase;
-
-import org.apache.flink.api.common.time.Deadline;
-import org.apache.flink.test.resources.ResourceTestUtils;
-import org.apache.flink.test.util.SQLJobSubmission;
-import org.apache.flink.tests.util.cache.DownloadCache;
-import org.apache.flink.tests.util.flink.ClusterController;
-import org.apache.flink.tests.util.flink.FlinkResource;
-import org.apache.flink.tests.util.flink.FlinkResourceSetup;
-import org.apache.flink.tests.util.flink.LocalStandaloneFlinkResourceFactory;
-import org.apache.flink.testutils.junit.FailsOnJava11;
-import org.apache.flink.testutils.junit.FailsOnJava17;
-import org.apache.flink.util.FileUtils;
-import org.apache.flink.util.TestLogger;
-
-import org.hamcrest.CoreMatchers;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.ClassRule;
-import org.junit.Ignore;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.junit.rules.TemporaryFolder;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.File;
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.net.URL;
-import java.nio.file.Files;
-import java.nio.file.Path;
-import java.nio.file.Paths;
-import java.time.Duration;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.stream.Collectors;
-
-import static org.hamcrest.Matchers.arrayContainingInAnyOrder;
-import static org.hamcrest.Matchers.containsString;
-import static org.junit.Assert.assertThat;
-
-/** End-to-end test for the HBase connectors. */
-@RunWith(Parameterized.class)
-@Category(value = {FailsOnJava11.class, FailsOnJava17.class})
-@Ignore("FLINK-21519")
-public class SQLClientHBaseITCase extends TestLogger {
-
-    private static final Logger LOG = LoggerFactory.getLogger(SQLClientHBaseITCase.class);
-
-    private static final String HBASE_E2E_SQL = "hbase_e2e.sql";
-
-    @Parameterized.Parameters(name = "{index}: hbase-version:{0}")
-    public static Collection<Object[]> data() {
-        return Arrays.asList(
-                new Object[][] {
-                    {"1.4.3", "hbase-1.4"},
-                    {"2.2.3", "hbase-2.2"}
-                });
-    }
-
-    @Rule public final HBaseResource hbase;
-
-    @Rule
-    public final FlinkResource flink =
-            new LocalStandaloneFlinkResourceFactory().create(FlinkResourceSetup.builder().build());
-
-    @Rule public final TemporaryFolder tmp = new TemporaryFolder();
-
-    private final String hbaseConnector;
-    private final Path sqlConnectorHBaseJar;
-
-    @ClassRule public static final DownloadCache DOWNLOAD_CACHE = DownloadCache.get();
-
-    private static final Path sqlToolBoxJar = ResourceTestUtils.getResource(".*SqlToolbox.jar");
-    private static final Path hadoopClasspath = ResourceTestUtils.getResource(".*hadoop.classpath");
-    private List<Path> hadoopClasspathJars;
-
-    public SQLClientHBaseITCase(String hbaseVersion, String hbaseConnector) {
-        this.hbase = HBaseResource.get(hbaseVersion);
-        this.hbaseConnector = hbaseConnector;
-        this.sqlConnectorHBaseJar =
-                ResourceTestUtils.getResource(".*sql-" + hbaseConnector + ".jar");
-    }
-
-    @Before
-    public void before() throws Exception {
-        DOWNLOAD_CACHE.before();
-        Path tmpPath = tmp.getRoot().toPath();
-        LOG.info("The current temporary path: {}", tmpPath);
-
-        // Prepare all hadoop jars to mock HADOOP_CLASSPATH, use hadoop.classpath which contains all
-        // hadoop jars
-        File hadoopClasspathFile = new File(hadoopClasspath.toAbsolutePath().toString());
-
-        if (!hadoopClasspathFile.exists()) {
-            throw new FileNotFoundException(
-                    "File that contains hadoop classpath "
-                            + hadoopClasspath.toString()
-                            + " does not exist.");
-        }
-
-        String classPathContent = FileUtils.readFileUtf8(hadoopClasspathFile);
-        hadoopClasspathJars =
-                Arrays.stream(classPathContent.split(":"))
-                        .map(jar -> Paths.get(jar))
-                        .collect(Collectors.toList());
-    }
-
-    @Test
-    public void testHBase() throws Exception {
-        try (ClusterController clusterController = flink.startCluster(2)) {
-            // Create table and put data
-            hbase.createTable("source", "family1", "family2");
-            hbase.createTable("sink", "family1", "family2");
-            hbase.putData("source", "row1", "family1", "f1c1", "v1");
-            hbase.putData("source", "row1", "family2", "f2c1", "v2");
-            hbase.putData("source", "row1", "family2", "f2c2", "v3");
-            hbase.putData("source", "row2", "family1", "f1c1", "v4");
-            hbase.putData("source", "row2", "family2", "f2c1", "v5");
-            hbase.putData("source", "row2", "family2", "f2c2", "v6");
-
-            // Initialize the SQL statements from "hbase_e2e.sql" file
-            Map<String, String> varsMap = new HashMap<>();
-            varsMap.put("$HBASE_CONNECTOR", hbaseConnector);
-            List<String> sqlLines = initializeSqlLines(varsMap);
-
-            // Execute SQL statements in "hbase_e2e.sql" file
-            executeSqlStatements(clusterController, sqlLines);
-
-            LOG.info("Verify the sink table result.");
-            // Wait until all the results flushed to the HBase sink table.
-            checkHBaseSinkResult();
-            LOG.info("The HBase SQL client test run successfully.");
-        }
-    }
-
-    private void checkHBaseSinkResult() throws Exception {
-        boolean success = false;
-        final Deadline deadline = Deadline.fromNow(Duration.ofSeconds(120));
-        while (deadline.hasTimeLeft()) {
-            final List<String> lines = hbase.scanTable("sink");
-            if (lines.size() == 6) {
-                success = true;
-                assertThat(
-                        lines.toArray(new String[0]),
-                        arrayContainingInAnyOrder(
-                                CoreMatchers.allOf(
-                                        containsString("row1"),
-                                        containsString("family1"),
-                                        containsString("f1c1"),
-                                        containsString("value1")),
-                                CoreMatchers.allOf(
-                                        containsString("row1"),
-                                        containsString("family2"),
-                                        containsString("f2c1"),
-                                        containsString("v2")),
-                                CoreMatchers.allOf(
-                                        containsString("row1"),
-                                        containsString("family2"),
-                                        containsString("f2c2"),
-                                        containsString("v3")),
-                                CoreMatchers.allOf(
-                                        containsString("row2"),
-                                        containsString("family1"),
-                                        containsString("f1c1"),
-                                        containsString("value4")),
-                                CoreMatchers.allOf(
-                                        containsString("row2"),
-                                        containsString("family2"),
-                                        containsString("f2c1"),
-                                        containsString("v5")),
-                                CoreMatchers.allOf(
-                                        containsString("row2"),
-                                        containsString("family2"),
-                                        containsString("f2c2"),
-                                        containsString("v6"))));
-                break;
-            } else {
-                LOG.info(
-                        "The HBase sink table does not contain enough records, current {} records, left time: {}s",
-                        lines.size(),
-                        deadline.timeLeft().getSeconds());
-            }
-            Thread.sleep(500);
-        }
-        Assert.assertTrue("Did not get expected results before timeout.", success);
-    }
-
-    private List<String> initializeSqlLines(Map<String, String> vars) throws IOException {
-        URL url = SQLClientHBaseITCase.class.getClassLoader().getResource(HBASE_E2E_SQL);
-        if (url == null) {
-            throw new FileNotFoundException(HBASE_E2E_SQL);
-        }
-        List<String> lines = Files.readAllLines(new File(url.getFile()).toPath());
-        List<String> result = new ArrayList<>();
-        for (String line : lines) {
-            for (Map.Entry<String, String> var : vars.entrySet()) {
-                line = line.replace(var.getKey(), var.getValue());
-            }
-            result.add(line);
-        }
-
-        return result;
-    }
-
-    private void executeSqlStatements(ClusterController clusterController, List<String> sqlLines)
-            throws Exception {
-        LOG.info("Executing SQL: HBase source table -> HBase sink table");
-        clusterController.submitSQLJob(
-                new SQLJobSubmission.SQLJobSubmissionBuilder(sqlLines)
-                        .addJar(sqlToolBoxJar)
-                        .addJar(sqlConnectorHBaseJar)
-                        .addJars(hadoopClasspathJars)
-                        .build(),
-                Duration.ofMinutes(2L));
-    }
-}
diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-hbase/src/test/resources/hbase_e2e.sql b/flink-end-to-end-tests/flink-end-to-end-tests-hbase/src/test/resources/hbase_e2e.sql
deleted file mode 100644
index 5ae3845..0000000
--- a/flink-end-to-end-tests/flink-end-to-end-tests-hbase/src/test/resources/hbase_e2e.sql
+++ /dev/null
@@ -1,56 +0,0 @@
--- 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.
-
-CREATE TABLE MyHBaseSource (
-  rowkey STRING,
-  family1 ROW<f1c1 STRING>,
-  family2 ROW<f2c1 STRING, f2c2 STRING>
-) WITH (
-  'connector' = '$HBASE_CONNECTOR',
-  'table-name' = 'source',
-  'zookeeper.quorum' = 'localhost:2181',
-  'zookeeper.znode.parent' = '/hbase'
-);
-
-CREATE TABLE MyHBaseSink (
-  rowkey STRING,
-  family1 ROW<f1c1 STRING>,
-  family2 ROW<f2c1 STRING, f2c2 STRING>
-) WITH (
-  'connector' = '$HBASE_CONNECTOR',
-  'table-name' = 'sink',
-  'zookeeper.quorum' = 'localhost:2181',
-  'zookeeper.znode.parent' = '/hbase',
-  'sink.buffer-flush.max-rows' = '1',
-  'sink.buffer-flush.interval' = '2s'
-);
-
-CREATE FUNCTION RegReplace AS 'org.apache.flink.table.toolbox.StringRegexReplaceFunction';
-
-INSERT INTO MyHBaseSink
-SELECT
-  rowkey,
-  ROW(a),
-  ROW(b, c)
-FROM (
-  SELECT
-    rowkey,
-    RegReplace(family1.f1c1, 'v', 'value') as a,
-    family2.f2c1 as b,
-    family2.f2c2 as c
-  FROM MyHBaseSource)
-source;
-
diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-hbase/src/test/resources/log4j2-test.properties b/flink-end-to-end-tests/flink-end-to-end-tests-hbase/src/test/resources/log4j2-test.properties
deleted file mode 100644
index 835c2ec..0000000
--- a/flink-end-to-end-tests/flink-end-to-end-tests-hbase/src/test/resources/log4j2-test.properties
+++ /dev/null
@@ -1,28 +0,0 @@
-################################################################################
-#  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.
-################################################################################
-
-# Set root logger level to OFF to not flood build logs
-# set manually to INFO for debugging purposes
-rootLogger.level = OFF
-rootLogger.appenderRef.test.ref = TestLogger
-
-appender.testlogger.name = TestLogger
-appender.testlogger.type = CONSOLE
-appender.testlogger.target = SYSTEM_ERR
-appender.testlogger.layout.type = PatternLayout
-appender.testlogger.layout.pattern = %-4r [%t] %-5p %c %x - %m%n
diff --git a/flink-end-to-end-tests/pom.xml b/flink-end-to-end-tests/pom.xml
index 0cc5c1a..8a8e84d 100644
--- a/flink-end-to-end-tests/pom.xml
+++ b/flink-end-to-end-tests/pom.xml
@@ -70,7 +70,6 @@
 		<module>flink-tpcds-test</module>
 		<module>flink-netty-shuffle-memory-control-test</module>
 		<module>flink-python-test</module>
-		<module>flink-end-to-end-tests-hbase</module>
 		<module>flink-end-to-end-tests-scala</module>
 		<module>flink-end-to-end-tests-sql</module>
 		<module>flink-end-to-end-tests-jdbc-driver</module>
diff --git a/tools/ci/flink-ci-tools/src/main/resources/modules-defining-excess-dependencies.modulelist b/tools/ci/flink-ci-tools/src/main/resources/modules-defining-excess-dependencies.modulelist
index 11ff91e..62f8f6c 100644
--- a/tools/ci/flink-ci-tools/src/main/resources/modules-defining-excess-dependencies.modulelist
+++ b/tools/ci/flink-ci-tools/src/main/resources/modules-defining-excess-dependencies.modulelist
@@ -21,6 +21,4 @@
 
 flink-sql-connector-hive-2.3.9
 flink-sql-connector-hive-3.1.3
-flink-sql-connector-hbase-1.4
-flink-sql-connector-hbase-2.2
 flink-python
diff --git a/tools/ci/stage.sh b/tools/ci/stage.sh
index 66e100a..60dbc98 100755
--- a/tools/ci/stage.sh
+++ b/tools/ci/stage.sh
@@ -113,11 +113,6 @@
 flink-formats/flink-sql-orc,\
 flink-formats/flink-orc-nohive,\
 flink-connectors/flink-file-sink-common,\
-flink-connectors/flink-connector-hbase-base,\
-flink-connectors/flink-connector-hbase-1.4,\
-flink-connectors/flink-sql-connector-hbase-1.4,\
-flink-connectors/flink-connector-hbase-2.2,\
-flink-connectors/flink-sql-connector-hbase-2.2,\
 flink-connectors/flink-hadoop-compatibility,\
 flink-connectors,\
 flink-connectors/flink-connector-files,\