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 0d4c05c290b Add SQLFederationDatabaseTest, SQLFederationSchemaTest and 
SQLFederationTableTest (#37327)
0d4c05c290b is described below

commit 0d4c05c290bbe999e6ea96cde9fb09b718829148
Author: Liang Zhang <[email protected]>
AuthorDate: Wed Dec 10 16:33:14 2025 +0800

    Add SQLFederationDatabaseTest, SQLFederationSchemaTest and 
SQLFederationTableTest (#37327)
    
    * Add SQLFederationDatabaseTest, SQLFederationSchemaTest and 
SQLFederationTableTest
    
    * Add SQLFederationDatabaseTest, SQLFederationSchemaTest and 
SQLFederationTableTest
---
 .../metadata/schema/SQLFederationTable.java        |  90 ++++++-------
 .../metadata/schema/SQLFederationDatabaseTest.java |  55 ++++++++
 .../metadata/schema/SQLFederationSchemaTest.java   |  56 ++++++++
 .../metadata/schema/SQLFederationTableTest.java    | 146 +++++++++++++++++++++
 4 files changed, 300 insertions(+), 47 deletions(-)

diff --git 
a/kernel/sql-federation/compiler/src/main/java/org/apache/shardingsphere/sqlfederation/compiler/metadata/schema/SQLFederationTable.java
 
b/kernel/sql-federation/compiler/src/main/java/org/apache/shardingsphere/sqlfederation/compiler/metadata/schema/SQLFederationTable.java
index b0e7e92b0fd..8a6d4ceb869 100644
--- 
a/kernel/sql-federation/compiler/src/main/java/org/apache/shardingsphere/sqlfederation/compiler/metadata/schema/SQLFederationTable.java
+++ 
b/kernel/sql-federation/compiler/src/main/java/org/apache/shardingsphere/sqlfederation/compiler/metadata/schema/SQLFederationTable.java
@@ -68,45 +68,17 @@ public final class SQLFederationTable extends AbstractTable 
implements Modifiabl
     
     private final DatabaseType protocolType;
     
-    @Override
-    public RelDataType getRowType(final RelDataTypeFactory typeFactory) {
-        return SQLFederationDataTypeBuilder.build(table, protocolType, 
typeFactory);
-    }
-    
-    @Override
-    public Expression getExpression(final SchemaPlus schema, final String 
tableName, final Class clazz) {
-        return Schemas.tableExpression(schema, getElementType(), tableName, 
clazz);
-    }
-    
-    @Override
-    public Type getElementType() {
-        return Object[].class;
-    }
-    
-    @Override
-    public <T> Queryable<T> asQueryable(final QueryProvider queryProvider, 
final SchemaPlus schema, final String tableName) {
-        throw new UnsupportedSQLOperationException("asQueryable");
-    }
-    
-    @Override
-    public RelNode toRel(final ToRelContext context, final RelOptTable 
relOptTable) {
-        return LogicalTableScan.create(context.getCluster(), relOptTable, 
Collections.emptyList());
-    }
-    
     /**
      * Implement.
      *
      * @param root data context
-     * @param sql sql
+     * @param sql SQL
      * @param paramIndexes param indexes
      * @return enumerable result
      */
     @SuppressWarnings("unused")
     public Enumerable<Object> implement(final DataContext root, final String 
sql, final int[] paramIndexes) {
-        if (null == SCAN_IMPLEMENTOR_HOLDER.get()) {
-            return createEmptyEnumerable();
-        }
-        return SCAN_IMPLEMENTOR_HOLDER.get().implement(table, new 
ScanImplementorContext(root, sql, paramIndexes));
+        return null == SCAN_IMPLEMENTOR_HOLDER.get() ? createEmptyEnumerable() 
: SCAN_IMPLEMENTOR_HOLDER.get().implement(table, new 
ScanImplementorContext(root, sql, paramIndexes));
     }
     
     private AbstractEnumerable<Object> createEmptyEnumerable() {
@@ -119,23 +91,6 @@ public final class SQLFederationTable extends AbstractTable 
implements Modifiabl
         };
     }
     
-    @Override
-    public String toString() {
-        return "SQLFederationTable";
-    }
-    
-    @Override
-    public Collection<Object[]> getModifiableCollection() {
-        throw new UnsupportedOperationException();
-    }
-    
-    @Override
-    public TableModify toModificationRel(final RelOptCluster relOptCluster, 
final RelOptTable table, final CatalogReader schema,
-                                         final RelNode relNode, final 
Operation operation, final List<String> updateColumnList,
-                                         final List<RexNode> 
sourceExpressionList, final boolean flattened) {
-        return LogicalTableModify.create(table, schema, relNode, operation, 
updateColumnList, sourceExpressionList, flattened);
-    }
-    
     /**
      * Set scan implementor.
      *
@@ -151,4 +106,45 @@ public final class SQLFederationTable extends 
AbstractTable implements Modifiabl
     public void clearScanImplementor() {
         SCAN_IMPLEMENTOR_HOLDER.remove();
     }
+    
+    @Override
+    public RelDataType getRowType(final RelDataTypeFactory typeFactory) {
+        return SQLFederationDataTypeBuilder.build(table, protocolType, 
typeFactory);
+    }
+    
+    @Override
+    public Expression getExpression(final SchemaPlus schema, final String 
tableName, final Class clazz) {
+        return Schemas.tableExpression(schema, getElementType(), tableName, 
clazz);
+    }
+    
+    @Override
+    public Type getElementType() {
+        return Object[].class;
+    }
+    
+    @Override
+    public <T> Queryable<T> asQueryable(final QueryProvider queryProvider, 
final SchemaPlus schema, final String tableName) {
+        throw new UnsupportedSQLOperationException("asQueryable");
+    }
+    
+    @Override
+    public RelNode toRel(final ToRelContext context, final RelOptTable 
relOptTable) {
+        return LogicalTableScan.create(context.getCluster(), relOptTable, 
Collections.emptyList());
+    }
+    
+    @Override
+    public Collection<Object[]> getModifiableCollection() {
+        throw new UnsupportedOperationException();
+    }
+    
+    @Override
+    public TableModify toModificationRel(final RelOptCluster relOptCluster, 
final RelOptTable table, final CatalogReader schema,
+                                         final RelNode relNode, final 
Operation operation, final List<String> updateColumnList, final List<RexNode> 
sourceExpressionList, final boolean flattened) {
+        return LogicalTableModify.create(table, schema, relNode, operation, 
updateColumnList, sourceExpressionList, flattened);
+    }
+    
+    @Override
+    public String toString() {
+        return "SQLFederationTable";
+    }
 }
diff --git 
a/kernel/sql-federation/compiler/src/test/java/org/apache/shardingsphere/sqlfederation/compiler/metadata/schema/SQLFederationDatabaseTest.java
 
b/kernel/sql-federation/compiler/src/test/java/org/apache/shardingsphere/sqlfederation/compiler/metadata/schema/SQLFederationDatabaseTest.java
new file mode 100644
index 00000000000..d29800652ff
--- /dev/null
+++ 
b/kernel/sql-federation/compiler/src/test/java/org/apache/shardingsphere/sqlfederation/compiler/metadata/schema/SQLFederationDatabaseTest.java
@@ -0,0 +1,55 @@
+/*
+ * 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.sqlfederation.compiler.metadata.schema;
+
+import org.apache.shardingsphere.database.connector.core.type.DatabaseType;
+import 
org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
+import 
org.apache.shardingsphere.infra.metadata.database.resource.ResourceMetaData;
+import org.apache.shardingsphere.infra.metadata.database.rule.RuleMetaData;
+import 
org.apache.shardingsphere.infra.metadata.database.schema.model.ShardingSphereColumn;
+import 
org.apache.shardingsphere.infra.metadata.database.schema.model.ShardingSphereSchema;
+import 
org.apache.shardingsphere.infra.metadata.database.schema.model.ShardingSphereTable;
+import org.apache.shardingsphere.infra.spi.type.typed.TypedSPILoader;
+import org.junit.jupiter.api.Test;
+
+import java.sql.Types;
+import java.util.Collections;
+
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.is;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.mockito.Mockito.mock;
+
+class SQLFederationDatabaseTest {
+    
+    private final DatabaseType databaseType = 
TypedSPILoader.getService(DatabaseType.class, "FIXTURE");
+    
+    @Test
+    void assertNew() {
+        ShardingSphereSchema schema = new ShardingSphereSchema("foo_schema", 
Collections.singleton(createTable()), Collections.emptyList());
+        ShardingSphereDatabase database = new ShardingSphereDatabase("foo_db", 
databaseType, mock(ResourceMetaData.class), new 
RuleMetaData(Collections.emptyList()), Collections.singleton(schema));
+        SQLFederationDatabase actual = new SQLFederationDatabase(database, 
databaseType);
+        assertThat(actual.getName(), is("foo_db"));
+        assertNotNull(actual.getSubSchemaMap().get("foo_schema"));
+    }
+    
+    private ShardingSphereTable createTable() {
+        ShardingSphereColumn column = new ShardingSphereColumn("id", 
Types.INTEGER, true, false, false, true, false, true);
+        return new ShardingSphereTable("foo_tbl", 
Collections.singletonList(column), Collections.emptyList(), 
Collections.emptyList());
+    }
+}
diff --git 
a/kernel/sql-federation/compiler/src/test/java/org/apache/shardingsphere/sqlfederation/compiler/metadata/schema/SQLFederationSchemaTest.java
 
b/kernel/sql-federation/compiler/src/test/java/org/apache/shardingsphere/sqlfederation/compiler/metadata/schema/SQLFederationSchemaTest.java
new file mode 100644
index 00000000000..c4bce642692
--- /dev/null
+++ 
b/kernel/sql-federation/compiler/src/test/java/org/apache/shardingsphere/sqlfederation/compiler/metadata/schema/SQLFederationSchemaTest.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.sqlfederation.compiler.metadata.schema;
+
+import org.apache.calcite.schema.impl.ViewTable;
+import org.apache.shardingsphere.database.connector.core.type.DatabaseType;
+import 
org.apache.shardingsphere.infra.metadata.database.schema.model.ShardingSphereColumn;
+import 
org.apache.shardingsphere.infra.metadata.database.schema.model.ShardingSphereSchema;
+import 
org.apache.shardingsphere.infra.metadata.database.schema.model.ShardingSphereTable;
+import 
org.apache.shardingsphere.infra.metadata.database.schema.model.ShardingSphereView;
+import org.apache.shardingsphere.infra.spi.type.typed.TypedSPILoader;
+import org.junit.jupiter.api.Test;
+
+import java.sql.Types;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.is;
+import static org.hamcrest.Matchers.isA;
+
+class SQLFederationSchemaTest {
+    
+    private final DatabaseType databaseType = 
TypedSPILoader.getService(DatabaseType.class, "FIXTURE");
+    
+    @Test
+    void assertNew() {
+        Collection<ShardingSphereTable> tables = 
Arrays.asList(createTable("foo_table"), createTable("foo_view"));
+        Collection<ShardingSphereView> views = Collections.singleton(new 
ShardingSphereView("foo_view", "SELECT 1"));
+        SQLFederationSchema actual = new SQLFederationSchema("foo_schema", new 
ShardingSphereSchema("foo_schema", tables, views), databaseType);
+        assertThat(actual.getName(), is("foo_schema"));
+        assertThat(actual.getTableMap().get("foo_table"), 
isA(SQLFederationTable.class));
+        assertThat(actual.getTableMap().get("foo_view"), isA(ViewTable.class));
+    }
+    
+    private ShardingSphereTable createTable(final String tableName) {
+        ShardingSphereColumn column = new ShardingSphereColumn("id", 
Types.INTEGER, true, false, false, true, false, true);
+        return new ShardingSphereTable(tableName, 
Collections.singletonList(column), Collections.emptyList(), 
Collections.emptyList());
+    }
+}
diff --git 
a/kernel/sql-federation/compiler/src/test/java/org/apache/shardingsphere/sqlfederation/compiler/metadata/schema/SQLFederationTableTest.java
 
b/kernel/sql-federation/compiler/src/test/java/org/apache/shardingsphere/sqlfederation/compiler/metadata/schema/SQLFederationTableTest.java
new file mode 100644
index 00000000000..d5385a376c4
--- /dev/null
+++ 
b/kernel/sql-federation/compiler/src/test/java/org/apache/shardingsphere/sqlfederation/compiler/metadata/schema/SQLFederationTableTest.java
@@ -0,0 +1,146 @@
+/*
+ * 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.sqlfederation.compiler.metadata.schema;
+
+import org.apache.calcite.DataContext;
+import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
+import org.apache.calcite.linq4j.Enumerable;
+import org.apache.calcite.linq4j.Enumerator;
+import org.apache.calcite.linq4j.QueryProvider;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.plan.volcano.VolcanoPlanner;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.TableModify;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.shardingsphere.database.connector.core.type.DatabaseType;
+import 
org.apache.shardingsphere.infra.exception.generic.UnsupportedSQLOperationException;
+import 
org.apache.shardingsphere.infra.metadata.database.schema.model.ShardingSphereColumn;
+import 
org.apache.shardingsphere.infra.metadata.database.schema.model.ShardingSphereTable;
+import org.apache.shardingsphere.infra.spi.type.typed.TypedSPILoader;
+import 
org.apache.shardingsphere.sqlfederation.compiler.implementor.ScanImplementor;
+import 
org.apache.shardingsphere.sqlfederation.compiler.implementor.ScanImplementorContext;
+import 
org.apache.shardingsphere.sqlfederation.compiler.implementor.enumerator.EmptyDataRowEnumerator;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Test;
+
+import java.sql.Types;
+import java.util.Collection;
+import java.util.Collections;
+
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.is;
+import static org.hamcrest.Matchers.isA;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+class SQLFederationTableTest {
+    
+    private final DatabaseType databaseType = 
TypedSPILoader.getService(DatabaseType.class, "FIXTURE");
+    
+    private final ShardingSphereTable table = createTable();
+    
+    private ShardingSphereTable createTable() {
+        ShardingSphereColumn column = new ShardingSphereColumn("id", 
Types.INTEGER, true, false, false, true, false, true);
+        Collection<ShardingSphereColumn> columns = 
Collections.singletonList(column);
+        return new ShardingSphereTable("foo_tbl", columns, 
Collections.emptyList(), Collections.emptyList());
+    }
+    
+    @AfterEach
+    void tearDown() {
+        new SQLFederationTable(table, databaseType).clearScanImplementor();
+    }
+    
+    @Test
+    void assertGetRowType() {
+        RelDataTypeFactory typeFactory = new JavaTypeFactoryImpl();
+        SQLFederationTable actual = new SQLFederationTable(table, 
databaseType);
+        RelDataType rowType = actual.getRowType(typeFactory);
+        assertThat(rowType.getFieldList().size(), is(1));
+    }
+    
+    @Test
+    void assertGetExpression() {
+        assertNotNull(new SQLFederationTable(table, 
databaseType).getExpression(mock(), "foo_tbl", Object.class));
+    }
+    
+    @Test
+    void assertGetElementType() {
+        assertThat(new SQLFederationTable(table, 
databaseType).getElementType(), is(Object[].class));
+    }
+    
+    @Test
+    void assertAsQueryableThrowsException() {
+        assertThrows(UnsupportedSQLOperationException.class, () -> new 
SQLFederationTable(table, databaseType).asQueryable(mock(QueryProvider.class), 
mock(SchemaPlus.class), "foo_tbl"));
+    }
+    
+    @Test
+    void assertToRelBuildsLogicalTableScan() {
+        RelOptTable.ToRelContext context = 
mock(RelOptTable.ToRelContext.class);
+        when(context.getCluster()).thenReturn(RelOptCluster.create(new 
VolcanoPlanner(), new RexBuilder(new JavaTypeFactoryImpl())));
+        assertThat(new SQLFederationTable(table, databaseType).toRel(context, 
mock()), isA(RelNode.class));
+    }
+    
+    @Test
+    void assertImplementWithoutScanImplementorReturnsEmptyEnumerator() {
+        Enumerable<Object> actual = new SQLFederationTable(table, 
databaseType).implement(mock(DataContext.class), "SELECT 1", new int[0]);
+        Enumerator<Object> enumerator = actual.enumerator();
+        assertThat(enumerator, isA(EmptyDataRowEnumerator.class));
+        assertFalse(enumerator.moveNext());
+    }
+    
+    @SuppressWarnings("unchecked")
+    @Test
+    void assertImplementUsesScanImplementor() {
+        SQLFederationTable federationTable = new SQLFederationTable(table, 
databaseType);
+        Enumerable<Object> expectedEnumerable = mock(Enumerable.class);
+        ScanImplementor scanImplementor = mock(ScanImplementor.class);
+        when(scanImplementor.implement(any(ShardingSphereTable.class), 
any(ScanImplementorContext.class))).thenReturn(expectedEnumerable);
+        federationTable.setScanImplementor(scanImplementor);
+        Enumerable<Object> actual = 
federationTable.implement(mock(DataContext.class), "SELECT 1", new int[0]);
+        assertThat(actual, is(expectedEnumerable));
+        verify(scanImplementor).implement(any(ShardingSphereTable.class), 
any(ScanImplementorContext.class));
+    }
+    
+    @Test
+    void assertGetModifiableCollectionThrowsException() {
+        assertThrows(UnsupportedOperationException.class, () -> new 
SQLFederationTable(table, databaseType).getModifiableCollection());
+    }
+    
+    @Test
+    void assertToModificationRelBuildsLogicalTableModify() {
+        RelOptCluster relOptCluster = RelOptCluster.create(new 
VolcanoPlanner(), new RexBuilder(new JavaTypeFactoryImpl()));
+        RelNode relNode = mock(RelNode.class);
+        when(relNode.getCluster()).thenReturn(relOptCluster);
+        assertThrows(IllegalArgumentException.class, () -> new 
SQLFederationTable(table, databaseType).toModificationRel(relOptCluster, 
mock(), mock(), relNode,
+                TableModify.Operation.UPDATE, Collections.singletonList("id"), 
Collections.emptyList(), false));
+    }
+    
+    @Test
+    void assertToStringValue() {
+        assertThat(new SQLFederationTable(table, databaseType).toString(), 
is("SQLFederationTable"));
+    }
+}

Reply via email to