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 e176c655bb8 add support for dynamic (non-static) column types in 
Firebird Proxy (#36928)
e176c655bb8 is described below

commit e176c655bb890792aaff7a73216d0707fb5877fc
Author: Maxim Sentyabrskiy <[email protected]>
AuthorDate: Thu Oct 30 16:23:33 2025 +0300

    add support for dynamic (non-static) column types in Firebird Proxy (#36928)
    
    * Add FirebirdMetaDataLoader
    
    * Add dynamic VarChar
    
    * refactor
    
    * delete FirebirdLengthAwareTypes
    
    * correct Column Type Names
    
    * refactor
    
    * add Firebird BLOB segment size loader
    
    * spotless fix
    
    * refactor FirebirdBinaryColumnType to represent dynamic types with zero 
length
    
    * add new tests and fix existing ones
    
    * add support for remaining dynamic Firebird column types
---
 .../metadata/data/FirebirdSizeRegistry.java        |  96 +++++++++++++++
 .../data/loader/FirebirdColumnSizeLoader.java      | 131 +++++++++++++++++++++
 .../data/loader/FirebirdMetaDataLoader.java        |  56 +++++++++
 ...core.metadata.data.loader.DialectMetaDataLoader |  18 +++
 .../metadata/data/FirebirdSizeRegistryTest.java    |  71 +++++++++++
 .../data/loader/FirebirdColumnSizeLoaderTest.java  | 105 +++++++++++++++++
 .../data/loader/FirebirdMetaDataLoaderTest.java    |  74 ++++++++++++
 database/protocol/dialect/firebird/pom.xml         |   5 +
 .../command/query/FirebirdBinaryColumnType.java    |   9 +-
 .../prepare/FirebirdReturnColumnPacket.java        |   9 +-
 .../FirebirdPrepareStatementReturnPacketTest.java  |   2 +-
 .../prepare/FirebirdReturnColumnPacketTest.java    |  15 ++-
 .../FirebirdPrepareStatementCommandExecutor.java   |  13 +-
 13 files changed, 593 insertions(+), 11 deletions(-)

diff --git 
a/database/connector/dialect/firebird/src/main/java/org/apache/shardingsphere/database/connector/firebird/metadata/data/FirebirdSizeRegistry.java
 
b/database/connector/dialect/firebird/src/main/java/org/apache/shardingsphere/database/connector/firebird/metadata/data/FirebirdSizeRegistry.java
new file mode 100644
index 00000000000..010c5a3c437
--- /dev/null
+++ 
b/database/connector/dialect/firebird/src/main/java/org/apache/shardingsphere/database/connector/firebird/metadata/data/FirebirdSizeRegistry.java
@@ -0,0 +1,96 @@
+/*
+ * 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.database.connector.firebird.metadata.data;
+
+import lombok.AccessLevel;
+import lombok.NoArgsConstructor;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Locale;
+import java.util.Map;
+import java.util.OptionalInt;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * Registry for Firebird sizes.
+ */
+@NoArgsConstructor(access = AccessLevel.PRIVATE)
+public final class FirebirdSizeRegistry {
+    
+    private static final Map<String, Map<String, Integer>> COLUMN_SIZES = new 
ConcurrentHashMap<>();
+    
+    /**
+     * Refresh column sizes for a table.
+     *
+     * @param schemaName schema name
+     * @param tableName table name
+     * @param columnSizes column sizes map
+     */
+    public static void refreshTable(final String schemaName, final String 
tableName, final Map<String, Integer> columnSizes) {
+        if (null == tableName) {
+            return;
+        }
+        String tableKey = buildTableKey(schemaName, tableName);
+        if (columnSizes.isEmpty()) {
+            COLUMN_SIZES.remove(tableKey);
+            return;
+        }
+        Map<String, Integer> normalizedColumnSizes = new 
HashMap<>(columnSizes.size(), 1F);
+        for (Map.Entry<String, Integer> entry : columnSizes.entrySet()) {
+            if (null == entry.getKey()) {
+                continue;
+            }
+            normalizedColumnSizes.put(toKey(entry.getKey()), entry.getValue());
+        }
+        if (normalizedColumnSizes.isEmpty()) {
+            COLUMN_SIZES.remove(tableKey);
+            return;
+        }
+        COLUMN_SIZES.put(tableKey, 
Collections.unmodifiableMap(normalizedColumnSizes));
+    }
+    
+    /**
+     * Find registered column size.
+     *
+     * @param schemaName schema name
+     * @param tableName table name
+     * @param columnName column name
+     * @return column size
+     */
+    public static OptionalInt findColumnSize(final String schemaName, final 
String tableName, final String columnName) {
+        if (null == tableName || null == columnName) {
+            return OptionalInt.empty();
+        }
+        Map<String, Integer> tableSizes = 
COLUMN_SIZES.get(buildTableKey(schemaName, tableName));
+        if (null == tableSizes) {
+            return OptionalInt.empty();
+        }
+        Integer columnSize = tableSizes.get(toKey(columnName));
+        return null == columnSize ? OptionalInt.empty() : 
OptionalInt.of(columnSize);
+    }
+    
+    private static String buildTableKey(final String schemaName, final String 
tableName) {
+        String schemaKey = null == schemaName ? "" : toKey(schemaName);
+        return schemaKey + "." + toKey(tableName);
+    }
+    
+    private static String toKey(final String value) {
+        return value.toUpperCase(Locale.ENGLISH);
+    }
+}
diff --git 
a/database/connector/dialect/firebird/src/main/java/org/apache/shardingsphere/database/connector/firebird/metadata/data/loader/FirebirdColumnSizeLoader.java
 
b/database/connector/dialect/firebird/src/main/java/org/apache/shardingsphere/database/connector/firebird/metadata/data/loader/FirebirdColumnSizeLoader.java
new file mode 100644
index 00000000000..e0231e0a5ff
--- /dev/null
+++ 
b/database/connector/dialect/firebird/src/main/java/org/apache/shardingsphere/database/connector/firebird/metadata/data/loader/FirebirdColumnSizeLoader.java
@@ -0,0 +1,131 @@
+/*
+ * 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.database.connector.firebird.metadata.data.loader;
+
+import 
org.apache.shardingsphere.database.connector.core.metadata.data.loader.MetaDataLoaderConnection;
+import 
org.apache.shardingsphere.database.connector.core.metadata.data.loader.MetaDataLoaderMaterial;
+import 
org.apache.shardingsphere.database.connector.core.type.DatabaseTypeRegistry;
+
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Types;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Objects;
+
+/**
+ * Loader for Firebird column sizes.
+ */
+final class FirebirdColumnSizeLoader {
+    
+    private static final String LOAD_BLOB_SEGMENT_SIZES_SQL = "SELECT 
TRIM(rf.RDB$FIELD_NAME) AS COLUMN_NAME, "
+            + "COALESCE(f.RDB$SEGMENT_LENGTH, 0) AS SEGMENT_SIZE "
+            + "FROM RDB$RELATION_FIELDS rf "
+            + "JOIN RDB$FIELDS f ON rf.RDB$FIELD_SOURCE = f.RDB$FIELD_NAME "
+            + "WHERE TRIM(UPPER(rf.RDB$RELATION_NAME)) = ? "
+            + "AND f.RDB$FIELD_TYPE = 261";
+    
+    private final MetaDataLoaderMaterial material;
+    
+    FirebirdColumnSizeLoader(final MetaDataLoaderMaterial material) {
+        this.material = material;
+    }
+    
+    Map<String, Map<String, Integer>> load() throws SQLException {
+        if (material.getActualTableNames().isEmpty()) {
+            return Collections.emptyMap();
+        }
+        Map<String, Map<String, Integer>> result = new 
HashMap<>(material.getActualTableNames().size(), 1F);
+        DatabaseTypeRegistry databaseTypeRegistry = new 
DatabaseTypeRegistry(material.getStorageType());
+        try (MetaDataLoaderConnection connection = new 
MetaDataLoaderConnection(material.getStorageType(), 
material.getDataSource().getConnection())) {
+            for (String each : material.getActualTableNames()) {
+                String formattedTableName = 
databaseTypeRegistry.formatIdentifierPattern(each);
+                Map<String, Integer> columnSizes = 
loadTableColumnSizes(connection, formattedTableName);
+                result.put(each, columnSizes);
+            }
+        }
+        return result;
+    }
+    
+    private Map<String, Integer> loadTableColumnSizes(final 
MetaDataLoaderConnection connection, final String formattedTableName) throws 
SQLException {
+        Map<String, Integer> result = new HashMap<>();
+        loadColumnSizesFromMetaData(connection, formattedTableName, result);
+        loadBlobSegmentSizes(connection, formattedTableName, result);
+        return result.isEmpty() ? Collections.emptyMap() : 
Collections.unmodifiableMap(result);
+    }
+    
+    private void loadColumnSizesFromMetaData(final MetaDataLoaderConnection 
connection, final String formattedTableName, final Map<String, Integer> result) 
throws SQLException {
+        try (ResultSet resultSet = 
connection.getMetaData().getColumns(connection.getCatalog(), 
connection.getSchema(), formattedTableName, "%")) {
+            while (resultSet.next()) {
+                if (!Objects.equals(formattedTableName, 
resultSet.getString("TABLE_NAME"))) {
+                    continue;
+                }
+                int dataType = resultSet.getInt("DATA_TYPE");
+                if (!isDynamicLengthType(dataType)) {
+                    continue;
+                }
+                String columnName = resultSet.getString("COLUMN_NAME");
+                if (null != columnName) {
+                    int columnSize = resultSet.getInt("COLUMN_SIZE");
+                    if (!resultSet.wasNull() && columnSize > 0) {
+                        result.put(columnName.toUpperCase(Locale.ENGLISH), 
columnSize);
+                    }
+                }
+            }
+        }
+    }
+    
+    private void loadBlobSegmentSizes(final MetaDataLoaderConnection 
connection, final String formattedTableName, final Map<String, Integer> result) 
throws SQLException {
+        try (PreparedStatement preparedStatement = 
connection.prepareStatement(LOAD_BLOB_SEGMENT_SIZES_SQL)) {
+            preparedStatement.setString(1, 
formattedTableName.toUpperCase(Locale.ENGLISH));
+            try (ResultSet resultSet = preparedStatement.executeQuery()) {
+                while (resultSet.next()) {
+                    String columnName = resultSet.getString("COLUMN_NAME");
+                    if (null == columnName) {
+                        continue;
+                    }
+                    String trimmedColumnName = columnName.trim();
+                    if (trimmedColumnName.isEmpty()) {
+                        continue;
+                    }
+                    result.put(trimmedColumnName.toUpperCase(Locale.ENGLISH), 
resultSet.getInt("SEGMENT_SIZE"));
+                }
+            }
+        }
+    }
+    
+    private boolean isDynamicLengthType(final int dataType) {
+        switch (dataType) {
+            case Types.CHAR:
+            case Types.NCHAR:
+            case Types.VARCHAR:
+            case Types.NVARCHAR:
+            case Types.LONGVARCHAR:
+            case Types.LONGNVARCHAR:
+            case Types.BINARY:
+            case Types.VARBINARY:
+            case Types.LONGVARBINARY:
+                return true;
+            default:
+                return false;
+        }
+    }
+}
diff --git 
a/database/connector/dialect/firebird/src/main/java/org/apache/shardingsphere/database/connector/firebird/metadata/data/loader/FirebirdMetaDataLoader.java
 
b/database/connector/dialect/firebird/src/main/java/org/apache/shardingsphere/database/connector/firebird/metadata/data/loader/FirebirdMetaDataLoader.java
new file mode 100644
index 00000000000..c90a8a96cf0
--- /dev/null
+++ 
b/database/connector/dialect/firebird/src/main/java/org/apache/shardingsphere/database/connector/firebird/metadata/data/loader/FirebirdMetaDataLoader.java
@@ -0,0 +1,56 @@
+/*
+ * 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.database.connector.firebird.metadata.data.loader;
+
+import 
org.apache.shardingsphere.database.connector.core.metadata.data.loader.DialectMetaDataLoader;
+import 
org.apache.shardingsphere.database.connector.core.metadata.data.loader.MetaDataLoaderMaterial;
+import 
org.apache.shardingsphere.database.connector.core.metadata.data.loader.type.TableMetaDataLoader;
+import 
org.apache.shardingsphere.database.connector.core.metadata.data.model.SchemaMetaData;
+import 
org.apache.shardingsphere.database.connector.core.metadata.data.model.TableMetaData;
+import 
org.apache.shardingsphere.database.connector.firebird.metadata.data.FirebirdSizeRegistry;
+
+import java.sql.SQLException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.LinkedList;
+import java.util.Map;
+
+/**
+ * Meta data loader for Firebird.
+ */
+public final class FirebirdMetaDataLoader implements DialectMetaDataLoader {
+    
+    @Override
+    public Collection<SchemaMetaData> load(final MetaDataLoaderMaterial 
material) throws SQLException {
+        Collection<TableMetaData> tableMetaData = new LinkedList<>();
+        for (String each : material.getActualTableNames()) {
+            TableMetaDataLoader.load(material.getDataSource(), each, 
material.getStorageType()).ifPresent(tableMetaData::add);
+        }
+        Map<String, Map<String, Integer>> columnSizes = new 
FirebirdColumnSizeLoader(material).load();
+        for (String each : material.getActualTableNames()) {
+            Map<String, Integer> tableSizes = columnSizes.getOrDefault(each, 
Collections.emptyMap());
+            FirebirdSizeRegistry.refreshTable(material.getDefaultSchemaName(), 
each, tableSizes);
+        }
+        return Collections.singleton(new 
SchemaMetaData(material.getDefaultSchemaName(), tableMetaData));
+    }
+    
+    @Override
+    public String getDatabaseType() {
+        return "Firebird";
+    }
+}
diff --git 
a/database/connector/dialect/firebird/src/main/resources/META-INF/services/org.apache.shardingsphere.database.connector.core.metadata.data.loader.DialectMetaDataLoader
 
b/database/connector/dialect/firebird/src/main/resources/META-INF/services/org.apache.shardingsphere.database.connector.core.metadata.data.loader.DialectMetaDataLoader
new file mode 100644
index 00000000000..9913e407ac4
--- /dev/null
+++ 
b/database/connector/dialect/firebird/src/main/resources/META-INF/services/org.apache.shardingsphere.database.connector.core.metadata.data.loader.DialectMetaDataLoader
@@ -0,0 +1,18 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+org.apache.shardingsphere.database.connector.firebird.metadata.data.loader.FirebirdMetaDataLoader
diff --git 
a/database/connector/dialect/firebird/src/test/java/org/apache/shardingsphere/database/connector/firebird/metadata/data/FirebirdSizeRegistryTest.java
 
b/database/connector/dialect/firebird/src/test/java/org/apache/shardingsphere/database/connector/firebird/metadata/data/FirebirdSizeRegistryTest.java
new file mode 100644
index 00000000000..35f49d5de23
--- /dev/null
+++ 
b/database/connector/dialect/firebird/src/test/java/org/apache/shardingsphere/database/connector/firebird/metadata/data/FirebirdSizeRegistryTest.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.database.connector.firebird.metadata.data;
+
+import org.junit.jupiter.api.Test;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.OptionalInt;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+class FirebirdSizeRegistryTest {
+    
+    @Test
+    void assertRefreshAndFindColumnSize() {
+        Map<String, Integer> columnSizes = 
Collections.singletonMap("varchar_col", 64);
+        FirebirdSizeRegistry.refreshTable("schema_a", "table_a", columnSizes);
+        OptionalInt actual = FirebirdSizeRegistry.findColumnSize("schema_a", 
"table_a", "VARCHAR_COL");
+        assertTrue(actual.isPresent());
+        assertThat(actual.getAsInt(), is(64));
+        FirebirdSizeRegistry.refreshTable("schema_a", "table_a", 
Collections.emptyMap());
+    }
+    
+    @Test
+    void assertRefreshTableRemovesEntryWhenEmptyColumnSizesProvided() {
+        FirebirdSizeRegistry.refreshTable("schema_b", "table_b", 
Collections.singletonMap("col", 32));
+        FirebirdSizeRegistry.refreshTable("schema_b", "table_b", 
Collections.emptyMap());
+        assertFalse(FirebirdSizeRegistry.findColumnSize("schema_b", "table_b", 
"COL").isPresent());
+    }
+    
+    @Test
+    void assertRefreshTableSkipsNullColumnNames() {
+        Map<String, Integer> columnSizes = new HashMap<>(2, 1F);
+        columnSizes.put("valid", 12);
+        columnSizes.put(null, 24);
+        FirebirdSizeRegistry.refreshTable("schema_c", "table_c", columnSizes);
+        OptionalInt actual = FirebirdSizeRegistry.findColumnSize("schema_c", 
"table_c", "VaLiD");
+        assertTrue(actual.isPresent());
+        assertThat(actual.getAsInt(), is(12));
+        assertFalse(FirebirdSizeRegistry.findColumnSize("schema_c", "table_c", 
null).isPresent());
+        FirebirdSizeRegistry.refreshTable("schema_c", "table_c", 
Collections.emptyMap());
+    }
+    
+    @Test
+    void assertRefreshTableRemovesWhenAllColumnsInvalid() {
+        Map<String, Integer> columnSizes = new HashMap<>(1, 1F);
+        columnSizes.put(null, 48);
+        FirebirdSizeRegistry.refreshTable("schema_d", "table_d", columnSizes);
+        assertFalse(FirebirdSizeRegistry.findColumnSize("schema_d", "table_d", 
"any").isPresent());
+    }
+}
diff --git 
a/database/connector/dialect/firebird/src/test/java/org/apache/shardingsphere/database/connector/firebird/metadata/data/loader/FirebirdColumnSizeLoaderTest.java
 
b/database/connector/dialect/firebird/src/test/java/org/apache/shardingsphere/database/connector/firebird/metadata/data/loader/FirebirdColumnSizeLoaderTest.java
new file mode 100644
index 00000000000..c30223aedcf
--- /dev/null
+++ 
b/database/connector/dialect/firebird/src/test/java/org/apache/shardingsphere/database/connector/firebird/metadata/data/loader/FirebirdColumnSizeLoaderTest.java
@@ -0,0 +1,105 @@
+/*
+ * 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.database.connector.firebird.metadata.data.loader;
+
+import 
org.apache.shardingsphere.database.connector.core.metadata.data.loader.MetaDataLoaderMaterial;
+import org.apache.shardingsphere.database.connector.core.type.DatabaseType;
+import org.apache.shardingsphere.infra.spi.type.typed.TypedSPILoader;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.mockito.Mock;
+import org.mockito.junit.jupiter.MockitoExtension;
+
+import javax.sql.DataSource;
+import java.sql.Connection;
+import java.sql.DatabaseMetaData;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Types;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Map;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.hasKey;
+import static org.mockito.ArgumentMatchers.anyString;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+
+@ExtendWith(MockitoExtension.class)
+class FirebirdColumnSizeLoaderTest {
+    
+    private static final Collection<String> TABLES = 
Collections.singleton("test_table");
+    
+    @Mock
+    private DataSource dataSource;
+    
+    @Mock
+    private Connection connection;
+    
+    @Mock
+    private DatabaseMetaData databaseMetaData;
+    
+    @Mock
+    private ResultSet columnsResultSet;
+    
+    @Mock
+    private PreparedStatement preparedStatement;
+    
+    @Mock
+    private ResultSet blobResultSet;
+    
+    private MetaDataLoaderMaterial material;
+    
+    @BeforeEach
+    void setUp() throws SQLException {
+        DatabaseType databaseType = 
TypedSPILoader.getService(DatabaseType.class, "Firebird");
+        material = new MetaDataLoaderMaterial(TABLES, "logic_ds", dataSource, 
databaseType, "schema");
+        when(dataSource.getConnection()).thenReturn(connection);
+        when(connection.getMetaData()).thenReturn(databaseMetaData);
+        when(connection.getCatalog()).thenReturn("catalog");
+        when(connection.getSchema()).thenReturn("schema");
+        
when(connection.prepareStatement(anyString())).thenReturn(preparedStatement);
+        when(preparedStatement.executeQuery()).thenReturn(blobResultSet);
+        when(databaseMetaData.getColumns("catalog", "schema", "TEST_TABLE", 
"%")).thenReturn(columnsResultSet);
+    }
+    
+    @Test
+    void assertLoadReturnsCombinedVarcharAndBlobSizes() throws SQLException {
+        when(columnsResultSet.next()).thenReturn(true, true, true, false);
+        
when(columnsResultSet.getString("TABLE_NAME")).thenReturn("TEST_TABLE", 
"TEST_TABLE", "TEST_TABLE");
+        when(columnsResultSet.getInt("DATA_TYPE")).thenReturn(Types.CHAR, 
Types.BIGINT, Types.BLOB);
+        when(columnsResultSet.getString("COLUMN_NAME")).thenReturn("char_col", 
"bigint_col", "ignored_blob");
+        when(columnsResultSet.getInt("COLUMN_SIZE")).thenReturn(128, 19, 0);
+        when(blobResultSet.next()).thenReturn(true, true, false);
+        when(blobResultSet.getString("COLUMN_NAME")).thenReturn(" blob_col ", 
"   ");
+        when(blobResultSet.getInt("SEGMENT_SIZE")).thenReturn(2048, 4096);
+        Map<String, Map<String, Integer>> actual = new 
FirebirdColumnSizeLoader(material).load();
+        assertThat(actual, hasKey("test_table"));
+        Map<String, Integer> tableSizes = actual.get("test_table");
+        assertThat(tableSizes.size(), is(2));
+        assertThat(tableSizes.get("CHAR_COL"), is(128));
+        assertFalse(tableSizes.containsKey("BIGINT_COL"));
+        assertThat(tableSizes.get("BLOB_COL"), is(2048));
+        verify(preparedStatement).setString(1, "TEST_TABLE");
+    }
+}
diff --git 
a/database/connector/dialect/firebird/src/test/java/org/apache/shardingsphere/database/connector/firebird/metadata/data/loader/FirebirdMetaDataLoaderTest.java
 
b/database/connector/dialect/firebird/src/test/java/org/apache/shardingsphere/database/connector/firebird/metadata/data/loader/FirebirdMetaDataLoaderTest.java
new file mode 100644
index 00000000000..2f2793acc34
--- /dev/null
+++ 
b/database/connector/dialect/firebird/src/test/java/org/apache/shardingsphere/database/connector/firebird/metadata/data/loader/FirebirdMetaDataLoaderTest.java
@@ -0,0 +1,74 @@
+/*
+ * 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.database.connector.firebird.metadata.data.loader;
+
+import 
org.apache.shardingsphere.database.connector.core.metadata.data.loader.MetaDataLoaderMaterial;
+import 
org.apache.shardingsphere.database.connector.core.metadata.data.loader.type.TableMetaDataLoader;
+import 
org.apache.shardingsphere.database.connector.core.metadata.data.model.SchemaMetaData;
+import 
org.apache.shardingsphere.database.connector.core.metadata.data.model.TableMetaData;
+import org.apache.shardingsphere.database.connector.core.type.DatabaseType;
+import 
org.apache.shardingsphere.database.connector.firebird.metadata.data.FirebirdSizeRegistry;
+import org.apache.shardingsphere.infra.spi.type.typed.TypedSPILoader;
+import org.junit.jupiter.api.Test;
+import org.mockito.MockedConstruction;
+import org.mockito.MockedStatic;
+
+import javax.sql.DataSource;
+import java.sql.SQLException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Optional;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.contains;
+import static org.hamcrest.Matchers.hasSize;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.mockConstruction;
+import static org.mockito.Mockito.mockStatic;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+class FirebirdMetaDataLoaderTest {
+    
+    @Test
+    void assertLoadRefreshesSizeRegistry() throws SQLException {
+        DataSource dataSource = mock(DataSource.class);
+        DatabaseType databaseType = 
TypedSPILoader.getService(DatabaseType.class, "Firebird");
+        MetaDataLoaderMaterial material = new 
MetaDataLoaderMaterial(Collections.singleton("test_table"), "logic_ds", 
dataSource,
+                databaseType, "schema");
+        TableMetaData tableMetaData = new TableMetaData("test_table", 
Collections.emptyList(), Collections.emptyList(), Collections.emptyList());
+        Map<String, Integer> tableSizes = Collections.singletonMap("COLUMN", 
16);
+        Map<String, Map<String, Integer>> allSizes = 
Collections.singletonMap("test_table", tableSizes);
+        try (
+                MockedStatic<TableMetaDataLoader> tableLoaderMocked = 
mockStatic(TableMetaDataLoader.class);
+                MockedStatic<FirebirdSizeRegistry> sizeRegistryMocked = 
mockStatic(FirebirdSizeRegistry.class);
+                MockedConstruction<FirebirdColumnSizeLoader> 
columnSizeLoaderMocked = mockConstruction(FirebirdColumnSizeLoader.class,
+                        (mock, context) -> 
when(mock.load()).thenReturn(allSizes))) {
+            tableLoaderMocked.when(() -> TableMetaDataLoader.load(dataSource, 
"test_table", databaseType)).thenReturn(Optional.of(tableMetaData));
+            Collection<SchemaMetaData> actual = new 
FirebirdMetaDataLoader().load(material);
+            assertThat(actual, hasSize(1));
+            SchemaMetaData schema = actual.iterator().next();
+            assertThat(schema.getName(), is("schema"));
+            assertThat(schema.getTables(), contains(tableMetaData));
+            sizeRegistryMocked.verify(() -> 
FirebirdSizeRegistry.refreshTable("schema", "test_table", tableSizes));
+            verify(columnSizeLoaderMocked.constructed().get(0)).load();
+        }
+    }
+}
diff --git a/database/protocol/dialect/firebird/pom.xml 
b/database/protocol/dialect/firebird/pom.xml
index 7342bc649ff..73ebedba867 100644
--- a/database/protocol/dialect/firebird/pom.xml
+++ b/database/protocol/dialect/firebird/pom.xml
@@ -32,6 +32,11 @@
             <artifactId>shardingsphere-database-protocol-core</artifactId>
             <version>${project.version}</version>
         </dependency>
+        <dependency>
+            <groupId>org.apache.shardingsphere</groupId>
+            <artifactId>shardingsphere-database-connector-firebird</artifactId>
+            <version>${project.version}</version>
+        </dependency>
         <dependency>
             <groupId>org.apache.shardingsphere</groupId>
             <artifactId>shardingsphere-database-exception-core</artifactId>
diff --git 
a/database/protocol/dialect/firebird/src/main/java/org/apache/shardingsphere/database/protocol/firebird/packet/command/query/FirebirdBinaryColumnType.java
 
b/database/protocol/dialect/firebird/src/main/java/org/apache/shardingsphere/database/protocol/firebird/packet/command/query/FirebirdBinaryColumnType.java
index e5db4101480..3d42ecf0507 100644
--- 
a/database/protocol/dialect/firebird/src/main/java/org/apache/shardingsphere/database/protocol/firebird/packet/command/query/FirebirdBinaryColumnType.java
+++ 
b/database/protocol/dialect/firebird/src/main/java/org/apache/shardingsphere/database/protocol/firebird/packet/command/query/FirebirdBinaryColumnType.java
@@ -34,18 +34,19 @@ import java.util.Map;
 @Getter
 public enum FirebirdBinaryColumnType implements BinaryColumnType {
     
-    // TODO add different varying length based on a row length
+    // TODO add support for retrieving length for ARRAY types.
+    // Currently not possible due to existing issues with ARRAY handling in 
the database itself.
     TEXT(452, 255),
-    VARYING(448, 255),
+    VARYING(448, 0),
     LEGACY_TEXT(452, 255),
-    LEGACY_VARYING(448, 255),
+    LEGACY_VARYING(448, 0),
     SHORT(500, 2),
     LONG(496, 4),
     FLOAT(482, 4),
     DOUBLE(480, 8),
     D_FLOAT(530, 8),
     TIMESTAMP(510, 8),
-    BLOB(520, 8),
+    BLOB(520, 0),
     ARRAY(540, 255),
     QUAD(550, 4),
     TIME(560, 4),
diff --git 
a/database/protocol/dialect/firebird/src/main/java/org/apache/shardingsphere/database/protocol/firebird/packet/command/query/statement/prepare/FirebirdReturnColumnPacket.java
 
b/database/protocol/dialect/firebird/src/main/java/org/apache/shardingsphere/database/protocol/firebird/packet/command/query/statement/prepare/FirebirdReturnColumnPacket.java
index 1265f2f73ff..ea0f8ebfb99 100644
--- 
a/database/protocol/dialect/firebird/src/main/java/org/apache/shardingsphere/database/protocol/firebird/packet/command/query/statement/prepare/FirebirdReturnColumnPacket.java
+++ 
b/database/protocol/dialect/firebird/src/main/java/org/apache/shardingsphere/database/protocol/firebird/packet/command/query/statement/prepare/FirebirdReturnColumnPacket.java
@@ -48,24 +48,27 @@ public final class FirebirdReturnColumnPacket extends 
FirebirdPacket {
     
     private final String owner;
     
+    private final Integer columnLength;
+    
     @Override
     protected void write(final FirebirdPacketPayload payload) {
+        FirebirdBinaryColumnType columnType = 
FirebirdBinaryColumnType.valueOfJDBCType(column.getDataType());
         for (FirebirdSQLInfoPacketType requestedItem : requestedItems) {
             switch (requestedItem) {
                 case SQLDA_SEQ:
                     
FirebirdPrepareStatementReturnPacket.writeInt(FirebirdSQLInfoPacketType.SQLDA_SEQ,
 index, payload);
                     break;
                 case TYPE:
-                    
FirebirdPrepareStatementReturnPacket.writeInt(FirebirdSQLInfoPacketType.TYPE, 
FirebirdBinaryColumnType.valueOfJDBCType(column.getDataType()).getValue() + 1, 
payload);
+                    
FirebirdPrepareStatementReturnPacket.writeInt(FirebirdSQLInfoPacketType.TYPE, 
columnType.getValue() + 1, payload);
                     break;
                 case SUB_TYPE:
-                    
FirebirdPrepareStatementReturnPacket.writeInt(FirebirdSQLInfoPacketType.SUB_TYPE,
 FirebirdBinaryColumnType.valueOfJDBCType(column.getDataType()).getSubtype(), 
payload);
+                    
FirebirdPrepareStatementReturnPacket.writeInt(FirebirdSQLInfoPacketType.SUB_TYPE,
 columnType.getSubtype(), payload);
                     break;
                 case SCALE:
                     
FirebirdPrepareStatementReturnPacket.writeInt(FirebirdSQLInfoPacketType.SCALE, 
0, payload);
                     break;
                 case LENGTH:
-                    
FirebirdPrepareStatementReturnPacket.writeInt(FirebirdSQLInfoPacketType.LENGTH, 
FirebirdBinaryColumnType.valueOfJDBCType(column.getDataType()).getLength(), 
payload);
+                    
FirebirdPrepareStatementReturnPacket.writeInt(FirebirdSQLInfoPacketType.LENGTH, 
null != columnLength ? columnLength : columnType.getLength(), payload);
                     break;
                 case FIELD:
                     
FirebirdPrepareStatementReturnPacket.writeString(FirebirdSQLInfoPacketType.FIELD,
 column.getName(), payload);
diff --git 
a/database/protocol/dialect/firebird/src/test/java/org/apache/shardingsphere/database/protocol/firebird/packet/command/query/statement/prepare/FirebirdPrepareStatementReturnPacketTest.java
 
b/database/protocol/dialect/firebird/src/test/java/org/apache/shardingsphere/database/protocol/firebird/packet/command/query/statement/prepare/FirebirdPrepareStatementReturnPacketTest.java
index 403c10516e4..730c5b52ddd 100644
--- 
a/database/protocol/dialect/firebird/src/test/java/org/apache/shardingsphere/database/protocol/firebird/packet/command/query/statement/prepare/FirebirdPrepareStatementReturnPacketTest.java
+++ 
b/database/protocol/dialect/firebird/src/test/java/org/apache/shardingsphere/database/protocol/firebird/packet/command/query/statement/prepare/FirebirdPrepareStatementReturnPacketTest.java
@@ -46,7 +46,7 @@ class FirebirdPrepareStatementReturnPacketTest {
         ShardingSphereColumn column = new ShardingSphereColumn("col", 
Types.INTEGER, false, false, false, true, false, true);
         ShardingSphereTable table = new ShardingSphereTable("tbl", 
Collections.singleton(column), Collections.emptyList(), 
Collections.emptyList());
         FirebirdReturnColumnPacket columnPacket =
-                new 
FirebirdReturnColumnPacket(Collections.singleton(FirebirdSQLInfoPacketType.DESCRIBE_END),
 1, table, column, "", "", "");
+                new 
FirebirdReturnColumnPacket(Collections.singleton(FirebirdSQLInfoPacketType.DESCRIBE_END),
 1, table, column, "", "", "", null);
         packet.getDescribeSelect().add(columnPacket);
         packet.getDescribeBind().add(columnPacket);
         packet.write(payload);
diff --git 
a/database/protocol/dialect/firebird/src/test/java/org/apache/shardingsphere/database/protocol/firebird/packet/command/query/statement/prepare/FirebirdReturnColumnPacketTest.java
 
b/database/protocol/dialect/firebird/src/test/java/org/apache/shardingsphere/database/protocol/firebird/packet/command/query/statement/prepare/FirebirdReturnColumnPacketTest.java
index aba3b5a2f72..0f556d6fcdc 100644
--- 
a/database/protocol/dialect/firebird/src/test/java/org/apache/shardingsphere/database/protocol/firebird/packet/command/query/statement/prepare/FirebirdReturnColumnPacketTest.java
+++ 
b/database/protocol/dialect/firebird/src/test/java/org/apache/shardingsphere/database/protocol/firebird/packet/command/query/statement/prepare/FirebirdReturnColumnPacketTest.java
@@ -41,7 +41,7 @@ class FirebirdReturnColumnPacketTest {
     
     @Test
     void assertWrite() {
-        ShardingSphereColumn column = new ShardingSphereColumn("col", 
Types.INTEGER, false, false, false, true, false, true);
+        ShardingSphereColumn column = new ShardingSphereColumn("col", 
Types.VARCHAR, false, false, false, true, false, true);
         ShardingSphereTable table = new ShardingSphereTable("tbl", 
Collections.singleton(column), Collections.emptyList(), 
Collections.emptyList());
         FirebirdReturnColumnPacket packet = new 
FirebirdReturnColumnPacket(Arrays.asList(
                 FirebirdSQLInfoPacketType.SQLDA_SEQ,
@@ -54,10 +54,21 @@ class FirebirdReturnColumnPacketTest {
                 FirebirdSQLInfoPacketType.RELATION,
                 FirebirdSQLInfoPacketType.RELATION_ALIAS,
                 FirebirdSQLInfoPacketType.OWNER,
-                FirebirdSQLInfoPacketType.DESCRIBE_END), 1, table, column, 
"t", "c", "o");
+                FirebirdSQLInfoPacketType.DESCRIBE_END), 1, table, column, 
"t", "c", "o", 99);
         
when(payload.getCharset()).thenReturn(java.nio.charset.StandardCharsets.UTF_8);
         packet.write(payload);
         
verify(payload).writeInt1(FirebirdSQLInfoPacketType.SQLDA_SEQ.getCode());
         
verify(payload).writeInt1(FirebirdSQLInfoPacketType.DESCRIBE_END.getCode());
+        verify(payload).writeInt4LE(99);
+    }
+    
+    @Test
+    void assertWriteUsesDefaultColumnLength() {
+        ShardingSphereColumn column = new ShardingSphereColumn("col", 
Types.INTEGER, false, false, false, true, false, true);
+        ShardingSphereTable table = new ShardingSphereTable("tbl", 
Collections.singleton(column), Collections.emptyList(), 
Collections.emptyList());
+        FirebirdReturnColumnPacket packet = new 
FirebirdReturnColumnPacket(Collections.singletonList(FirebirdSQLInfoPacketType.LENGTH),
+                1, table, column, "t", "c", "o", null);
+        packet.write(payload);
+        verify(payload).writeInt4LE(4);
     }
 }
diff --git 
a/proxy/frontend/dialect/firebird/src/main/java/org/apache/shardingsphere/proxy/frontend/firebird/command/query/statement/prepare/FirebirdPrepareStatementCommandExecutor.java
 
b/proxy/frontend/dialect/firebird/src/main/java/org/apache/shardingsphere/proxy/frontend/firebird/command/query/statement/prepare/FirebirdPrepareStatementCommandExecutor.java
index ad845566744..1377cbd8100 100644
--- 
a/proxy/frontend/dialect/firebird/src/main/java/org/apache/shardingsphere/proxy/frontend/firebird/command/query/statement/prepare/FirebirdPrepareStatementCommandExecutor.java
+++ 
b/proxy/frontend/dialect/firebird/src/main/java/org/apache/shardingsphere/proxy/frontend/firebird/command/query/statement/prepare/FirebirdPrepareStatementCommandExecutor.java
@@ -20,6 +20,7 @@ package 
org.apache.shardingsphere.proxy.frontend.firebird.command.query.statemen
 import lombok.RequiredArgsConstructor;
 import org.apache.shardingsphere.database.connector.core.type.DatabaseType;
 import 
org.apache.shardingsphere.database.connector.core.type.DatabaseTypeRegistry;
+import 
org.apache.shardingsphere.database.connector.firebird.metadata.data.FirebirdSizeRegistry;
 import 
org.apache.shardingsphere.database.protocol.firebird.exception.FirebirdProtocolException;
 import 
org.apache.shardingsphere.database.protocol.firebird.packet.command.query.info.type.sql.FirebirdSQLInfoPacketType;
 import 
org.apache.shardingsphere.database.protocol.firebird.packet.command.query.info.type.sql.FirebirdSQLInfoReturnValue;
@@ -88,6 +89,7 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.LinkedList;
 import java.util.Map;
+import java.util.OptionalInt;
 
 /**
  * Firebird prepare transaction command executor.
@@ -457,6 +459,15 @@ public final class FirebirdPrepareStatementCommandExecutor 
implements CommandExe
         String tableAliasString = null == tableAlias ? table.getName() : 
tableAlias.getValue();
         String columnAliasString = null == columnAlias ? column.getName() : 
columnAlias.getValue();
         String owner = 
connectionSession.getConnectionContext().getGrantee().getUsername();
-        describeColumns.add(new FirebirdReturnColumnPacket(requestedItems, 
idx, table, column, tableAliasString, columnAliasString, owner));
+        Integer columnLength = resolveColumnLength(table, column);
+        describeColumns.add(new FirebirdReturnColumnPacket(requestedItems, 
idx, table, column, tableAliasString, columnAliasString, owner, columnLength));
+    }
+    
+    private Integer resolveColumnLength(final ShardingSphereTable table, final 
ShardingSphereColumn column) {
+        if (null == table || null == column) {
+            return null;
+        }
+        OptionalInt columnSize = 
FirebirdSizeRegistry.findColumnSize(connectionSession.getCurrentDatabaseName(), 
table.getName(), column.getName());
+        return columnSize.isPresent() ? columnSize.getAsInt() : null;
     }
 }


Reply via email to