You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by "galaxylqx (via GitHub)" <gi...@apache.org> on 2023/03/10 03:44:02 UTC

[GitHub] [shardingsphere] galaxylqx opened a new pull request, #24536: Proxy For HBase add query and return result logic

galaxylqx opened a new pull request, #24536:
URL: https://github.com/apache/shardingsphere/pull/24536

   Fixes #14061.
   
   Changes proposed in this pull request:
   - add HBase query 
   - add HBase sql converter
   - add HBase return result
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@shardingsphere.apache.org

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


[GitHub] [shardingsphere] terrymanu commented on a diff in pull request #24536: Proxy For HBase add query and return result logic

Posted by "terrymanu (via GitHub)" <gi...@apache.org>.
terrymanu commented on code in PR #24536:
URL: https://github.com/apache/shardingsphere/pull/24536#discussion_r1133084525


##########
proxy/backend/type/hbase/src/main/java/org/apache/shardingsphere/proxy/backend/hbase/bean/HBaseOperation.java:
##########
@@ -0,0 +1,32 @@
+/*
+ * 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.shardingsphere.proxy.backend.hbase.bean;
+
+import lombok.AllArgsConstructor;
+import lombok.Getter;
+import org.apache.hadoop.hbase.client.Operation;
+
+@AllArgsConstructor
+@Getter
+public class HBaseOperation {
+    
+    private final String tableName;
+    
+    private final Operation operation;
+    

Review Comment:
   Please remove useless blank line



##########
proxy/backend/type/hbase/src/main/java/org/apache/shardingsphere/proxy/backend/hbase/context/HBaseRegionWarmUpContext.java:
##########
@@ -72,6 +74,24 @@ public void submitWarmUpTask(final String tableName, final HBaseCluster hbaseClu
         executorManager.submit(() -> loadRegionInfo(tableName, hbaseCluster));
     }
     
+    /**
+     * load one table region info.

Review Comment:
   1. The first letter should be upper-case
   2. Javadoc and @param should keep a blank line



##########
proxy/backend/type/hbase/src/main/java/org/apache/shardingsphere/proxy/backend/hbase/context/HBaseRegionWarmUpContext.java:
##########
@@ -72,6 +74,24 @@ public void submitWarmUpTask(final String tableName, final HBaseCluster hbaseClu
         executorManager.submit(() -> loadRegionInfo(tableName, hbaseCluster));
     }
     
+    /**
+     * load one table region info.
+     * @param tableName tableName

Review Comment:
   Please rewrite to `@param tableName table name`



##########
proxy/backend/type/hbase/src/main/java/org/apache/shardingsphere/proxy/backend/hbase/converter/HBaseDatabaseDeleteConverter.java:
##########
@@ -0,0 +1,71 @@
+/*
+ * 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.shardingsphere.proxy.backend.hbase.converter;
+
+import com.google.common.base.Preconditions;
+import lombok.AllArgsConstructor;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
+import org.apache.shardingsphere.infra.binder.statement.dml.DeleteStatementContext;
+import org.apache.shardingsphere.proxy.backend.hbase.bean.HBaseOperation;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.BinaryOperationExpression;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.ExpressionSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.InExpression;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.simple.LiteralExpressionSegment;
+import java.util.List;
+import java.util.stream.Collectors;
+
+@AllArgsConstructor
+public class HBaseDatabaseDeleteConverter extends HBaseDatabaseRowKeysConverterAdapter implements HBaseDatabaseConverter {

Review Comment:
   1. Please add final
   2. Please add java doc



##########
proxy/backend/type/hbase/src/main/java/org/apache/shardingsphere/proxy/backend/hbase/context/HBaseRegionWarmUpContext.java:
##########
@@ -72,6 +74,24 @@ public void submitWarmUpTask(final String tableName, final HBaseCluster hbaseClu
         executorManager.submit(() -> loadRegionInfo(tableName, hbaseCluster));
     }
     
+    /**
+     * load one table region info.
+     * @param tableName tableName
+     * @param connection hbase connection

Review Comment:
   HBase is keyword, please rename all `hbase` to `HBase`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@shardingsphere.apache.org

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


[GitHub] [shardingsphere] terrymanu commented on a diff in pull request #24536: Proxy For HBase add query and return result logic

Posted by "terrymanu (via GitHub)" <gi...@apache.org>.
terrymanu commented on code in PR #24536:
URL: https://github.com/apache/shardingsphere/pull/24536#discussion_r1134882295


##########
proxy/backend/type/hbase/src/main/java/org/apache/shardingsphere/proxy/backend/hbase/bean/HBaseOperation.java:
##########
@@ -0,0 +1,31 @@
+/*
+ * 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.shardingsphere.proxy.backend.hbase.bean;
+
+import lombok.AllArgsConstructor;
+import lombok.Getter;
+import org.apache.hadoop.hbase.client.Operation;
+
+@AllArgsConstructor
+@Getter
+public class HBaseOperation {

Review Comment:
   Please add final



##########
proxy/backend/type/hbase/src/main/java/org/apache/shardingsphere/proxy/backend/hbase/converter/HBaseDatabaseConverterFactory.java:
##########
@@ -0,0 +1,57 @@
+/*
+ * 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.shardingsphere.proxy.backend.hbase.converter;
+
+import lombok.AccessLevel;
+import lombok.NoArgsConstructor;
+import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
+import org.apache.shardingsphere.infra.binder.statement.dal.FlushStatementContext;
+import org.apache.shardingsphere.infra.binder.statement.dml.DeleteStatementContext;
+import org.apache.shardingsphere.infra.binder.statement.dml.InsertStatementContext;
+import org.apache.shardingsphere.infra.binder.statement.dml.SelectStatementContext;
+import org.apache.shardingsphere.infra.binder.statement.dml.UpdateStatementContext;
+import org.apache.shardingsphere.proxy.backend.hbase.exception.HBaseOperationException;
+
+/**
+ * Heterogeneous database converter factory.
+ */
+@NoArgsConstructor(access = AccessLevel.PRIVATE)
+public class HBaseDatabaseConverterFactory {

Review Comment:
   Please add final



##########
proxy/backend/type/hbase/src/main/java/org/apache/shardingsphere/proxy/backend/hbase/converter/HBaseDatabaseSelectConverter.java:
##########
@@ -0,0 +1,155 @@
+/*
+ * 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.shardingsphere.proxy.backend.hbase.converter;
+
+import lombok.AllArgsConstructor;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Query;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
+import org.apache.shardingsphere.infra.binder.statement.dml.SelectStatementContext;
+import org.apache.shardingsphere.proxy.backend.hbase.bean.HBaseOperation;
+import org.apache.shardingsphere.proxy.backend.hbase.util.HeterogeneousUtil;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.column.ColumnSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.BetweenExpression;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.BinaryOperationExpression;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.ExpressionSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.InExpression;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.simple.LiteralExpressionSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.pagination.PaginationValueSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.pagination.limit.NumberLiteralLimitValueSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.predicate.WhereSegment;
+import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dml.MySQLSelectStatement;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Optional;
+import java.util.stream.Collectors;
+
+@AllArgsConstructor
+@Slf4j
+public final class HBaseDatabaseSelectConverter extends HBaseDatabaseRowKeysConverterAdapter implements HBaseDatabaseConverter {
+    
+    private final SQLStatementContext<?> sqlStatementContext;
+    
+    @Override
+    public HBaseOperation convert() {
+        SelectStatementContext context = (SelectStatementContext) sqlStatementContext;
+        if (isUseGetRequest(context)) {
+            return createGetRequest(context);
+        } else {
+            return createScanRequest(context);
+        }
+    }
+    
+    boolean isUseGetRequest(final SelectStatementContext context) {
+        return context.getWhereSegments().stream().findFirst().isPresent()
+                && (context.getWhereSegments().stream().findFirst().get().getExpr() instanceof BinaryOperationExpression
+                        || context.getWhereSegments().stream().findFirst().get().getExpr() instanceof InExpression);
+    }
+    
+    private List<String> getRowKeyFromWhereSegment(final ExpressionSegment expressionSegment) {
+        if (expressionSegment instanceof InExpression) {
+            InExpression expression = (InExpression) expressionSegment;
+            return getRowKeysFromWhereSegmentByIn(expression);
+        } else {
+            BinaryOperationExpression expression = (BinaryOperationExpression) expressionSegment;
+            return new ArrayList<>(Collections.singleton(String.valueOf(((LiteralExpressionSegment) expression.getRight()).getLiterals())));
+        }
+    }
+    
+    private HBaseOperation createGetRequest(final SelectStatementContext context) {
+        ExpressionSegment expression = context.getWhereSegments().stream().findFirst().get().getExpr();
+        // 处理 in

Review Comment:
   Comment only can be English



##########
proxy/backend/type/hbase/src/main/java/org/apache/shardingsphere/proxy/backend/hbase/impl/HBaseDatabaseBackendUpdateHandler.java:
##########
@@ -0,0 +1,53 @@
+/*
+ * 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.shardingsphere.proxy.backend.hbase.impl;
+
+import lombok.AllArgsConstructor;
+import lombok.Getter;
+import org.apache.shardingsphere.infra.binder.SQLStatementContextFactory;
+import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
+import org.apache.shardingsphere.infra.executor.sql.execute.result.update.UpdateResult;
+import org.apache.shardingsphere.proxy.backend.handler.data.DatabaseBackendHandler;
+import org.apache.shardingsphere.proxy.backend.hbase.bean.HBaseOperation;
+import org.apache.shardingsphere.proxy.backend.hbase.converter.HBaseDatabaseConverter;
+import org.apache.shardingsphere.proxy.backend.hbase.converter.HBaseDatabaseConverterFactory;
+import org.apache.shardingsphere.proxy.backend.hbase.result.update.HBaseDatabaseUpdater;
+import org.apache.shardingsphere.proxy.backend.response.header.update.UpdateResponseHeader;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
+import java.util.Collection;
+
+/**
+ * HBase database backend updater factory.
+ */
+@AllArgsConstructor

Review Comment:
   Please use @RequiredArgsConstructor



##########
proxy/backend/type/hbase/src/main/java/org/apache/shardingsphere/proxy/backend/hbase/converter/HBaseDatabaseUpdateConverter.java:
##########
@@ -0,0 +1,91 @@
+/*
+ * 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.shardingsphere.proxy.backend.hbase.converter;
+
+import com.google.common.base.Preconditions;
+import lombok.AllArgsConstructor;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
+import org.apache.shardingsphere.infra.binder.statement.dml.UpdateStatementContext;
+import org.apache.shardingsphere.proxy.backend.hbase.bean.HBaseOperation;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.assignment.AssignmentSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.BinaryOperationExpression;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.ExpressionSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.InExpression;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.simple.LiteralExpressionSegment;
+import java.util.Collection;
+import java.util.List;
+import java.util.stream.Collectors;
+
+@AllArgsConstructor
+public final class HBaseDatabaseUpdateConverter extends HBaseDatabaseRowKeysConverterAdapter implements HBaseDatabaseConverter {
+    
+    private final SQLStatementContext<?> sqlStatementContext;
+    
+    @Override
+    public HBaseOperation convert() {
+        UpdateStatementContext context = (UpdateStatementContext) sqlStatementContext;
+        Preconditions.checkArgument(context.getWhereSegments().stream().findFirst().isPresent(), "Where segment is not present");
+        
+        if (context.getWhereSegments().stream().findFirst().get().getExpr() instanceof InExpression) {
+            return createHBasePutsOperation(context);
+        }
+        
+        return new HBaseOperation(context.getTablesContext().getTableNames().iterator().next(), createHBaseRequest(context));
+    }
+    
+    private HBaseOperation createHBasePutsOperation(final UpdateStatementContext context) {
+        List<String> rowKeys = getRowKeysFromWhereSegmentByIn((InExpression) context.getWhereSegments().stream().findFirst().get().getExpr());
+        List<Put> puts = rowKeys.stream().map(this::getPutByRowKey).collect(Collectors.toList());
+        for (Put put : puts) {
+            addPutColumn(context, put);
+        }
+        return new HBaseOperation(context.getTablesContext().getTableNames().iterator().next(),
+                new HBaseUpdateOperationAdapter(context.getTablesContext().getTableNames().iterator().next(), puts));
+    }
+    
+    private Put getPutByRowKey(final String rowKey) {
+        return new Put(Bytes.toBytes(rowKey));
+    }
+    
+    private Put createHBaseRequest(final UpdateStatementContext context) {
+        String rowKey = getRowKeyFromWhereSegment(context.getWhereSegments().stream().findFirst().get().getExpr());
+        Put put = getPutByRowKey(rowKey);
+        addPutColumn(context, put);
+        return put;

Review Comment:
   Result value should name as `result`



##########
proxy/backend/type/hbase/src/main/java/org/apache/shardingsphere/proxy/backend/hbase/converter/HBaseDatabaseDeleteConverter.java:
##########
@@ -0,0 +1,71 @@
+/*
+ * 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.shardingsphere.proxy.backend.hbase.converter;
+
+import com.google.common.base.Preconditions;
+import lombok.AllArgsConstructor;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
+import org.apache.shardingsphere.infra.binder.statement.dml.DeleteStatementContext;
+import org.apache.shardingsphere.proxy.backend.hbase.bean.HBaseOperation;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.BinaryOperationExpression;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.ExpressionSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.InExpression;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.simple.LiteralExpressionSegment;
+import java.util.List;
+import java.util.stream.Collectors;
+
+@AllArgsConstructor

Review Comment:
   Please use @RequiredArgsConstructor



##########
proxy/backend/type/hbase/src/main/java/org/apache/shardingsphere/proxy/backend/hbase/bean/HBaseOperation.java:
##########
@@ -0,0 +1,31 @@
+/*
+ * 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.shardingsphere.proxy.backend.hbase.bean;
+
+import lombok.AllArgsConstructor;
+import lombok.Getter;
+import org.apache.hadoop.hbase.client.Operation;
+
+@AllArgsConstructor

Review Comment:
   Please use @RequiredArgsConstructor



##########
proxy/backend/type/hbase/src/main/java/org/apache/shardingsphere/proxy/backend/hbase/converter/HBaseDatabaseDeleteConverter.java:
##########
@@ -0,0 +1,71 @@
+/*
+ * 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.shardingsphere.proxy.backend.hbase.converter;
+
+import com.google.common.base.Preconditions;
+import lombok.AllArgsConstructor;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
+import org.apache.shardingsphere.infra.binder.statement.dml.DeleteStatementContext;
+import org.apache.shardingsphere.proxy.backend.hbase.bean.HBaseOperation;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.BinaryOperationExpression;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.ExpressionSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.InExpression;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.simple.LiteralExpressionSegment;
+import java.util.List;
+import java.util.stream.Collectors;
+
+@AllArgsConstructor
+public final class HBaseDatabaseDeleteConverter extends HBaseDatabaseRowKeysConverterAdapter implements HBaseDatabaseConverter {

Review Comment:
   Please add java doc



##########
proxy/backend/type/hbase/src/main/java/org/apache/shardingsphere/proxy/backend/hbase/converter/HBaseDatabaseInsertConverter.java:
##########
@@ -0,0 +1,57 @@
+/*
+ * 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.shardingsphere.proxy.backend.hbase.converter;
+
+import lombok.AllArgsConstructor;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.shardingsphere.infra.binder.segment.insert.values.InsertValueContext;
+import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
+import org.apache.shardingsphere.infra.binder.statement.dml.InsertStatementContext;
+import org.apache.shardingsphere.proxy.backend.hbase.bean.HBaseOperation;
+import org.apache.shardingsphere.proxy.backend.hbase.context.HBaseContext;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.simple.LiteralExpressionSegment;
+import java.util.List;
+import java.util.stream.Collectors;
+
+@AllArgsConstructor

Review Comment:
   Please use @RequiredArgsConstructor



##########
proxy/backend/type/hbase/src/main/java/org/apache/shardingsphere/proxy/backend/hbase/converter/HBaseDatabaseInsertConverter.java:
##########
@@ -0,0 +1,57 @@
+/*
+ * 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.shardingsphere.proxy.backend.hbase.converter;
+
+import lombok.AllArgsConstructor;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.shardingsphere.infra.binder.segment.insert.values.InsertValueContext;
+import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
+import org.apache.shardingsphere.infra.binder.statement.dml.InsertStatementContext;
+import org.apache.shardingsphere.proxy.backend.hbase.bean.HBaseOperation;
+import org.apache.shardingsphere.proxy.backend.hbase.context.HBaseContext;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.simple.LiteralExpressionSegment;
+import java.util.List;
+import java.util.stream.Collectors;
+
+@AllArgsConstructor
+public final class HBaseDatabaseInsertConverter implements HBaseDatabaseConverter {
+    
+    private final SQLStatementContext<?> sqlStatementContext;
+    
+    @Override
+    public HBaseOperation convert() {
+        InsertStatementContext context = (InsertStatementContext) sqlStatementContext;
+        String tableName = context.getTablesContext().getTableNames().iterator().next();
+        return new HBaseOperation(tableName, new HBaseInsertOperationAdapter(createHBaseRequest(context)));
+    }
+    
+    private Put generateHBaseRequest(final InsertStatementContext context, final InsertValueContext insertValueContext) {
+        List<String> columns = context.getInsertColumnNames();
+        List<Object> values = insertValueContext.getValueExpressions().stream().map(each -> ((LiteralExpressionSegment) each).getLiterals()).collect(Collectors.toList());
+        Put put = new Put(Bytes.toBytes(String.valueOf(values.get(0))));
+        for (int i = 1; i < columns.size(); i++) {
+            put.addColumn(Bytes.toBytes(HBaseContext.getInstance().getColumnFamily()), Bytes.toBytes(String.valueOf(columns.get(i))), Bytes.toBytes(String.valueOf(values.get(i))));
+        }
+        return put;

Review Comment:
   Return value should name as `result`



##########
proxy/backend/type/hbase/src/main/java/org/apache/shardingsphere/proxy/backend/hbase/converter/HBaseDatabaseConverterFactory.java:
##########
@@ -0,0 +1,57 @@
+/*
+ * 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.shardingsphere.proxy.backend.hbase.converter;
+
+import lombok.AccessLevel;
+import lombok.NoArgsConstructor;
+import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
+import org.apache.shardingsphere.infra.binder.statement.dal.FlushStatementContext;
+import org.apache.shardingsphere.infra.binder.statement.dml.DeleteStatementContext;
+import org.apache.shardingsphere.infra.binder.statement.dml.InsertStatementContext;
+import org.apache.shardingsphere.infra.binder.statement.dml.SelectStatementContext;
+import org.apache.shardingsphere.infra.binder.statement.dml.UpdateStatementContext;
+import org.apache.shardingsphere.proxy.backend.hbase.exception.HBaseOperationException;
+
+/**
+ * Heterogeneous database converter factory.

Review Comment:
   should `Heterogeneous` rename to `HBase`?



##########
proxy/backend/type/hbase/src/main/java/org/apache/shardingsphere/proxy/backend/hbase/converter/HBaseDatabaseRowKeysConverterAdapter.java:
##########
@@ -0,0 +1,36 @@
+/*
+ * 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.shardingsphere.proxy.backend.hbase.converter;
+
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.ExpressionSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.InExpression;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.ListExpression;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.simple.LiteralExpressionSegment;
+import java.util.List;
+import java.util.stream.Collectors;
+
+public class HBaseDatabaseRowKeysConverterAdapter {

Review Comment:
   1. Please add final
   2. Please add javadoc



##########
proxy/backend/type/hbase/src/main/java/org/apache/shardingsphere/proxy/backend/hbase/converter/HBaseRegionReloadConverter.java:
##########
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.shardingsphere.proxy.backend.hbase.converter;
+
+import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
+import org.apache.shardingsphere.infra.binder.statement.dal.FlushStatementContext;
+import org.apache.shardingsphere.proxy.backend.hbase.bean.HBaseOperation;
+import java.util.List;
+import java.util.stream.Collectors;
+
+public final class HBaseRegionReloadConverter implements HBaseDatabaseConverter {
+    
+    private final SQLStatementContext<?> sqlStatementContext;
+    
+    public HBaseRegionReloadConverter(final SQLStatementContext<?> sqlStatementContext) {
+        this.sqlStatementContext = sqlStatementContext;
+    }
+    
+    @Override
+    public HBaseOperation convert() {
+        List<String> tables = ((FlushStatementContext) sqlStatementContext).getAllTables()
+                .stream().map(simpleTableSegment -> simpleTableSegment.getTableName().getIdentifier().getValue()).collect(Collectors.toList());
+        
+        return new HBaseOperation(String.join(",", tables), null);

Review Comment:
   Pass null  parameter is forbidden, please refactor it



##########
proxy/backend/type/hbase/src/main/java/org/apache/shardingsphere/proxy/backend/hbase/converter/HBaseSelectOperationAdapter.java:
##########
@@ -0,0 +1,44 @@
+/*
+ * 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.shardingsphere.proxy.backend.hbase.converter;
+
+import lombok.AllArgsConstructor;
+import lombok.Getter;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Operation;
+import java.util.List;
+import java.util.Map;
+
+@AllArgsConstructor

Review Comment:
   Please use @Requiredargsconstructor



##########
proxy/backend/type/hbase/src/main/java/org/apache/shardingsphere/proxy/backend/hbase/converter/HBaseDatabaseInsertConverter.java:
##########
@@ -0,0 +1,57 @@
+/*
+ * 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.shardingsphere.proxy.backend.hbase.converter;
+
+import lombok.AllArgsConstructor;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.shardingsphere.infra.binder.segment.insert.values.InsertValueContext;
+import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
+import org.apache.shardingsphere.infra.binder.statement.dml.InsertStatementContext;
+import org.apache.shardingsphere.proxy.backend.hbase.bean.HBaseOperation;
+import org.apache.shardingsphere.proxy.backend.hbase.context.HBaseContext;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.simple.LiteralExpressionSegment;
+import java.util.List;
+import java.util.stream.Collectors;
+
+@AllArgsConstructor
+public final class HBaseDatabaseInsertConverter implements HBaseDatabaseConverter {

Review Comment:
   Please add java doc



##########
proxy/backend/type/hbase/src/main/java/org/apache/shardingsphere/proxy/backend/hbase/exception/HBaseOperationException.java:
##########
@@ -25,7 +25,6 @@
  */
 @RequiredArgsConstructor
 @Getter
-// TODO should extend to ShardingSphereExternalException
 public final class HBaseOperationException extends RuntimeException {

Review Comment:
   Why remove the `TODO`? It seems not finish the task



##########
proxy/backend/type/hbase/src/main/java/org/apache/shardingsphere/proxy/backend/hbase/converter/HBaseSelectOperationAdapter.java:
##########
@@ -0,0 +1,44 @@
+/*
+ * 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.shardingsphere.proxy.backend.hbase.converter;
+
+import lombok.AllArgsConstructor;
+import lombok.Getter;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Operation;
+import java.util.List;
+import java.util.Map;
+
+@AllArgsConstructor
+@Getter
+public final class HBaseSelectOperationAdapter extends Operation {
+    
+    private final String tableName;
+    
+    private final List<Get> gets;
+    
+    @Override
+    public Map<String, Object> getFingerprint() {
+        return null;

Review Comment:
   return null is forbidden, please return empty Map



##########
proxy/backend/type/hbase/src/main/java/org/apache/shardingsphere/proxy/backend/hbase/converter/HBaseSelectOperationAdapter.java:
##########
@@ -0,0 +1,44 @@
+/*
+ * 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.shardingsphere.proxy.backend.hbase.converter;
+
+import lombok.AllArgsConstructor;
+import lombok.Getter;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Operation;
+import java.util.List;
+import java.util.Map;
+
+@AllArgsConstructor
+@Getter
+public final class HBaseSelectOperationAdapter extends Operation {

Review Comment:
   Please add javadoc



##########
proxy/backend/type/hbase/src/main/java/org/apache/shardingsphere/proxy/backend/hbase/converter/HBaseRegionReloadConverter.java:
##########
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.shardingsphere.proxy.backend.hbase.converter;
+
+import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
+import org.apache.shardingsphere.infra.binder.statement.dal.FlushStatementContext;
+import org.apache.shardingsphere.proxy.backend.hbase.bean.HBaseOperation;
+import java.util.List;
+import java.util.stream.Collectors;
+
+public final class HBaseRegionReloadConverter implements HBaseDatabaseConverter {
+    
+    private final SQLStatementContext<?> sqlStatementContext;
+    
+    public HBaseRegionReloadConverter(final SQLStatementContext<?> sqlStatementContext) {
+        this.sqlStatementContext = sqlStatementContext;
+    }

Review Comment:
   Please use @Requiredargsconstructor



##########
proxy/backend/type/hbase/src/main/java/org/apache/shardingsphere/proxy/backend/hbase/converter/HBaseRegionReloadConverter.java:
##########
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.shardingsphere.proxy.backend.hbase.converter;
+
+import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
+import org.apache.shardingsphere.infra.binder.statement.dal.FlushStatementContext;
+import org.apache.shardingsphere.proxy.backend.hbase.bean.HBaseOperation;
+import java.util.List;
+import java.util.stream.Collectors;
+
+public final class HBaseRegionReloadConverter implements HBaseDatabaseConverter {

Review Comment:
   Please javadoc



##########
proxy/backend/type/hbase/src/main/java/org/apache/shardingsphere/proxy/backend/hbase/converter/HBaseDatabaseSelectConverter.java:
##########
@@ -0,0 +1,155 @@
+/*
+ * 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.shardingsphere.proxy.backend.hbase.converter;
+
+import lombok.AllArgsConstructor;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Query;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
+import org.apache.shardingsphere.infra.binder.statement.dml.SelectStatementContext;
+import org.apache.shardingsphere.proxy.backend.hbase.bean.HBaseOperation;
+import org.apache.shardingsphere.proxy.backend.hbase.util.HeterogeneousUtil;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.column.ColumnSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.BetweenExpression;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.BinaryOperationExpression;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.ExpressionSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.InExpression;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.simple.LiteralExpressionSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.pagination.PaginationValueSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.pagination.limit.NumberLiteralLimitValueSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.predicate.WhereSegment;
+import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dml.MySQLSelectStatement;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Optional;
+import java.util.stream.Collectors;
+
+@AllArgsConstructor
+@Slf4j
+public final class HBaseDatabaseSelectConverter extends HBaseDatabaseRowKeysConverterAdapter implements HBaseDatabaseConverter {

Review Comment:
   Please add javadoc



##########
proxy/backend/type/hbase/src/main/java/org/apache/shardingsphere/proxy/backend/hbase/converter/HBaseSelectOperationAdapter.java:
##########
@@ -0,0 +1,44 @@
+/*
+ * 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.shardingsphere.proxy.backend.hbase.converter;
+
+import lombok.AllArgsConstructor;
+import lombok.Getter;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Operation;
+import java.util.List;
+import java.util.Map;
+
+@AllArgsConstructor
+@Getter
+public final class HBaseSelectOperationAdapter extends Operation {
+    
+    private final String tableName;
+    
+    private final List<Get> gets;
+    
+    @Override
+    public Map<String, Object> getFingerprint() {
+        return null;
+    }
+    
+    @Override
+    public Map<String, Object> toMap(final int i) {
+        return null;

Review Comment:
   return null is forbidden, please return empty Map



##########
proxy/backend/type/hbase/src/main/java/org/apache/shardingsphere/proxy/backend/hbase/converter/HBaseDatabaseSelectConverter.java:
##########
@@ -0,0 +1,155 @@
+/*
+ * 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.shardingsphere.proxy.backend.hbase.converter;
+
+import lombok.AllArgsConstructor;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Query;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
+import org.apache.shardingsphere.infra.binder.statement.dml.SelectStatementContext;
+import org.apache.shardingsphere.proxy.backend.hbase.bean.HBaseOperation;
+import org.apache.shardingsphere.proxy.backend.hbase.util.HeterogeneousUtil;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.column.ColumnSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.BetweenExpression;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.BinaryOperationExpression;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.ExpressionSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.InExpression;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.simple.LiteralExpressionSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.pagination.PaginationValueSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.pagination.limit.NumberLiteralLimitValueSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.predicate.WhereSegment;
+import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dml.MySQLSelectStatement;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Optional;
+import java.util.stream.Collectors;
+
+@AllArgsConstructor

Review Comment:
   Please use @RequiredArgsConstructor



##########
proxy/backend/type/hbase/src/main/java/org/apache/shardingsphere/proxy/backend/hbase/converter/HBaseDatabaseSelectConverter.java:
##########
@@ -0,0 +1,155 @@
+/*
+ * 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.shardingsphere.proxy.backend.hbase.converter;
+
+import lombok.AllArgsConstructor;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Query;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
+import org.apache.shardingsphere.infra.binder.statement.dml.SelectStatementContext;
+import org.apache.shardingsphere.proxy.backend.hbase.bean.HBaseOperation;
+import org.apache.shardingsphere.proxy.backend.hbase.util.HeterogeneousUtil;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.column.ColumnSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.BetweenExpression;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.BinaryOperationExpression;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.ExpressionSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.InExpression;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.simple.LiteralExpressionSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.pagination.PaginationValueSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.pagination.limit.NumberLiteralLimitValueSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.predicate.WhereSegment;
+import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dml.MySQLSelectStatement;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Optional;
+import java.util.stream.Collectors;
+
+@AllArgsConstructor
+@Slf4j
+public final class HBaseDatabaseSelectConverter extends HBaseDatabaseRowKeysConverterAdapter implements HBaseDatabaseConverter {
+    
+    private final SQLStatementContext<?> sqlStatementContext;
+    
+    @Override
+    public HBaseOperation convert() {
+        SelectStatementContext context = (SelectStatementContext) sqlStatementContext;
+        if (isUseGetRequest(context)) {
+            return createGetRequest(context);
+        } else {
+            return createScanRequest(context);
+        }
+    }
+    
+    boolean isUseGetRequest(final SelectStatementContext context) {
+        return context.getWhereSegments().stream().findFirst().isPresent()
+                && (context.getWhereSegments().stream().findFirst().get().getExpr() instanceof BinaryOperationExpression
+                        || context.getWhereSegments().stream().findFirst().get().getExpr() instanceof InExpression);
+    }
+    
+    private List<String> getRowKeyFromWhereSegment(final ExpressionSegment expressionSegment) {
+        if (expressionSegment instanceof InExpression) {
+            InExpression expression = (InExpression) expressionSegment;
+            return getRowKeysFromWhereSegmentByIn(expression);
+        } else {
+            BinaryOperationExpression expression = (BinaryOperationExpression) expressionSegment;
+            return new ArrayList<>(Collections.singleton(String.valueOf(((LiteralExpressionSegment) expression.getRight()).getLiterals())));
+        }
+    }
+    
+    private HBaseOperation createGetRequest(final SelectStatementContext context) {
+        ExpressionSegment expression = context.getWhereSegments().stream().findFirst().get().getExpr();
+        // 处理 in
+        List<Get> gets = getRowKeyFromWhereSegment(expression).stream().map(this::getGetByRowKey).collect(Collectors.toList());
+        if (!HeterogeneousUtil.isUseShorthandProjection(context)) {
+            for (Get get : gets) {
+                decorateWithColumns(get, context);
+            }
+        }
+        if (expression instanceof InExpression) {
+            return new HBaseOperation(context.getTablesContext().getTableNames().iterator().next(),
+                    new HBaseSelectOperationAdapter(context.getTablesContext().getTableNames().iterator().next(), gets));
+        }
+        return new HBaseOperation(context.getTablesContext().getTableNames().iterator().next(), gets.get(0));
+    }
+    
+    private Get getGetByRowKey(final String rowKey) {
+        return new Get(Bytes.toBytes(rowKey));
+    }
+    
+    private void decorateWithColumns(final Query query, final SelectStatementContext statementContext) {
+        Collection<ColumnSegment> columns = statementContext.getColumnSegments();
+        
+        if (query instanceof Get) {
+            columns.forEach(each -> ((Get) query).addColumn(Bytes.toBytes("i"), Bytes.toBytes(String.valueOf(each))));
+        } else {
+            columns.forEach(each -> ((Scan) query).addColumn(Bytes.toBytes("i"), Bytes.toBytes(String.valueOf(each))));
+        }
+    }
+    
+    private void decoratedWithLimit(final Scan scan, final SelectStatementContext statementContext) {
+        MySQLSelectStatement selectStatement = (MySQLSelectStatement) statementContext.getSqlStatement();
+        if (selectStatement.getLimit().isPresent()) {
+            Optional<PaginationValueSegment> paginationValueSegment = selectStatement.getLimit().get().getRowCount();
+            paginationValueSegment.ifPresent(valueSegment -> scan.setLimit((int) ((NumberLiteralLimitValueSegment) valueSegment).getValue()));
+        }
+    }
+    
+    private HBaseOperation createScanRequest(final SelectStatementContext context) {
+        Scan scan = new Scan();
+        Optional<WhereSegment> whereSegment = context.getWhereSegments().stream().findFirst();
+        if (whereSegment.isPresent() && whereSegment.get().getExpr() instanceof BetweenExpression) {
+            decorateScanOperationWithBetweenExpression(scan, whereSegment.get().getExpr(), false);
+        }
+        if (!HeterogeneousUtil.isUseShorthandProjection(context)) {
+            decorateWithColumns(scan, context);
+        }
+        decoratedWithLimit(scan, context);
+        return new HBaseOperation(context.getTablesContext().getTableNames().iterator().next(), scan);
+    }
+    
+    private void decorateScanOperationWithBetweenExpression(final Scan scan, final ExpressionSegment expressionSegment, final boolean reversed) {
+        BetweenExpression betweenExpression = (BetweenExpression) expressionSegment;
+        LiteralExpressionSegment betweenExpr = (LiteralExpressionSegment) betweenExpression.getBetweenExpr();
+        LiteralExpressionSegment andExpr = (LiteralExpressionSegment) betweenExpression.getAndExpr();
+        String startRowKey = betweenExpr.getLiterals().toString();
+        String stopRowKey = andExpr.getLiterals().toString();
+        if (null != startRowKey && null != stopRowKey) {
+            if (reversed) {
+                scan.withStopRow(calBytes(startRowKey, 0), true);
+                // refer: <https://github.com/apache/hbase/blob/master/hbase-backup/src/main/java/org/apache/hadoop/hbase/backup/impl/BackupSystemTable.java#L1853>
+                scan.withStartRow(calBytes(stopRowKey + "~", 0), true);
+            } else {
+                scan.withStartRow(calBytes(startRowKey, 0), true);
+                scan.withStopRow(calBytes(stopRowKey + "~", 0), true);
+            }
+        }
+    }
+    
+    private byte[] calBytes(final String row, final int step) {
+        byte[] result = Bytes.toBytes(row);
+        byte[] rowBytes = Arrays.copyOf(result, result.length);
+        rowBytes[rowBytes.length - 1] = (byte) (rowBytes[rowBytes.length - 1] + step);
+        return rowBytes;

Review Comment:
   Return value should name as `result`



##########
proxy/backend/type/hbase/src/main/java/org/apache/shardingsphere/proxy/backend/hbase/converter/HBaseDatabaseSelectConverter.java:
##########
@@ -0,0 +1,155 @@
+/*
+ * 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.shardingsphere.proxy.backend.hbase.converter;
+
+import lombok.AllArgsConstructor;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Query;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
+import org.apache.shardingsphere.infra.binder.statement.dml.SelectStatementContext;
+import org.apache.shardingsphere.proxy.backend.hbase.bean.HBaseOperation;
+import org.apache.shardingsphere.proxy.backend.hbase.util.HeterogeneousUtil;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.column.ColumnSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.BetweenExpression;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.BinaryOperationExpression;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.ExpressionSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.InExpression;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.simple.LiteralExpressionSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.pagination.PaginationValueSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.pagination.limit.NumberLiteralLimitValueSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.predicate.WhereSegment;
+import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dml.MySQLSelectStatement;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Optional;
+import java.util.stream.Collectors;
+
+@AllArgsConstructor
+@Slf4j
+public final class HBaseDatabaseSelectConverter extends HBaseDatabaseRowKeysConverterAdapter implements HBaseDatabaseConverter {
+    
+    private final SQLStatementContext<?> sqlStatementContext;
+    
+    @Override
+    public HBaseOperation convert() {
+        SelectStatementContext context = (SelectStatementContext) sqlStatementContext;
+        if (isUseGetRequest(context)) {
+            return createGetRequest(context);
+        } else {
+            return createScanRequest(context);
+        }
+    }
+    
+    boolean isUseGetRequest(final SelectStatementContext context) {
+        return context.getWhereSegments().stream().findFirst().isPresent()
+                && (context.getWhereSegments().stream().findFirst().get().getExpr() instanceof BinaryOperationExpression
+                        || context.getWhereSegments().stream().findFirst().get().getExpr() instanceof InExpression);
+    }
+    
+    private List<String> getRowKeyFromWhereSegment(final ExpressionSegment expressionSegment) {
+        if (expressionSegment instanceof InExpression) {
+            InExpression expression = (InExpression) expressionSegment;
+            return getRowKeysFromWhereSegmentByIn(expression);
+        } else {
+            BinaryOperationExpression expression = (BinaryOperationExpression) expressionSegment;
+            return new ArrayList<>(Collections.singleton(String.valueOf(((LiteralExpressionSegment) expression.getRight()).getLiterals())));
+        }
+    }
+    
+    private HBaseOperation createGetRequest(final SelectStatementContext context) {
+        ExpressionSegment expression = context.getWhereSegments().stream().findFirst().get().getExpr();
+        // 处理 in
+        List<Get> gets = getRowKeyFromWhereSegment(expression).stream().map(this::getGetByRowKey).collect(Collectors.toList());
+        if (!HeterogeneousUtil.isUseShorthandProjection(context)) {
+            for (Get get : gets) {

Review Comment:
   Please name as `each` in loop



##########
proxy/backend/type/hbase/src/main/java/org/apache/shardingsphere/proxy/backend/hbase/converter/HBaseDatabaseUpdateConverter.java:
##########
@@ -0,0 +1,91 @@
+/*
+ * 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.shardingsphere.proxy.backend.hbase.converter;
+
+import com.google.common.base.Preconditions;
+import lombok.AllArgsConstructor;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
+import org.apache.shardingsphere.infra.binder.statement.dml.UpdateStatementContext;
+import org.apache.shardingsphere.proxy.backend.hbase.bean.HBaseOperation;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.assignment.AssignmentSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.BinaryOperationExpression;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.ExpressionSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.InExpression;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.simple.LiteralExpressionSegment;
+import java.util.Collection;
+import java.util.List;
+import java.util.stream.Collectors;
+
+@AllArgsConstructor
+public final class HBaseDatabaseUpdateConverter extends HBaseDatabaseRowKeysConverterAdapter implements HBaseDatabaseConverter {

Review Comment:
   Please add javadoc



##########
proxy/backend/type/hbase/src/main/java/org/apache/shardingsphere/proxy/backend/hbase/converter/HBaseDatabaseSelectConverter.java:
##########
@@ -0,0 +1,155 @@
+/*
+ * 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.shardingsphere.proxy.backend.hbase.converter;
+
+import lombok.AllArgsConstructor;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Query;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
+import org.apache.shardingsphere.infra.binder.statement.dml.SelectStatementContext;
+import org.apache.shardingsphere.proxy.backend.hbase.bean.HBaseOperation;
+import org.apache.shardingsphere.proxy.backend.hbase.util.HeterogeneousUtil;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.column.ColumnSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.BetweenExpression;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.BinaryOperationExpression;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.ExpressionSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.InExpression;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.simple.LiteralExpressionSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.pagination.PaginationValueSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.pagination.limit.NumberLiteralLimitValueSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.predicate.WhereSegment;
+import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dml.MySQLSelectStatement;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Optional;
+import java.util.stream.Collectors;
+
+@AllArgsConstructor
+@Slf4j

Review Comment:
   Is the annotation necessary?



##########
proxy/backend/type/hbase/src/main/java/org/apache/shardingsphere/proxy/backend/hbase/converter/HBaseDeleteOperationAdapter.java:
##########
@@ -0,0 +1,44 @@
+/*
+ * 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.shardingsphere.proxy.backend.hbase.converter;
+
+import lombok.AllArgsConstructor;
+import lombok.Getter;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Operation;
+import java.util.List;
+import java.util.Map;
+
+@AllArgsConstructor
+@Getter
+public final class HBaseDeleteOperationAdapter extends Operation {

Review Comment:
   Please add javadoc



##########
proxy/backend/type/hbase/src/main/java/org/apache/shardingsphere/proxy/backend/hbase/converter/HBaseDatabaseUpdateConverter.java:
##########
@@ -0,0 +1,91 @@
+/*
+ * 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.shardingsphere.proxy.backend.hbase.converter;
+
+import com.google.common.base.Preconditions;
+import lombok.AllArgsConstructor;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
+import org.apache.shardingsphere.infra.binder.statement.dml.UpdateStatementContext;
+import org.apache.shardingsphere.proxy.backend.hbase.bean.HBaseOperation;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.assignment.AssignmentSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.BinaryOperationExpression;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.ExpressionSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.InExpression;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.simple.LiteralExpressionSegment;
+import java.util.Collection;
+import java.util.List;
+import java.util.stream.Collectors;
+
+@AllArgsConstructor
+public final class HBaseDatabaseUpdateConverter extends HBaseDatabaseRowKeysConverterAdapter implements HBaseDatabaseConverter {
+    
+    private final SQLStatementContext<?> sqlStatementContext;
+    
+    @Override
+    public HBaseOperation convert() {
+        UpdateStatementContext context = (UpdateStatementContext) sqlStatementContext;
+        Preconditions.checkArgument(context.getWhereSegments().stream().findFirst().isPresent(), "Where segment is not present");
+        

Review Comment:
   Please remove blank line



##########
proxy/backend/type/hbase/src/main/java/org/apache/shardingsphere/proxy/backend/hbase/converter/HBaseDatabaseUpdateConverter.java:
##########
@@ -0,0 +1,91 @@
+/*
+ * 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.shardingsphere.proxy.backend.hbase.converter;
+
+import com.google.common.base.Preconditions;
+import lombok.AllArgsConstructor;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
+import org.apache.shardingsphere.infra.binder.statement.dml.UpdateStatementContext;
+import org.apache.shardingsphere.proxy.backend.hbase.bean.HBaseOperation;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.assignment.AssignmentSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.BinaryOperationExpression;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.ExpressionSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.InExpression;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.simple.LiteralExpressionSegment;
+import java.util.Collection;
+import java.util.List;
+import java.util.stream.Collectors;
+
+@AllArgsConstructor

Review Comment:
   Please use @RequiredArgsConstructor



##########
proxy/backend/type/hbase/src/main/java/org/apache/shardingsphere/proxy/backend/hbase/converter/HBaseDatabaseUpdateConverter.java:
##########
@@ -0,0 +1,91 @@
+/*
+ * 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.shardingsphere.proxy.backend.hbase.converter;
+
+import com.google.common.base.Preconditions;
+import lombok.AllArgsConstructor;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
+import org.apache.shardingsphere.infra.binder.statement.dml.UpdateStatementContext;
+import org.apache.shardingsphere.proxy.backend.hbase.bean.HBaseOperation;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.assignment.AssignmentSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.BinaryOperationExpression;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.ExpressionSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.InExpression;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.simple.LiteralExpressionSegment;
+import java.util.Collection;
+import java.util.List;
+import java.util.stream.Collectors;
+
+@AllArgsConstructor
+public final class HBaseDatabaseUpdateConverter extends HBaseDatabaseRowKeysConverterAdapter implements HBaseDatabaseConverter {
+    
+    private final SQLStatementContext<?> sqlStatementContext;
+    
+    @Override
+    public HBaseOperation convert() {
+        UpdateStatementContext context = (UpdateStatementContext) sqlStatementContext;
+        Preconditions.checkArgument(context.getWhereSegments().stream().findFirst().isPresent(), "Where segment is not present");
+        
+        if (context.getWhereSegments().stream().findFirst().get().getExpr() instanceof InExpression) {
+            return createHBasePutsOperation(context);
+        }
+        

Review Comment:
   Please remove blank line



##########
proxy/backend/type/hbase/src/main/java/org/apache/shardingsphere/proxy/backend/hbase/converter/HBaseDeleteOperationAdapter.java:
##########
@@ -0,0 +1,44 @@
+/*
+ * 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.shardingsphere.proxy.backend.hbase.converter;
+
+import lombok.AllArgsConstructor;
+import lombok.Getter;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Operation;
+import java.util.List;
+import java.util.Map;
+
+@AllArgsConstructor

Review Comment:
   Please use @RequiredArgsConstructor



##########
proxy/backend/type/hbase/src/main/java/org/apache/shardingsphere/proxy/backend/hbase/converter/HBaseDeleteOperationAdapter.java:
##########
@@ -0,0 +1,44 @@
+/*
+ * 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.shardingsphere.proxy.backend.hbase.converter;
+
+import lombok.AllArgsConstructor;
+import lombok.Getter;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Operation;
+import java.util.List;
+import java.util.Map;
+
+@AllArgsConstructor
+@Getter
+public final class HBaseDeleteOperationAdapter extends Operation {
+    
+    private final String tableName;
+    
+    private final List<Delete> deletes;
+    
+    @Override
+    public Map<String, Object> getFingerprint() {
+        return null;
+    }
+    
+    @Override
+    public Map<String, Object> toMap(final int i) {
+        return null;

Review Comment:
   return null is forbidden, please return empty Map



##########
proxy/backend/type/hbase/src/main/java/org/apache/shardingsphere/proxy/backend/hbase/converter/HBaseDeleteOperationAdapter.java:
##########
@@ -0,0 +1,44 @@
+/*
+ * 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.shardingsphere.proxy.backend.hbase.converter;
+
+import lombok.AllArgsConstructor;
+import lombok.Getter;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Operation;
+import java.util.List;
+import java.util.Map;
+
+@AllArgsConstructor
+@Getter
+public final class HBaseDeleteOperationAdapter extends Operation {
+    
+    private final String tableName;
+    
+    private final List<Delete> deletes;
+    
+    @Override
+    public Map<String, Object> getFingerprint() {
+        return null;

Review Comment:
   return null is forbidden, please return empty Map



##########
proxy/backend/type/hbase/src/main/java/org/apache/shardingsphere/proxy/backend/hbase/converter/HBaseInsertOperationAdapter.java:
##########
@@ -0,0 +1,42 @@
+/*
+ * 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.shardingsphere.proxy.backend.hbase.converter;
+
+import lombok.AllArgsConstructor;
+import lombok.Getter;
+import org.apache.hadoop.hbase.client.Operation;
+import org.apache.hadoop.hbase.client.Put;
+import java.util.List;
+import java.util.Map;
+
+@AllArgsConstructor

Review Comment:
   Please use @Requiredargsconstructor



##########
proxy/backend/type/hbase/src/main/java/org/apache/shardingsphere/proxy/backend/hbase/converter/HBaseInsertOperationAdapter.java:
##########
@@ -0,0 +1,42 @@
+/*
+ * 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.shardingsphere.proxy.backend.hbase.converter;
+
+import lombok.AllArgsConstructor;
+import lombok.Getter;
+import org.apache.hadoop.hbase.client.Operation;
+import org.apache.hadoop.hbase.client.Put;
+import java.util.List;
+import java.util.Map;
+
+@AllArgsConstructor
+@Getter
+public final class HBaseInsertOperationAdapter extends Operation {
+    
+    private final List<Put> puts;
+    
+    @Override
+    public Map<String, Object> getFingerprint() {
+        return null;

Review Comment:
   return null is forbidden, please return empty Map



##########
proxy/backend/type/hbase/src/main/java/org/apache/shardingsphere/proxy/backend/hbase/converter/HBaseInsertOperationAdapter.java:
##########
@@ -0,0 +1,42 @@
+/*
+ * 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.shardingsphere.proxy.backend.hbase.converter;
+
+import lombok.AllArgsConstructor;
+import lombok.Getter;
+import org.apache.hadoop.hbase.client.Operation;
+import org.apache.hadoop.hbase.client.Put;
+import java.util.List;
+import java.util.Map;
+
+@AllArgsConstructor
+@Getter
+public final class HBaseInsertOperationAdapter extends Operation {
+    
+    private final List<Put> puts;
+    
+    @Override
+    public Map<String, Object> getFingerprint() {
+        return null;
+    }
+    
+    @Override
+    public Map<String, Object> toMap(final int i) {
+        return null;

Review Comment:
   return null is forbidden, please return empty Map



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@shardingsphere.apache.org

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


[GitHub] [shardingsphere] terrymanu merged pull request #24536: Proxy For HBase add query and return result logic

Posted by "terrymanu (via GitHub)" <gi...@apache.org>.
terrymanu merged PR #24536:
URL: https://github.com/apache/shardingsphere/pull/24536


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@shardingsphere.apache.org

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


[GitHub] [shardingsphere] codecov-commenter commented on pull request #24536: Proxy For HBase add query and return result logic

Posted by "codecov-commenter (via GitHub)" <gi...@apache.org>.
codecov-commenter commented on PR #24536:
URL: https://github.com/apache/shardingsphere/pull/24536#issuecomment-1463230485

   ## [Codecov](https://codecov.io/gh/apache/shardingsphere/pull/24536?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#24536](https://codecov.io/gh/apache/shardingsphere/pull/24536?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (fa5f15a) into [master](https://codecov.io/gh/apache/shardingsphere/commit/ea06d6a506edc8186ab68e5dc6545b04ef510363?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (ea06d6a) will **decrease** coverage by `0.29%`.
   > The diff coverage is `0.00%`.
   
   > :exclamation: Current head fa5f15a differs from pull request most recent head 182c5bc. Consider uploading reports for the commit 182c5bc to get more accurate results
   
   ```diff
   @@             Coverage Diff              @@
   ##             master   #24536      +/-   ##
   ============================================
   - Coverage     50.20%   49.91%   -0.29%     
     Complexity     1599     1599              
   ============================================
     Files          3299     3318      +19     
     Lines         53981    54290     +309     
     Branches       9927     9979      +52     
   ============================================
     Hits          27100    27100              
   - Misses        24461    24770     +309     
     Partials       2420     2420              
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/shardingsphere/pull/24536?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...y/backend/hbase/config/YamlHBaseConfiguration.java](https://codecov.io/gh/apache/shardingsphere/pull/24536?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cHJveHkvYmFja2VuZC90eXBlL2hiYXNlL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9zaGFyZGluZ3NwaGVyZS9wcm94eS9iYWNrZW5kL2hiYXNlL2NvbmZpZy9ZYW1sSEJhc2VDb25maWd1cmF0aW9uLmphdmE=) | `0.00% <ø> (ø)` | |
   | [...here/proxy/backend/hbase/context/HBaseContext.java](https://codecov.io/gh/apache/shardingsphere/pull/24536?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cHJveHkvYmFja2VuZC90eXBlL2hiYXNlL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9zaGFyZGluZ3NwaGVyZS9wcm94eS9iYWNrZW5kL2hiYXNlL2NvbnRleHQvSEJhc2VDb250ZXh0LmphdmE=) | `0.00% <ø> (ø)` | |
   | [...ackend/hbase/context/HBaseRegionWarmUpContext.java](https://codecov.io/gh/apache/shardingsphere/pull/24536?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cHJveHkvYmFja2VuZC90eXBlL2hiYXNlL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9zaGFyZGluZ3NwaGVyZS9wcm94eS9iYWNrZW5kL2hiYXNlL2NvbnRleHQvSEJhc2VSZWdpb25XYXJtVXBDb250ZXh0LmphdmE=) | `0.00% <0.00%> (ø)` | |
   | [...hbase/converter/HBaseDatabaseConverterFactory.java](https://codecov.io/gh/apache/shardingsphere/pull/24536?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cHJveHkvYmFja2VuZC90eXBlL2hiYXNlL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9zaGFyZGluZ3NwaGVyZS9wcm94eS9iYWNrZW5kL2hiYXNlL2NvbnZlcnRlci9IQmFzZURhdGFiYXNlQ29udmVydGVyRmFjdG9yeS5qYXZh) | `0.00% <0.00%> (ø)` | |
   | [.../hbase/converter/HBaseDatabaseDeleteConverter.java](https://codecov.io/gh/apache/shardingsphere/pull/24536?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cHJveHkvYmFja2VuZC90eXBlL2hiYXNlL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9zaGFyZGluZ3NwaGVyZS9wcm94eS9iYWNrZW5kL2hiYXNlL2NvbnZlcnRlci9IQmFzZURhdGFiYXNlRGVsZXRlQ29udmVydGVyLmphdmE=) | `0.00% <0.00%> (ø)` | |
   | [.../hbase/converter/HBaseDatabaseInsertConverter.java](https://codecov.io/gh/apache/shardingsphere/pull/24536?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cHJveHkvYmFja2VuZC90eXBlL2hiYXNlL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9zaGFyZGluZ3NwaGVyZS9wcm94eS9iYWNrZW5kL2hiYXNlL2NvbnZlcnRlci9IQmFzZURhdGFiYXNlSW5zZXJ0Q29udmVydGVyLmphdmE=) | `0.00% <0.00%> (ø)` | |
   | [...onverter/HBaseDatabaseRowKeysConverterAdapter.java](https://codecov.io/gh/apache/shardingsphere/pull/24536?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cHJveHkvYmFja2VuZC90eXBlL2hiYXNlL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9zaGFyZGluZ3NwaGVyZS9wcm94eS9iYWNrZW5kL2hiYXNlL2NvbnZlcnRlci9IQmFzZURhdGFiYXNlUm93S2V5c0NvbnZlcnRlckFkYXB0ZXIuamF2YQ==) | `0.00% <0.00%> (ø)` | |
   | [.../hbase/converter/HBaseDatabaseSelectConverter.java](https://codecov.io/gh/apache/shardingsphere/pull/24536?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cHJveHkvYmFja2VuZC90eXBlL2hiYXNlL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9zaGFyZGluZ3NwaGVyZS9wcm94eS9iYWNrZW5kL2hiYXNlL2NvbnZlcnRlci9IQmFzZURhdGFiYXNlU2VsZWN0Q29udmVydGVyLmphdmE=) | `0.00% <0.00%> (ø)` | |
   | [.../hbase/converter/HBaseDatabaseUpdateConverter.java](https://codecov.io/gh/apache/shardingsphere/pull/24536?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cHJveHkvYmFja2VuZC90eXBlL2hiYXNlL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9zaGFyZGluZ3NwaGVyZS9wcm94eS9iYWNrZW5kL2hiYXNlL2NvbnZlcnRlci9IQmFzZURhdGFiYXNlVXBkYXRlQ29udmVydGVyLmphdmE=) | `0.00% <0.00%> (ø)` | |
   | [...d/hbase/converter/HBaseDeleteOperationAdapter.java](https://codecov.io/gh/apache/shardingsphere/pull/24536?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-cHJveHkvYmFja2VuZC90eXBlL2hiYXNlL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9zaGFyZGluZ3NwaGVyZS9wcm94eS9iYWNrZW5kL2hiYXNlL2NvbnZlcnRlci9IQmFzZURlbGV0ZU9wZXJhdGlvbkFkYXB0ZXIuamF2YQ==) | `0.00% <0.00%> (ø)` | |
   | ... and [15 more](https://codecov.io/gh/apache/shardingsphere/pull/24536?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   :mega: We’re building smart automated test selection to slash your CI/CD build times. [Learn more](https://about.codecov.io/iterative-testing/?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@shardingsphere.apache.org

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