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"));
+ }
+}