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

[shardingsphere] branch master updated: Remove QueryResult.getResultSetMetaData() (#8173)

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

zhangliang 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 d5ee48d  Remove QueryResult.getResultSetMetaData() (#8173)
d5ee48d is described below

commit d5ee48d9c05f60eb2d42885d83ecfb848943012a
Author: Liang Zhang <te...@163.com>
AuthorDate: Mon Nov 16 17:07:47 2020 +0800

    Remove QueryResult.getResultSetMetaData() (#8173)
    
    * Refactor ShowTablesBackendHandler
    
    * Refactor PostgreSQLComBindExecutor & PostgreSQLComQueryExecutor
    
    * Remove QueryResult.getResultSetMetaData()
---
 .../command/query/PostgreSQLColumnDescription.java   | 20 ++++----------------
 .../query/PostgreSQLRowDescriptionPacket.java        |  4 ++--
 .../query/PostgreSQLColumnDescriptionTest.java       | 10 ++--------
 .../infra/executor/sql/QueryResult.java              | 17 +++++++++--------
 .../jdbc/queryresult/MemoryQueryResult.java          | 12 ++++++------
 .../jdbc/queryresult/StreamQueryResult.java          | 14 +++++++-------
 .../backend/text/admin/ShowTablesBackendHandler.java |  6 +++---
 .../query/binary/bind/PostgreSQLComBindExecutor.java | 13 ++++++-------
 .../query/text/PostgreSQLComQueryExecutor.java       | 13 ++++++-------
 9 files changed, 45 insertions(+), 64 deletions(-)

diff --git a/shardingsphere-db-protocol/shardingsphere-db-protocol-postgresql/src/main/java/org/apache/shardingsphere/db/protocol/postgresql/packet/command/query/PostgreSQLColumnDescription.java b/shardingsphere-db-protocol/shardingsphere-db-protocol-postgresql/src/main/java/org/apache/shardingsphere/db/protocol/postgresql/packet/command/query/PostgreSQLColumnDescription.java
index 31514ae..00f9cf5 100644
--- a/shardingsphere-db-protocol/shardingsphere-db-protocol-postgresql/src/main/java/org/apache/shardingsphere/db/protocol/postgresql/packet/command/query/PostgreSQLColumnDescription.java
+++ b/shardingsphere-db-protocol/shardingsphere-db-protocol-postgresql/src/main/java/org/apache/shardingsphere/db/protocol/postgresql/packet/command/query/PostgreSQLColumnDescription.java
@@ -22,8 +22,6 @@ import lombok.extern.slf4j.Slf4j;
 import org.apache.shardingsphere.db.protocol.postgresql.constant.PostgreSQLArrayColumnType;
 import org.apache.shardingsphere.db.protocol.postgresql.constant.PostgreSQLColumnType;
 
-import java.sql.ResultSetMetaData;
-import java.sql.SQLException;
 import java.sql.Types;
 
 /**
@@ -39,28 +37,18 @@ public final class PostgreSQLColumnDescription {
     
     private final int columnIndex;
     
-    private final int typeOID;
-    
     private final int columnLength;
+
+    private final int typeOID;
     
     private final int typeModifier = -1;
     
     private final int dataFormat = 0;
     
-    public PostgreSQLColumnDescription(final String columnName, final int columnIndex, final int columnType, final int columnLength, final ResultSetMetaData resultSetMetaData) {
+    public PostgreSQLColumnDescription(final String columnName, final int columnIndex, final int columnType, final int columnLength, final String columnTypeName) {
         this.columnName = columnName;
         this.columnIndex = columnIndex;
-        if (Types.ARRAY == columnType && null != resultSetMetaData) {
-            String columnTypeName = null;
-            try {
-                columnTypeName = resultSetMetaData.getColumnTypeName(columnIndex);
-            } catch (final SQLException ex) {
-                log.error("getColumnTypeName failed, columnName={}, columnIndex={}", columnName, columnIndex, ex);
-            }
-            typeOID = PostgreSQLArrayColumnType.getTypeOidByColumnTypeName(columnTypeName);
-        } else {
-            typeOID = PostgreSQLColumnType.valueOfJDBCType(columnType).getValue();
-        }
         this.columnLength = columnLength;
+        typeOID = Types.ARRAY == columnType ? PostgreSQLArrayColumnType.getTypeOidByColumnTypeName(columnTypeName) : PostgreSQLColumnType.valueOfJDBCType(columnType).getValue();
     }
 }
diff --git a/shardingsphere-db-protocol/shardingsphere-db-protocol-postgresql/src/main/java/org/apache/shardingsphere/db/protocol/postgresql/packet/command/query/PostgreSQLRowDescriptionPacket.java b/shardingsphere-db-protocol/shardingsphere-db-protocol-postgresql/src/main/java/org/apache/shardingsphere/db/protocol/postgresql/packet/command/query/PostgreSQLRowDescriptionPacket.java
index 8d4d3dc..bf50ef8 100644
--- a/shardingsphere-db-protocol/shardingsphere-db-protocol-postgresql/src/main/java/org/apache/shardingsphere/db/protocol/postgresql/packet/command/query/PostgreSQLRowDescriptionPacket.java
+++ b/shardingsphere-db-protocol/shardingsphere-db-protocol-postgresql/src/main/java/org/apache/shardingsphere/db/protocol/postgresql/packet/command/query/PostgreSQLRowDescriptionPacket.java
@@ -23,7 +23,7 @@ import org.apache.shardingsphere.db.protocol.postgresql.packet.PostgreSQLPacket;
 import org.apache.shardingsphere.db.protocol.postgresql.packet.command.PostgreSQLCommandPacketType;
 import org.apache.shardingsphere.db.protocol.postgresql.payload.PostgreSQLPacketPayload;
 
-import java.util.List;
+import java.util.Collection;
 
 /**
  * Row description packet for PostgreSQL.
@@ -37,7 +37,7 @@ public final class PostgreSQLRowDescriptionPacket implements PostgreSQLPacket {
     @Getter
     private final int fieldCount;
     
-    private final List<PostgreSQLColumnDescription> columnDescriptions;
+    private final Collection<PostgreSQLColumnDescription> columnDescriptions;
     
     @Override
     public void write(final PostgreSQLPacketPayload payload) {
diff --git a/shardingsphere-db-protocol/shardingsphere-db-protocol-postgresql/src/test/java/org/apache/shardingsphere/db/protocol/postgresql/packet/command/query/PostgreSQLColumnDescriptionTest.java b/shardingsphere-db-protocol/shardingsphere-db-protocol-postgresql/src/test/java/org/apache/shardingsphere/db/protocol/postgresql/packet/command/query/PostgreSQLColumnDescriptionTest.java
index 99d7951..ec1d734 100644
--- a/shardingsphere-db-protocol/shardingsphere-db-protocol-postgresql/src/test/java/org/apache/shardingsphere/db/protocol/postgresql/packet/command/query/PostgreSQLColumnDescriptionTest.java
+++ b/shardingsphere-db-protocol/shardingsphere-db-protocol-postgresql/src/test/java/org/apache/shardingsphere/db/protocol/postgresql/packet/command/query/PostgreSQLColumnDescriptionTest.java
@@ -19,14 +19,10 @@ package org.apache.shardingsphere.db.protocol.postgresql.packet.command.query;
 
 import org.junit.Test;
 
-import java.sql.ResultSetMetaData;
-import java.sql.SQLException;
 import java.sql.Types;
 
 import static org.hamcrest.CoreMatchers.is;
 import static org.junit.Assert.assertThat;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
 
 public final class PostgreSQLColumnDescriptionTest {
     
@@ -43,10 +39,8 @@ public final class PostgreSQLColumnDescriptionTest {
     }
     
     @Test
-    public void assertIntegerArrayTypeOid() throws SQLException {
-        ResultSetMetaData resultSetMetaData = mock(ResultSetMetaData.class);
-        when(resultSetMetaData.getColumnTypeName(2)).thenReturn("_int4");
-        PostgreSQLColumnDescription description = new PostgreSQLColumnDescription("ages", 2, Types.ARRAY, 12, resultSetMetaData);
+    public void assertIntegerArrayTypeOid() {
+        PostgreSQLColumnDescription description = new PostgreSQLColumnDescription("ages", 2, Types.ARRAY, 12, "_int4");
         assertThat(description.getTypeOID(), is(1007));
     }
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/QueryResult.java b/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/QueryResult.java
index ca8cc56..68bd011 100644
--- a/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/QueryResult.java
+++ b/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/QueryResult.java
@@ -18,7 +18,6 @@
 package org.apache.shardingsphere.infra.executor.sql;
 
 import java.io.InputStream;
-import java.sql.ResultSetMetaData;
 import java.sql.SQLException;
 import java.util.Calendar;
 
@@ -28,13 +27,6 @@ import java.util.Calendar;
 public interface QueryResult {
     
     /**
-     * Get JDBC ResultSetMetaData.
-     *
-     * @return JDBC ResultSetMetaData
-     */
-    ResultSetMetaData getResultSetMetaData();
-    
-    /**
      * iterate next data.
      *
      * @return has next data
@@ -106,4 +98,13 @@ public interface QueryResult {
      * @throws SQLException SQL Exception
      */
     String getColumnLabel(int columnIndex) throws SQLException;
+    
+    /**
+     * Get column type name.
+     * 
+     * @param columnIndex column index
+     * @return column type name
+     * @throws SQLException SQL Exception
+     */
+    String getColumnTypeName(int columnIndex) throws SQLException;
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/resourced/jdbc/queryresult/MemoryQueryResult.java b/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/resourced/jdbc/queryresult/MemoryQueryResult.java
index 0c478e0..2180fb1 100644
--- a/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/resourced/jdbc/queryresult/MemoryQueryResult.java
+++ b/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/resourced/jdbc/queryresult/MemoryQueryResult.java
@@ -114,12 +114,7 @@ public final class MemoryQueryResult implements QueryResult {
                 return resultSet.getObject(columnIndex);
         }
     }
-
-    @Override
-    public ResultSetMetaData getResultSetMetaData() {
-        return resultSetMetaData;
-    }
-
+    
     @Override
     public boolean next() {
         if (rows.hasNext()) {
@@ -174,4 +169,9 @@ public final class MemoryQueryResult implements QueryResult {
     public String getColumnLabel(final int columnIndex) throws SQLException {
         return resultSetMetaData.getColumnLabel(columnIndex);
     }
+    
+    @Override
+    public String getColumnTypeName(final int columnIndex) throws SQLException {
+        return resultSetMetaData.getColumnTypeName(columnIndex);
+    }
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/resourced/jdbc/queryresult/StreamQueryResult.java b/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/resourced/jdbc/queryresult/StreamQueryResult.java
index f8bd296..d30e20e 100644
--- a/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/resourced/jdbc/queryresult/StreamQueryResult.java
+++ b/shardingsphere-infra/shardingsphere-infra-executor/src/main/java/org/apache/shardingsphere/infra/executor/sql/resourced/jdbc/queryresult/StreamQueryResult.java
@@ -17,11 +17,11 @@
 
 package org.apache.shardingsphere.infra.executor.sql.resourced.jdbc.queryresult;
 
-import java.sql.Array;
 import org.apache.shardingsphere.infra.executor.sql.QueryResult;
 
 import java.io.InputStream;
 import java.math.BigDecimal;
+import java.sql.Array;
 import java.sql.Blob;
 import java.sql.Clob;
 import java.sql.Date;
@@ -45,12 +45,7 @@ public final class StreamQueryResult implements QueryResult {
         resultSetMetaData = resultSet.getMetaData();
         this.resultSet = resultSet;
     }
-
-    @Override
-    public ResultSetMetaData getResultSetMetaData() {
-        return resultSetMetaData;
-    }
-
+    
     @Override
     public boolean next() throws SQLException {
         return resultSet.next();
@@ -143,4 +138,9 @@ public final class StreamQueryResult implements QueryResult {
     public String getColumnLabel(final int columnIndex) throws SQLException {
         return resultSetMetaData.getColumnLabel(columnIndex);
     }
+    
+    @Override
+    public String getColumnTypeName(final int columnIndex) throws SQLException {
+        return resultSetMetaData.getColumnTypeName(columnIndex);
+    }
 }
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/admin/ShowTablesBackendHandler.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/admin/ShowTablesBackendHandler.java
index 6043376..70f961a 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/admin/ShowTablesBackendHandler.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/admin/ShowTablesBackendHandler.java
@@ -51,15 +51,15 @@ public final class ShowTablesBackendHandler implements TextProtocolBackendHandle
     
     @Override
     public BackendResponse execute() throws SQLException {
+        QueryResponse result = createQueryResponse(backendConnection.getSchemaName());
         if (!ProxyContext.getInstance().getMetaData(backendConnection.getSchemaName()).isComplete()) {
-            return getDefaultQueryResponse(backendConnection.getSchemaName());
+            return result;
         }
-        // TODO Get all tables from meta data
         databaseCommunicationEngine = databaseCommunicationEngineFactory.newTextProtocolInstance(sqlStatement, sql, backendConnection);
         return databaseCommunicationEngine.execute();
     }
     
-    private QueryResponse getDefaultQueryResponse(final String schemaName) {
+    private QueryResponse createQueryResponse(final String schemaName) {
         String column = String.format("Tables_in_%s", schemaName);
         return new QueryResponse(Collections.singletonList(new QueryHeader(schemaName, "", column, column, 64, Types.VARCHAR, 0, false, false, false, false)));
     }
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/main/java/org/apache/shardingsphere/proxy/frontend/postgresql/command/query/binary/bind/PostgreSQLComBindExecutor.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/main/java/org/apache/shardingsphere/proxy/frontend/postgresql/command/query/binary/bind/PostgreSQLComBindExecutor.java
index 0801e0d..c5955f6 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/main/java/org/apache/shardingsphere/proxy/frontend/postgresql/command/query/binary/bind/PostgreSQLComBindExecutor.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/main/java/org/apache/shardingsphere/proxy/frontend/postgresql/command/query/binary/bind/PostgreSQLComBindExecutor.java
@@ -45,7 +45,6 @@ import org.apache.shardingsphere.proxy.frontend.command.executor.QueryCommandExe
 import org.apache.shardingsphere.proxy.frontend.command.executor.ResponseType;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
 
-import java.sql.ResultSetMetaData;
 import java.sql.SQLException;
 import java.util.ArrayList;
 import java.util.Collection;
@@ -100,8 +99,8 @@ public final class PostgreSQLComBindExecutor implements QueryCommandExecutor {
         return result;
     }
     
-    private Optional<PostgreSQLRowDescriptionPacket> createQueryPacket(final QueryResponse queryResponse) {
-        List<PostgreSQLColumnDescription> columnDescriptions = getPostgreSQLColumnDescriptions(queryResponse);
+    private Optional<PostgreSQLRowDescriptionPacket> createQueryPacket(final QueryResponse queryResponse) throws SQLException {
+        Collection<PostgreSQLColumnDescription> columnDescriptions = createColumnDescriptions(queryResponse);
         if (columnDescriptions.isEmpty()) {
             responseType = ResponseType.QUERY;
         }
@@ -111,13 +110,13 @@ public final class PostgreSQLComBindExecutor implements QueryCommandExecutor {
         return Optional.of(new PostgreSQLRowDescriptionPacket(columnDescriptions.size(), columnDescriptions));
     }
     
-    private List<PostgreSQLColumnDescription> getPostgreSQLColumnDescriptions(final QueryResponse queryResponse) {
-        List<PostgreSQLColumnDescription> result = new LinkedList<>();
+    private Collection<PostgreSQLColumnDescription> createColumnDescriptions(final QueryResponse queryResponse) throws SQLException {
+        Collection<PostgreSQLColumnDescription> result = new LinkedList<>();
         List<QueryResult> queryResults = queryResponse.getQueryResults();
-        ResultSetMetaData resultSetMetaData = queryResults.isEmpty() ? null : queryResults.get(0).getResultSetMetaData();
         int columnIndex = 0;
         for (QueryHeader each : queryResponse.getQueryHeaders()) {
-            result.add(new PostgreSQLColumnDescription(each.getColumnName(), ++columnIndex, each.getColumnType(), each.getColumnLength(), resultSetMetaData));
+            String columnTypeName = queryResults.isEmpty() ? null : queryResults.get(0).getColumnTypeName(columnIndex + 1);
+            result.add(new PostgreSQLColumnDescription(each.getColumnName(), ++columnIndex, each.getColumnType(), each.getColumnLength(), columnTypeName));
         }
         return result;
     }
diff --git a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/main/java/org/apache/shardingsphere/proxy/frontend/postgresql/command/query/text/PostgreSQLComQueryExecutor.java b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/main/java/org/apache/shardingsphere/proxy/frontend/postgresql/command/query/text/PostgreSQLComQueryExecutor.java
index 947a6c6..7c2e52c 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/main/java/org/apache/shardingsphere/proxy/frontend/postgresql/command/query/text/PostgreSQLComQueryExecutor.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-frontend/shardingsphere-proxy-frontend-postgresql/src/main/java/org/apache/shardingsphere/proxy/frontend/postgresql/command/query/text/PostgreSQLComQueryExecutor.java
@@ -39,7 +39,6 @@ import org.apache.shardingsphere.proxy.backend.text.TextProtocolBackendHandlerFa
 import org.apache.shardingsphere.proxy.frontend.command.executor.QueryCommandExecutor;
 import org.apache.shardingsphere.proxy.frontend.command.executor.ResponseType;
 
-import java.sql.ResultSetMetaData;
 import java.sql.SQLException;
 import java.util.Collection;
 import java.util.Collections;
@@ -75,8 +74,8 @@ public final class PostgreSQLComQueryExecutor implements QueryCommandExecutor {
         return Collections.singletonList(createUpdatePacket((UpdateResponse) backendResponse));
     }
     
-    private Optional<PostgreSQLRowDescriptionPacket> createQueryPacket(final QueryResponse queryResponse) {
-        List<PostgreSQLColumnDescription> columnDescriptions = getPostgreSQLColumnDescriptions(queryResponse);
+    private Optional<PostgreSQLRowDescriptionPacket> createQueryPacket(final QueryResponse queryResponse) throws SQLException {
+        Collection<PostgreSQLColumnDescription> columnDescriptions = createColumnDescriptions(queryResponse);
         if (columnDescriptions.isEmpty()) {
             responseType = ResponseType.QUERY;
         }
@@ -86,13 +85,13 @@ public final class PostgreSQLComQueryExecutor implements QueryCommandExecutor {
         return Optional.of(new PostgreSQLRowDescriptionPacket(columnDescriptions.size(), columnDescriptions));
     }
     
-    private List<PostgreSQLColumnDescription> getPostgreSQLColumnDescriptions(final QueryResponse queryResponse) {
-        List<PostgreSQLColumnDescription> result = new LinkedList<>();
+    private Collection<PostgreSQLColumnDescription> createColumnDescriptions(final QueryResponse queryResponse) throws SQLException {
+        Collection<PostgreSQLColumnDescription> result = new LinkedList<>();
         List<QueryResult> queryResults = queryResponse.getQueryResults();
-        ResultSetMetaData resultSetMetaData = queryResults.isEmpty() ? null : queryResults.get(0).getResultSetMetaData();
         int columnIndex = 0;
         for (QueryHeader each : queryResponse.getQueryHeaders()) {
-            result.add(new PostgreSQLColumnDescription(each.getColumnName(), ++columnIndex, each.getColumnType(), each.getColumnLength(), resultSetMetaData));
+            String columnTypeName = queryResults.isEmpty() ? null : queryResults.get(0).getColumnTypeName(columnIndex + 1);
+            result.add(new PostgreSQLColumnDescription(each.getColumnName(), ++columnIndex, each.getColumnType(), each.getColumnLength(), columnTypeName));
         }
         return result;
     }