You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by pa...@apache.org on 2022/06/27 11:14:16 UTC

[shardingsphere] branch master updated: Simplify QueryResponseCell (#18633)

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

panjuan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/shardingsphere.git


The following commit(s) were added to refs/heads/master by this push:
     new ff3e402d429 Simplify QueryResponseCell (#18633)
ff3e402d429 is described below

commit ff3e402d429186af9ae1b02e940ac3145c159805
Author: 吴伟杰 <wu...@apache.org>
AuthorDate: Mon Jun 27 19:14:00 2022 +0800

    Simplify QueryResponseCell (#18633)
    
    * Simplify QueryResponseCell
    
    Signed-off-by: 吴伟杰 <wu...@apache.org>
    
    * Complete javadoc in QueryResponseCell
    
    Signed-off-by: 吴伟杰 <wu...@apache.org>
---
 .../communication/DatabaseCommunicationEngine.java | 16 ++-------
 .../backend/response/data/QueryResponseCell.java   | 16 +++++----
 .../data/impl/BinaryQueryResponseCell.java         | 34 -------------------
 .../response/data/impl/TextQueryResponseCell.java  | 32 ------------------
 .../distsql/ral/QueryableRALBackendHandler.java    |  3 +-
 .../hint/executor/AbstractHintQueryExecutor.java   |  5 ++-
 .../response/data/QueryResponseRowTest.java        | 39 +++-------------------
 .../execute/MySQLComStmtExecuteExecutor.java       |  5 ++-
 .../execute/MySQLComStmtExecuteExecutorTest.java   |  4 +--
 .../command/query/extended/JDBCPortal.java         |  5 ++-
 .../command/query/extended/JDBCPortalTest.java     | 11 +++---
 .../ReactiveMySQLComStmtExecuteExecutor.java       |  5 ++-
 12 files changed, 32 insertions(+), 143 deletions(-)

diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/DatabaseCommunicationEngine.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/DatabaseCommunicationEngine.java
index 4d39260c60d..0d5af9ab102 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/DatabaseCommunicationEngine.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/DatabaseCommunicationEngine.java
@@ -30,7 +30,6 @@ import org.apache.shardingsphere.infra.eventbus.ShardingSphereEventBus;
 import org.apache.shardingsphere.infra.executor.sql.context.ExecutionContext;
 import org.apache.shardingsphere.infra.executor.sql.execute.result.query.QueryResult;
 import org.apache.shardingsphere.infra.executor.sql.execute.result.update.UpdateResult;
-import org.apache.shardingsphere.infra.executor.sql.prepare.driver.jdbc.JDBCDriverType;
 import org.apache.shardingsphere.infra.merge.MergeEngine;
 import org.apache.shardingsphere.infra.merge.result.MergedResult;
 import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
@@ -41,10 +40,8 @@ import org.apache.shardingsphere.mode.manager.lock.LockJudgeEngine;
 import org.apache.shardingsphere.mode.manager.lock.LockJudgeEngineBuilder;
 import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
 import org.apache.shardingsphere.proxy.backend.exception.DatabaseLockedException;
-import org.apache.shardingsphere.proxy.backend.response.data.QueryResponseCell;
 import org.apache.shardingsphere.proxy.backend.response.data.QueryResponseRow;
-import org.apache.shardingsphere.proxy.backend.response.data.impl.BinaryQueryResponseCell;
-import org.apache.shardingsphere.proxy.backend.response.data.impl.TextQueryResponseCell;
+import org.apache.shardingsphere.proxy.backend.response.data.QueryResponseCell;
 import org.apache.shardingsphere.proxy.backend.response.header.query.QueryHeader;
 import org.apache.shardingsphere.proxy.backend.response.header.query.QueryHeaderBuilderEngine;
 import org.apache.shardingsphere.proxy.backend.response.header.query.QueryResponseHeader;
@@ -193,22 +190,13 @@ public abstract class DatabaseCommunicationEngine<T> {
      */
     public QueryResponseRow getQueryResponseRow() throws SQLException {
         List<QueryResponseCell> cells = new ArrayList<>(queryHeaders.size());
-        boolean isBinary = isBinary();
         for (int columnIndex = 1; columnIndex <= queryHeaders.size(); columnIndex++) {
             Object data = mergedResult.getValue(columnIndex, Object.class);
-            if (isBinary) {
-                cells.add(new BinaryQueryResponseCell(queryHeaders.get(columnIndex - 1).getColumnType(), data));
-            } else {
-                cells.add(new TextQueryResponseCell(data));
-            }
+            cells.add(new QueryResponseCell(queryHeaders.get(columnIndex - 1).getColumnType(), data));
         }
         return new QueryResponseRow(cells);
     }
     
-    protected boolean isBinary() {
-        return !JDBCDriverType.STATEMENT.equals(driverType);
-    }
-    
     protected void checkLockedDatabase(final ExecutionContext executionContext) {
         if (lockJudgeEngine.isLocked(backendConnection.getConnectionSession().getDatabaseName(), executionContext.getSqlStatementContext())) {
             throw new DatabaseLockedException(backendConnection.getConnectionSession().getDatabaseName());
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/response/data/QueryResponseCell.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/response/data/QueryResponseCell.java
index 139ea95d514..c6bff8a2902 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/response/data/QueryResponseCell.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/response/data/QueryResponseCell.java
@@ -17,15 +17,17 @@
 
 package org.apache.shardingsphere.proxy.backend.response.data;
 
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+
 /**
  * Query response cell.
  */
-public interface QueryResponseCell {
+@RequiredArgsConstructor
+@Getter
+public final class QueryResponseCell {
+    
+    private final int jdbcType;
     
-    /**
-     * Get data.
-     * 
-     * @return data
-     */
-    Object getData();
+    private final Object data;
 }
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/response/data/impl/BinaryQueryResponseCell.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/response/data/impl/BinaryQueryResponseCell.java
deleted file mode 100644
index bfb4166fea9..00000000000
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/response/data/impl/BinaryQueryResponseCell.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.shardingsphere.proxy.backend.response.data.impl;
-
-import lombok.Getter;
-import lombok.RequiredArgsConstructor;
-import org.apache.shardingsphere.proxy.backend.response.data.QueryResponseCell;
-
-/**
- * Binary query response cell.
- */
-@RequiredArgsConstructor
-@Getter
-public final class BinaryQueryResponseCell implements QueryResponseCell {
-    
-    private final int jdbcType;
-    
-    private final Object data;
-}
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/response/data/impl/TextQueryResponseCell.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/response/data/impl/TextQueryResponseCell.java
deleted file mode 100644
index e48e111b4dc..00000000000
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/response/data/impl/TextQueryResponseCell.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.shardingsphere.proxy.backend.response.data.impl;
-
-import lombok.Getter;
-import lombok.RequiredArgsConstructor;
-import org.apache.shardingsphere.proxy.backend.response.data.QueryResponseCell;
-
-/**
- * Text query response cell.
- */
-@RequiredArgsConstructor
-@Getter
-public final class TextQueryResponseCell implements QueryResponseCell {
-    
-    private final Object data;
-}
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/QueryableRALBackendHandler.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/QueryableRALBackendHandler.java
index f613b54a7eb..8502fb71312 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/QueryableRALBackendHandler.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/QueryableRALBackendHandler.java
@@ -25,7 +25,6 @@ import org.apache.shardingsphere.mode.manager.ContextManager;
 import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
 import org.apache.shardingsphere.proxy.backend.response.data.QueryResponseCell;
 import org.apache.shardingsphere.proxy.backend.response.data.QueryResponseRow;
-import org.apache.shardingsphere.proxy.backend.response.data.impl.TextQueryResponseCell;
 import org.apache.shardingsphere.proxy.backend.response.header.ResponseHeader;
 import org.apache.shardingsphere.proxy.backend.response.header.query.QueryHeader;
 import org.apache.shardingsphere.proxy.backend.response.header.query.QueryResponseHeader;
@@ -72,7 +71,7 @@ public abstract class QueryableRALBackendHandler<E extends RALStatement> extends
     public final Collection<Object> getRowData() throws SQLException {
         List<QueryResponseCell> cells = new ArrayList<>(queryHeaders.size());
         for (int i = 0; i < queryHeaders.size(); i++) {
-            cells.add(new TextQueryResponseCell(mergedResult.getValue(i + 1, Object.class)));
+            cells.add(new QueryResponseCell(queryHeaders.get(i).getColumnType(), mergedResult.getValue(i + 1, Object.class)));
         }
         return new QueryResponseRow(cells).getData();
     }
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/hint/executor/AbstractHintQueryExecutor.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/hint/executor/AbstractHintQueryExecutor.java
index 5b9b8018b1d..b48e36ae413 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/hint/executor/AbstractHintQueryExecutor.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/hint/executor/AbstractHintQueryExecutor.java
@@ -21,10 +21,9 @@ import org.apache.shardingsphere.distsql.parser.statement.ral.common.HintDistSQL
 import org.apache.shardingsphere.infra.merge.result.MergedResult;
 import org.apache.shardingsphere.proxy.backend.response.data.QueryResponseCell;
 import org.apache.shardingsphere.proxy.backend.response.data.QueryResponseRow;
-import org.apache.shardingsphere.proxy.backend.response.data.impl.TextQueryResponseCell;
 import org.apache.shardingsphere.proxy.backend.response.header.ResponseHeader;
-import org.apache.shardingsphere.proxy.backend.response.header.query.QueryResponseHeader;
 import org.apache.shardingsphere.proxy.backend.response.header.query.QueryHeader;
+import org.apache.shardingsphere.proxy.backend.response.header.query.QueryResponseHeader;
 import org.apache.shardingsphere.proxy.backend.text.distsql.ral.common.hint.HintStatementExecutor;
 
 import java.sql.SQLException;
@@ -60,7 +59,7 @@ public abstract class AbstractHintQueryExecutor<T extends HintDistSQLStatement>
     public final QueryResponseRow getQueryResponseRow() throws SQLException {
         List<QueryResponseCell> cells = new ArrayList<>(queryHeaders.size());
         for (int i = 0; i < queryHeaders.size(); i++) {
-            cells.add(new TextQueryResponseCell(mergedResult.getValue(i + 1, Object.class)));
+            cells.add(new QueryResponseCell(queryHeaders.get(i).getColumnType(), mergedResult.getValue(i + 1, Object.class)));
         }
         return new QueryResponseRow(cells);
     }
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/response/data/QueryResponseRowTest.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/response/data/QueryResponseRowTest.java
index e7842514e4e..c499c2acf28 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/response/data/QueryResponseRowTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/test/java/org/apache/shardingsphere/proxy/backend/response/data/QueryResponseRowTest.java
@@ -17,19 +17,15 @@
 
 package org.apache.shardingsphere.proxy.backend.response.data;
 
-import org.apache.shardingsphere.proxy.backend.response.data.impl.BinaryQueryResponseCell;
-import org.apache.shardingsphere.proxy.backend.response.data.impl.TextQueryResponseCell;
 import org.junit.Test;
 
+import java.sql.Types;
 import java.util.Arrays;
 import java.util.Collections;
 
-import static org.hamcrest.CoreMatchers.instanceOf;
 import static org.hamcrest.CoreMatchers.is;
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
 
 public final class QueryResponseRowTest {
     
@@ -40,35 +36,10 @@ public final class QueryResponseRowTest {
     }
     
     @Test
-    public void assertGetDataWhenQueryResponseCellsInstanceOfBinaryQueryResponseCell() {
-        BinaryQueryResponseCell binaryQueryResponseCell1 = mock(BinaryQueryResponseCell.class);
-        when(binaryQueryResponseCell1.getData()).thenReturn(1);
-        BinaryQueryResponseCell binaryQueryResponseCell2 = mock(BinaryQueryResponseCell.class);
-        when(binaryQueryResponseCell2.getData()).thenReturn("2");
-        QueryResponseRow queryResponseRow = new QueryResponseRow(Arrays.asList(binaryQueryResponseCell1, binaryQueryResponseCell2));
-        assertThat(queryResponseRow.getData().size(), is(2));
-        assertTrue(queryResponseRow.getData().containsAll(Arrays.asList(1, "2")));
-    }
-    
-    @Test
-    public void assertGetDataWhenQueryResponseCellsInstanceOfTextQueryResponseCell() {
-        TextQueryResponseCell textQueryResponseCell1 = mock(TextQueryResponseCell.class);
-        when(textQueryResponseCell1.getData()).thenReturn("column_1");
-        TextQueryResponseCell textQueryResponseCell2 = mock(TextQueryResponseCell.class);
-        when(textQueryResponseCell2.getData()).thenReturn("column_2");
-        QueryResponseRow queryResponseRow = new QueryResponseRow(Arrays.asList(textQueryResponseCell1, textQueryResponseCell2));
-        assertThat(queryResponseRow.getData().size(), is(2));
-        assertThat(queryResponseRow.getData().iterator().next(), instanceOf(String.class));
-        assertTrue(queryResponseRow.getData().containsAll(Arrays.asList("column_1", "column_2")));
-    }
-    
-    @Test
-    public void assertGetDataWhenQueryResponseCellsInstanceOfAllTypeQueryResponseCell() {
-        BinaryQueryResponseCell binaryQueryResponseCell = mock(BinaryQueryResponseCell.class);
-        when(binaryQueryResponseCell.getData()).thenReturn(1);
-        TextQueryResponseCell textQueryResponseCell = mock(TextQueryResponseCell.class);
-        when(textQueryResponseCell.getData()).thenReturn("column");
-        QueryResponseRow queryResponseRow = new QueryResponseRow(Arrays.asList(binaryQueryResponseCell, textQueryResponseCell));
+    public void assertGetDataWhenQueryResponseCellsPresent() {
+        QueryResponseCell queryResponseCell1 = new QueryResponseCell(Types.INTEGER, 1);
+        QueryResponseCell queryResponseCell2 = new QueryResponseCell(Types.VARCHAR, "column");
+        QueryResponseRow queryResponseRow = new QueryResponseRow(Arrays.asList(queryResponseCell1, queryResponseCell2));
         assertThat(queryResponseRow.getData().size(), is(2));
         assertTrue(queryResponseRow.getData().containsAll(Arrays.asList(1, "column")));
     }
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/main/java/org/apache/shardingsphere/proxy/frontend/mysql/command/query/binary/execute/MySQLComStmtExecuteExecutor.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/main/java/org/apache/shardingsphere/proxy/frontend/mysql/command/query/binary/execute/MySQLComStmtExecuteExecutor.java
index 5aaf00ef658..f8a9844c396 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/main/java/org/apache/shardingsphere/proxy/frontend/mysql/command/query/binary/execute/MySQLComStmtExecuteExecutor.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/main/java/org/apache/shardingsphere/proxy/frontend/mysql/command/query/binary/execute/MySQLComStmtExecuteExecutor.java
@@ -39,9 +39,8 @@ import org.apache.shardingsphere.proxy.backend.communication.DatabaseCommunicati
 import org.apache.shardingsphere.proxy.backend.communication.SQLStatementDatabaseHolder;
 import org.apache.shardingsphere.proxy.backend.communication.jdbc.JDBCDatabaseCommunicationEngine;
 import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
-import org.apache.shardingsphere.proxy.backend.response.data.QueryResponseCell;
 import org.apache.shardingsphere.proxy.backend.response.data.QueryResponseRow;
-import org.apache.shardingsphere.proxy.backend.response.data.impl.BinaryQueryResponseCell;
+import org.apache.shardingsphere.proxy.backend.response.data.QueryResponseCell;
 import org.apache.shardingsphere.proxy.backend.response.header.ResponseHeader;
 import org.apache.shardingsphere.proxy.backend.response.header.query.QueryResponseHeader;
 import org.apache.shardingsphere.proxy.backend.response.header.update.UpdateResponseHeader;
@@ -157,7 +156,7 @@ public final class MySQLComStmtExecuteExecutor implements QueryCommandExecutor {
     private BinaryRow createBinaryRow(final QueryResponseRow queryResponseRow) {
         List<BinaryCell> result = new ArrayList<>(queryResponseRow.getCells().size());
         for (QueryResponseCell each : queryResponseRow.getCells()) {
-            result.add(new BinaryCell(MySQLBinaryColumnType.valueOfJDBCType(((BinaryQueryResponseCell) each).getJdbcType()), each.getData()));
+            result.add(new BinaryCell(MySQLBinaryColumnType.valueOfJDBCType(each.getJdbcType()), each.getData()));
         }
         return new BinaryRow(result);
     }
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/test/java/org/apache/shardingsphere/proxy/frontend/mysql/command/query/binary/execute/MySQLComStmtExecuteExecutorTest.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/test/java/org/apache/shardingsphere/proxy/frontend/mysql/command/query/binary/execute/MySQLComStmtExecuteExecutorTest.java
index c60ea2db6b1..1aca60bc315 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/test/java/org/apache/shardingsphere/proxy/frontend/mysql/command/query/binary/execute/MySQLComStmtExecuteExecutorTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-mysql/src/test/java/org/apache/shardingsphere/proxy/frontend/mysql/command/query/binary/execute/MySQLComStmtExecuteExecutorTest.java
@@ -46,7 +46,7 @@ import org.apache.shardingsphere.proxy.backend.communication.jdbc.JDBCDatabaseCo
 import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.JDBCBackendConnection;
 import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
 import org.apache.shardingsphere.proxy.backend.response.data.QueryResponseRow;
-import org.apache.shardingsphere.proxy.backend.response.data.impl.BinaryQueryResponseCell;
+import org.apache.shardingsphere.proxy.backend.response.data.QueryResponseCell;
 import org.apache.shardingsphere.proxy.backend.response.header.query.QueryHeader;
 import org.apache.shardingsphere.proxy.backend.response.header.query.QueryResponseHeader;
 import org.apache.shardingsphere.proxy.backend.response.header.update.UpdateResponseHeader;
@@ -166,7 +166,7 @@ public final class MySQLComStmtExecuteExecutorTest extends ProxyContextRestorer
         MySQLComStmtExecuteExecutor mysqlComStmtExecuteExecutor = new MySQLComStmtExecuteExecutor(packet, connectionSession);
         when(databaseCommunicationEngine.execute()).thenReturn(new QueryResponseHeader(Collections.singletonList(mock(QueryHeader.class))));
         when(databaseCommunicationEngine.next()).thenReturn(true, false);
-        when(databaseCommunicationEngine.getQueryResponseRow()).thenReturn(new QueryResponseRow(Collections.singletonList(new BinaryQueryResponseCell(Types.INTEGER, 1))));
+        when(databaseCommunicationEngine.getQueryResponseRow()).thenReturn(new QueryResponseRow(Collections.singletonList(new QueryResponseCell(Types.INTEGER, 1))));
         Iterator<DatabasePacket<?>> actual;
         try (MockedStatic<DatabaseCommunicationEngineFactory> mockedStatic = mockStatic(DatabaseCommunicationEngineFactory.class, RETURNS_DEEP_STUBS)) {
             mockedStatic.when(() -> DatabaseCommunicationEngineFactory.getInstance().newBinaryProtocolInstance(any(SQLStatementContext.class), anyString(), anyList(), eq(backendConnection)))
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/main/java/org/apache/shardingsphere/proxy/frontend/postgresql/command/query/extended/JDBCPortal.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/main/java/org/apache/shardingsphere/proxy/frontend/postgresql/command/query/extended/JDBCPortal.java
index 0b2ea7952c9..6aa508f1a5b 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/main/java/org/apache/shardingsphere/proxy/frontend/postgresql/command/query/extended/JDBCPortal.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/main/java/org/apache/shardingsphere/proxy/frontend/postgresql/command/query/extended/JDBCPortal.java
@@ -46,9 +46,8 @@ import org.apache.shardingsphere.proxy.backend.communication.DatabaseCommunicati
 import org.apache.shardingsphere.proxy.backend.communication.jdbc.JDBCDatabaseCommunicationEngine;
 import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.JDBCBackendConnection;
 import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
-import org.apache.shardingsphere.proxy.backend.response.data.QueryResponseCell;
 import org.apache.shardingsphere.proxy.backend.response.data.QueryResponseRow;
-import org.apache.shardingsphere.proxy.backend.response.data.impl.BinaryQueryResponseCell;
+import org.apache.shardingsphere.proxy.backend.response.data.QueryResponseCell;
 import org.apache.shardingsphere.proxy.backend.response.header.ResponseHeader;
 import org.apache.shardingsphere.proxy.backend.response.header.query.QueryHeader;
 import org.apache.shardingsphere.proxy.backend.response.header.query.QueryResponseHeader;
@@ -214,7 +213,7 @@ public final class JDBCPortal implements Portal<Void> {
     }
     
     private BinaryCell createBinaryCell(final QueryResponseCell cell) {
-        return new BinaryCell(PostgreSQLColumnType.valueOfJDBCType(((BinaryQueryResponseCell) cell).getJdbcType()), cell.getData());
+        return new BinaryCell(PostgreSQLColumnType.valueOfJDBCType(cell.getJdbcType()), cell.getData());
     }
     
     private PostgreSQLIdentifierPacket createExecutionCompletedPacket(final boolean isSuspended, final int fetchedRows) {
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/test/java/org/apache/shardingsphere/proxy/frontend/postgresql/command/query/extended/JDBCPortalTest.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/test/java/org/apache/shardingsphere/proxy/frontend/postgresql/command/query/extended/JDBCPortalTest.java
index f4608d00b3f..a735569a6db 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/test/java/org/apache/shardingsphere/proxy/frontend/postgresql/command/query/extended/JDBCPortalTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/test/java/org/apache/shardingsphere/proxy/frontend/postgresql/command/query/extended/JDBCPortalTest.java
@@ -31,9 +31,8 @@ import org.apache.shardingsphere.mode.manager.ContextManager;
 import org.apache.shardingsphere.proxy.backend.communication.jdbc.JDBCDatabaseCommunicationEngine;
 import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.JDBCBackendConnection;
 import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
+import org.apache.shardingsphere.proxy.backend.response.data.QueryResponseCell;
 import org.apache.shardingsphere.proxy.backend.response.data.QueryResponseRow;
-import org.apache.shardingsphere.proxy.backend.response.data.impl.BinaryQueryResponseCell;
-import org.apache.shardingsphere.proxy.backend.response.data.impl.TextQueryResponseCell;
 import org.apache.shardingsphere.proxy.backend.response.header.query.QueryHeader;
 import org.apache.shardingsphere.proxy.backend.response.header.query.QueryResponseHeader;
 import org.apache.shardingsphere.proxy.backend.response.header.update.UpdateResponseHeader;
@@ -116,8 +115,8 @@ public final class JDBCPortalTest extends ProxyContextRestorer {
         when(responseHeader.getQueryHeaders()).thenReturn(Collections.singletonList(queryHeader));
         when(databaseCommunicationEngine.execute()).thenReturn(responseHeader);
         when(databaseCommunicationEngine.next()).thenReturn(true, true, false);
-        when(databaseCommunicationEngine.getQueryResponseRow())
-                .thenReturn(new QueryResponseRow(Collections.singletonList(new TextQueryResponseCell(0))), new QueryResponseRow(Collections.singletonList(new TextQueryResponseCell(1))));
+        when(databaseCommunicationEngine.getQueryResponseRow()).thenReturn(new QueryResponseRow(Collections.singletonList(new QueryResponseCell(Types.INTEGER, 0))),
+                new QueryResponseRow(Collections.singletonList(new QueryResponseCell(Types.INTEGER, 1))));
         portal.bind();
         assertThat(portal.describe(), instanceOf(PostgreSQLRowDescriptionPacket.class));
         setField(portal, "sqlStatement", mock(SelectStatement.class));
@@ -139,8 +138,8 @@ public final class JDBCPortalTest extends ProxyContextRestorer {
         when(databaseCommunicationEngine.execute()).thenReturn(responseHeader);
         when(databaseCommunicationEngine.next()).thenReturn(true, true);
         when(databaseCommunicationEngine.getQueryResponseRow()).thenReturn(
-                new QueryResponseRow(Collections.singletonList(new BinaryQueryResponseCell(Types.INTEGER, 0))),
-                new QueryResponseRow(Collections.singletonList(new BinaryQueryResponseCell(Types.INTEGER, 1))));
+                new QueryResponseRow(Collections.singletonList(new QueryResponseCell(Types.INTEGER, 0))),
+                new QueryResponseRow(Collections.singletonList(new QueryResponseCell(Types.INTEGER, 1))));
         setField(portal, "resultFormats", Collections.singletonList(PostgreSQLValueFormat.BINARY));
         portal.bind();
         assertThat(portal.describe(), instanceOf(PostgreSQLRowDescriptionPacket.class));
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-reactive-mysql/src/main/java/org/apache/shardingsphere/proxy/frontend/reactive/mysql/command/query/binary/execute/ReactiveMySQLComStmtExecuteExecutor.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-reactive-mysql/src/main/java/org/apache/shardingsphere/proxy/frontend/reactive/mysql/command/query/binary/execute/ReactiveMySQLComStmtExecuteExecutor.java
index 48880226557..d80e114cb0b 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-reactive-mysql/src/main/java/org/apache/shardingsphere/proxy/frontend/reactive/mysql/command/query/binary/execute/ReactiveMySQLComStmtExecuteExecutor.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-reactive-mysql/src/main/java/org/apache/shardingsphere/proxy/frontend/reactive/mysql/command/query/binary/execute/ReactiveMySQLComStmtExecuteExecutor.java
@@ -42,9 +42,8 @@ import org.apache.shardingsphere.proxy.backend.communication.DatabaseCommunicati
 import org.apache.shardingsphere.proxy.backend.communication.SQLStatementDatabaseHolder;
 import org.apache.shardingsphere.proxy.backend.communication.vertx.VertxDatabaseCommunicationEngine;
 import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
-import org.apache.shardingsphere.proxy.backend.response.data.QueryResponseCell;
 import org.apache.shardingsphere.proxy.backend.response.data.QueryResponseRow;
-import org.apache.shardingsphere.proxy.backend.response.data.impl.BinaryQueryResponseCell;
+import org.apache.shardingsphere.proxy.backend.response.data.QueryResponseCell;
 import org.apache.shardingsphere.proxy.backend.response.header.query.QueryResponseHeader;
 import org.apache.shardingsphere.proxy.backend.response.header.update.UpdateResponseHeader;
 import org.apache.shardingsphere.proxy.backend.session.ConnectionSession;
@@ -169,7 +168,7 @@ public final class ReactiveMySQLComStmtExecuteExecutor implements ReactiveComman
     private BinaryRow createBinaryRow(final QueryResponseRow queryResponseRow) {
         List<BinaryCell> result = new ArrayList<>(queryResponseRow.getCells().size());
         for (QueryResponseCell each : queryResponseRow.getCells()) {
-            result.add(new BinaryCell(MySQLBinaryColumnType.valueOfJDBCType(((BinaryQueryResponseCell) each).getJdbcType()), each.getData()));
+            result.add(new BinaryCell(MySQLBinaryColumnType.valueOfJDBCType(each.getJdbcType()), each.getData()));
         }
         return new BinaryRow(result);
     }