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

[shardingsphere] branch master updated: Optimize push-down for fitler, and support more operators (#21006)

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

duanzhengqiang 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 5eb85c51628 Optimize push-down for fitler, and support more operators (#21006)
5eb85c51628 is described below

commit 5eb85c51628311364a84eb51a6969a0ceee8642c
Author: boyjoy1127 <10...@users.noreply.github.com>
AuthorDate: Fri Sep 16 11:11:01 2022 +0800

    Optimize push-down for fitler, and support more operators (#21006)
    
    * feat: optimize push-down for fitler, and support more operators for operating integer parameter. These operators are: like, search, and, or, not ,equals, <, <=, >, >=, =,!=,<>.
    
    * style: rename variable 'temp' to 'filterCondition'.
---
 .../executor/TranslatableTableScanExecutor.java    |  26 +--
 .../pom.xml                                        |  21 ++
 .../optimizer/parser/rexnode/Alphabet.g4           |  48 +++++
 .../optimizer/parser/rexnode/Keyword.g4            |  38 ++++
 .../optimizer/parser/rexnode/Literals.g4           |  41 ++++
 .../optimizer/parser/rexnode/ParseRexNode.g4       |  76 +++++++
 .../optimizer/parser/rexnode/Symbol.g4             |  39 ++++
 .../exception/OptimizationSQLRexNodeException.java |  33 +++
 .../translatable/ParseRexNodeVisitorImpl.java      | 234 +++++++++++++++++++++
 .../metadata/translatable/StringToRexNodeUtil.java |  54 +++++
 .../translatable/TranslatableTableScan.java        |  34 +--
 .../optimizer/ShardingSphereOptimizerTest.java     |  18 +-
 12 files changed, 613 insertions(+), 49 deletions(-)

diff --git a/shardingsphere-kernel/shardingsphere-sql-federation/shardingsphere-sql-federation-executor/shardingsphere-sql-federation-executor-core/src/main/java/org/apache/shardingsphere/sqlfederation/executor/TranslatableTableScanExecutor.java b/shardingsphere-kernel/shardingsphere-sql-federation/shardingsphere-sql-federation-executor/shardingsphere-sql-federation-executor-core/src/main/java/org/apache/shardingsphere/sqlfederation/executor/TranslatableTableScanExecutor.java
index c340c2f6dd4..b563e41c84f 100644
--- a/shardingsphere-kernel/shardingsphere-sql-federation/shardingsphere-sql-federation-executor/shardingsphere-sql-federation-executor-core/src/main/java/org/apache/shardingsphere/sqlfederation/executor/TranslatableTableScanExecutor.java
+++ b/shardingsphere-kernel/shardingsphere-sql-federation/shardingsphere-sql-federation-executor/shardingsphere-sql-federation-executor-core/src/main/java/org/apache/shardingsphere/sqlfederation/executor/TranslatableTableScanExecutor.java
@@ -32,13 +32,10 @@ import org.apache.calcite.prepare.CalciteCatalogReader;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.RelFactories;
 import org.apache.calcite.rel.rel2sql.RelToSqlConverter;
-import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeSystem;
 import org.apache.calcite.rex.RexBuilder;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.sql.SqlDialect;
-import org.apache.calcite.sql.fun.SqlStdOperatorTable;
-import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.sql.util.SqlString;
 import org.apache.calcite.tools.RelBuilder;
 import org.apache.shardingsphere.infra.binder.QueryContext;
@@ -78,11 +75,13 @@ import org.apache.shardingsphere.sqlfederation.optimizer.executor.ScanNodeExecut
 import org.apache.shardingsphere.sqlfederation.optimizer.executor.TableScanExecutor;
 import org.apache.shardingsphere.sqlfederation.optimizer.executor.TranslatableScanNodeExecutorContext;
 import org.apache.shardingsphere.sqlfederation.optimizer.metadata.filter.FilterableSchema;
+import org.apache.shardingsphere.sqlfederation.optimizer.metadata.translatable.StringToRexNodeUtil;
 import org.apache.shardingsphere.sqlfederation.optimizer.planner.QueryOptimizePlannerFactory;
 import org.apache.shardingsphere.sqlfederation.row.SQLFederationRowEnumerator;
 import org.apache.shardingsphere.sqlfederation.row.EmptyRowEnumerator;
 import org.apache.shardingsphere.sqlfederation.spi.SQLFederationExecutorContext;
 
+import java.io.IOException;
 import java.sql.Connection;
 import java.sql.PreparedStatement;
 import java.sql.SQLException;
@@ -206,7 +205,7 @@ public final class TranslatableTableScanExecutor implements TableScanExecutor {
         RelOptCluster relOptCluster = RelOptCluster.create(QueryOptimizePlannerFactory.createVolcanoPlanner(), new RexBuilder(new JavaTypeFactoryImpl()));
         RelBuilder builder = RelFactories.LOGICAL_BUILDER.create(relOptCluster, catalogReader).scan(table.getName());
         if (null != scanContext.getFilterValues()) {
-            builder.filter(createFilters(scanContext.getFilterValues(), builder, table.getColumnNames()));
+            builder.filter(createFilters(scanContext.getFilterValues()));
         }
         if (null != scanContext.getProjects()) {
             builder.project(createProjections(scanContext.getProjects(), builder, table.getColumnNames()));
@@ -214,18 +213,21 @@ public final class TranslatableTableScanExecutor implements TableScanExecutor {
         return builder.build();
     }
     
-    private Collection<RexNode> createFilters(final String[] filterValues, final RelBuilder builder, final List<String> columnNames) {
+    private Collection<RexNode> createFilters(final String[] filterValues) {
         Collection<RexNode> result = new LinkedList<>();
         JavaTypeFactory typeFactory = new JavaTypeFactoryImpl(RelDataTypeSystem.DEFAULT);
         RexBuilder rexBuilder = new RexBuilder(typeFactory);
-        for (int i = 0; i < filterValues.length; i++) {
-            if (!Strings.isNullOrEmpty(filterValues[i])) {
-                RelDataType nonNullableInt = typeFactory.createSqlType(SqlTypeName.INTEGER);
-                RexNode n2 = rexBuilder.makeLiteral(Integer.valueOf(filterValues[i]), nonNullableInt, false);
-                RexNode n1 = rexBuilder.makeInputRef(nonNullableInt, i);
-                RexNode tmp = rexBuilder.makeCall(SqlStdOperatorTable.EQUALS, n1, n2);
-                result.add(tmp);
+        for (String each : filterValues) {
+            if (Strings.isNullOrEmpty(each)) {
+                continue;
             }
+            RexNode filterCondition = null;
+            try {
+                filterCondition = StringToRexNodeUtil.buildRexNode(each, rexBuilder);
+            } catch (IOException ignored) {
+                continue;
+            }
+            result.add(filterCondition);
         }
         return result;
     }
diff --git a/shardingsphere-kernel/shardingsphere-sql-federation/shardingsphere-sql-federation-optimizer/pom.xml b/shardingsphere-kernel/shardingsphere-sql-federation/shardingsphere-sql-federation-optimizer/pom.xml
index a4b6578aa96..09a804ccdf6 100644
--- a/shardingsphere-kernel/shardingsphere-sql-federation/shardingsphere-sql-federation-optimizer/pom.xml
+++ b/shardingsphere-kernel/shardingsphere-sql-federation/shardingsphere-sql-federation-optimizer/pom.xml
@@ -86,4 +86,25 @@
             <artifactId>calcite-core</artifactId>
         </dependency>
     </dependencies>
+    <build>
+        <plugins>
+            <plugin>
+                <groupId>org.antlr</groupId>
+                <artifactId>antlr4-maven-plugin</artifactId>
+                <executions>
+                    <execution>
+                        <id>antlr</id>
+                        <goals>
+                            <goal>antlr4</goal>
+                        </goals>
+                        <configuration>
+                            <libDirectory>src/main/antlr4/org/apache/shardingsphere/sqlfederation/optimizer/parser/rexnode</libDirectory>
+                            <listener>false</listener>
+                            <visitor>true</visitor>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
+        </plugins>
+    </build>
 </project>
diff --git a/shardingsphere-kernel/shardingsphere-sql-federation/shardingsphere-sql-federation-optimizer/src/main/antlr4/org/apache/shardingsphere/sqlfederation/optimizer/parser/rexnode/Alphabet.g4 b/shardingsphere-kernel/shardingsphere-sql-federation/shardingsphere-sql-federation-optimizer/src/main/antlr4/org/apache/shardingsphere/sqlfederation/optimizer/parser/rexnode/Alphabet.g4
new file mode 100644
index 00000000000..f7603cc7e63
--- /dev/null
+++ b/shardingsphere-kernel/shardingsphere-sql-federation/shardingsphere-sql-federation-optimizer/src/main/antlr4/org/apache/shardingsphere/sqlfederation/optimizer/parser/rexnode/Alphabet.g4
@@ -0,0 +1,48 @@
+/*
+ * 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.
+ */
+
+lexer grammar Alphabet;
+
+FOR_GENERATOR: 'DO NOT MATCH ANY THING, JUST FOR GENERATOR';
+
+fragment A:   [Aa];
+fragment B:   [Bb];
+fragment C:   [Cc];
+fragment D:   [Dd];
+fragment E:   [Ee];
+fragment F:   [Ff];
+fragment G:   [Gg];
+fragment H:   [Hh];
+fragment I:   [Ii];
+fragment J:   [Jj];
+fragment K:   [Kk];
+fragment L:   [Ll];
+fragment M:   [Mm];
+fragment N:   [Nn];
+fragment O:   [Oo];
+fragment P:   [Pp];
+fragment Q:   [Qq];
+fragment R:   [Rr];
+fragment S:   [Ss];
+fragment T:   [Tt];
+fragment U:   [Uu];
+fragment V:   [Vv];
+fragment W:   [Ww];
+fragment X:   [Xx];
+fragment Y:   [Yy];
+fragment Z:   [Zz];
+fragment UL_: '_';
diff --git a/shardingsphere-kernel/shardingsphere-sql-federation/shardingsphere-sql-federation-optimizer/src/main/antlr4/org/apache/shardingsphere/sqlfederation/optimizer/parser/rexnode/Keyword.g4 b/shardingsphere-kernel/shardingsphere-sql-federation/shardingsphere-sql-federation-optimizer/src/main/antlr4/org/apache/shardingsphere/sqlfederation/optimizer/parser/rexnode/Keyword.g4
new file mode 100644
index 00000000000..d416df1cd79
--- /dev/null
+++ b/shardingsphere-kernel/shardingsphere-sql-federation/shardingsphere-sql-federation-optimizer/src/main/antlr4/org/apache/shardingsphere/sqlfederation/optimizer/parser/rexnode/Keyword.g4
@@ -0,0 +1,38 @@
+/*
+ * 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.
+ */
+
+lexer grammar Keyword;
+
+import Alphabet;
+
+SEARCH: S E A R C H;
+
+LIKE: L I K E;
+
+INTEGER: I N T E G E R;
+
+VARCHAR: V A R C H A R;
+
+CAST: C A S T;
+
+SARG: S A R G;
+
+OR: O R;
+
+AND: A N D;
+
+NOT: N O T;
diff --git a/shardingsphere-kernel/shardingsphere-sql-federation/shardingsphere-sql-federation-optimizer/src/main/antlr4/org/apache/shardingsphere/sqlfederation/optimizer/parser/rexnode/Literals.g4 b/shardingsphere-kernel/shardingsphere-sql-federation/shardingsphere-sql-federation-optimizer/src/main/antlr4/org/apache/shardingsphere/sqlfederation/optimizer/parser/rexnode/Literals.g4
new file mode 100644
index 00000000000..84614199592
--- /dev/null
+++ b/shardingsphere-kernel/shardingsphere-sql-federation/shardingsphere-sql-federation-optimizer/src/main/antlr4/org/apache/shardingsphere/sqlfederation/optimizer/parser/rexnode/Literals.g4
@@ -0,0 +1,41 @@
+/*
+ * 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.
+ */
+
+lexer grammar Literals;
+
+import Alphabet, Symbol;
+
+STRING_ 
+    : (DQ_ ( '\\'. | '""' | ~('"'| '\\') )* DQ_)
+    | (SQ_ ('\\'. | '\'\'' | ~('\'' | '\\'))* SQ_)
+    ;
+
+INTEGER_
+    : INT_
+    ;
+
+NEGETIVE_INFINITY_:  '-∞';
+
+POSITIVE_INFINITY_:  '+∞';
+
+INT_
+    : [0-9]+
+    ;
+
+HEX_
+    : [0-9a-fA-F]
+    ;
diff --git a/shardingsphere-kernel/shardingsphere-sql-federation/shardingsphere-sql-federation-optimizer/src/main/antlr4/org/apache/shardingsphere/sqlfederation/optimizer/parser/rexnode/ParseRexNode.g4 b/shardingsphere-kernel/shardingsphere-sql-federation/shardingsphere-sql-federation-optimizer/src/main/antlr4/org/apache/shardingsphere/sqlfederation/optimizer/parser/rexnode/ParseRexNode.g4
new file mode 100644
index 00000000000..f3d06eba3d8
--- /dev/null
+++ b/shardingsphere-kernel/shardingsphere-sql-federation/shardingsphere-sql-federation-optimizer/src/main/antlr4/org/apache/shardingsphere/sqlfederation/optimizer/parser/rexnode/ParseRexNode.g4
@@ -0,0 +1,76 @@
+/*
+ * 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.
+ */
+
+grammar ParseRexNode;
+
+import Symbol,Keyword,Literals;
+
+expression
+    : op LP_ parameter COMMA_ parameter RP_
+    ;
+
+parameter
+    : input | expression
+    ;
+
+input
+    : inputRef | searchArgs | constant | cast | paramWithType
+    ;
+
+inputRef
+    : DOLLAR_ INTEGER_
+    ;
+
+searchArgs
+    : SARG LBT_ (argRange | argList | argRangeList) RBT_
+    ;
+
+constant
+    : INTEGER_ | STRING_
+    ;
+
+cast
+    : CAST LP_ inputRef  RP_ COLON_ type
+    ;
+
+paramWithType
+    : (STRING_|INTEGER_) COLON_ type
+    ;
+
+op
+    : SEARCH | LIKE | OR | NOT | AND | EQ_ | NEQ_ | GT_ | GTE_ | LT_ | LTE_
+    ;
+
+argRange
+    : LP_ (NEGETIVE_INFINITY_|INTEGER_) RANGE_ (INTEGER_|POSITIVE_INFINITY_) RP_ | LBT_ INTEGER_ RANGE_ INTEGER_ RBT_
+    ;
+
+argList
+    : (LP_|LBT_)? INTEGER_ (COMMA_ INTEGER_)* (RP_|RBT_)?
+    ;
+
+argRangeList
+    : argRange (COMMA_ argRange)*
+    ;
+
+type
+    : INTEGER|VARCHAR
+    ;
+
+WS
+    : [ \t]+ -> skip
+    ;
diff --git a/shardingsphere-kernel/shardingsphere-sql-federation/shardingsphere-sql-federation-optimizer/src/main/antlr4/org/apache/shardingsphere/sqlfederation/optimizer/parser/rexnode/Symbol.g4 b/shardingsphere-kernel/shardingsphere-sql-federation/shardingsphere-sql-federation-optimizer/src/main/antlr4/org/apache/shardingsphere/sqlfederation/optimizer/parser/rexnode/Symbol.g4
new file mode 100644
index 00000000000..a6b5acd85c1
--- /dev/null
+++ b/shardingsphere-kernel/shardingsphere-sql-federation/shardingsphere-sql-federation-optimizer/src/main/antlr4/org/apache/shardingsphere/sqlfederation/optimizer/parser/rexnode/Symbol.g4
@@ -0,0 +1,39 @@
+/*
+ * 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.
+ */
+
+lexer grammar Symbol;
+
+EQ_:                 '=';
+NEQ_:                '<>' | '!=' | '^=';
+GT_:                 '>';
+GTE_:                '>=';
+LT_:                 '<';
+LTE_:                '<=';
+LP_:                 '(';
+RP_:                 ')';
+LBE_:                '{';
+RBE_:                '}';
+LBT_:                '[';
+RBT_:                ']';
+COMMA_:              ',';
+DQ_:                 '"';
+SQ_ :                '\'';
+DOLLAR_:             '$';
+COLON_:              ':';
+RANGE_:              '..';
+NEGETIVE_INFINITY_:  '-∞';
+POSITIVE_INFINITY_:  '+∞';
diff --git a/shardingsphere-kernel/shardingsphere-sql-federation/shardingsphere-sql-federation-optimizer/src/main/java/org/apache/shardingsphere/sqlfederation/optimizer/converter/exception/OptimizationSQLRexNodeException.java b/shardingsphere-kernel/shardingsphere-sql-federation/shardingsphere-sql-federation-optimizer/src/main/java/org/apache/shardingsphere/sqlfederation/optimizer/converter/exception/OptimizationSQLRexNodeException.java
new file mode 100644
index 00000000000..6eee037f8b4
--- /dev/null
+++ b/shardingsphere-kernel/shardingsphere-sql-federation/shardingsphere-sql-federation-optimizer/src/main/java/org/apache/shardingsphere/sqlfederation/optimizer/converter/exception/OptimizationSQLRexNodeException.java
@@ -0,0 +1,33 @@
+/*
+ * 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.optimizer.converter.exception;
+
+import org.apache.shardingsphere.infra.exception.MetaDataSQLException;
+import org.apache.shardingsphere.infra.util.exception.external.sql.sqlstate.XOpenSQLState;
+
+/**
+ * Optimization SQL rex node exception.
+ */
+public final class OptimizationSQLRexNodeException extends MetaDataSQLException {
+    
+    private static final long serialVersionUID = -5486229929620713963L;
+    
+    public OptimizationSQLRexNodeException(final String reason) {
+        super(XOpenSQLState.SYNTAX_ERROR, 4, "Unsupported SQL condition `%s`", reason);
+    }
+}
diff --git a/shardingsphere-kernel/shardingsphere-sql-federation/shardingsphere-sql-federation-optimizer/src/main/java/org/apache/shardingsphere/sqlfederation/optimizer/metadata/translatable/ParseRexNodeVisitorImpl.java b/shardingsphere-kernel/shardingsphere-sql-federation/shardingsphere-sql-federation-optimizer/src/main/java/org/apache/shardingsphere/sqlfederation/optimizer/metadata/translatable/ParseRexNodeVisitorImpl.java
new file mode 100644
index 00000000000..c1c49ef091f
--- /dev/null
+++ b/shardingsphere-kernel/shardingsphere-sql-federation/shardingsphere-sql-federation-optimizer/src/main/java/org/apache/shardingsphere/sqlfederation/optimizer/metadata/translatable/ParseRexNodeVisitorImpl.java
@@ -0,0 +1,234 @@
+/*
+ * 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.optimizer.metadata.translatable;
+
+import com.google.common.collect.BoundType;
+import com.google.common.collect.ImmutableRangeSet;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeSet;
+import com.google.common.collect.TreeRangeSet;
+import lombok.AllArgsConstructor;
+import org.antlr.v4.runtime.tree.TerminalNode;
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexUnknownAs;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.util.Sarg;
+import org.apache.shardingsphere.sqlfederation.optimizer.converter.exception.OptimizationSQLRexNodeException;
+import org.apache.shardingsphere.sqlfederation.optimizer.parser.rexnode.ParseRexNodeBaseVisitor;
+import org.apache.shardingsphere.sqlfederation.optimizer.parser.rexnode.ParseRexNodeParser.ArgListContext;
+import org.apache.shardingsphere.sqlfederation.optimizer.parser.rexnode.ParseRexNodeParser.ArgRangeContext;
+import org.apache.shardingsphere.sqlfederation.optimizer.parser.rexnode.ParseRexNodeParser.ArgRangeListContext;
+import org.apache.shardingsphere.sqlfederation.optimizer.parser.rexnode.ParseRexNodeParser.CastContext;
+import org.apache.shardingsphere.sqlfederation.optimizer.parser.rexnode.ParseRexNodeParser.ConstantContext;
+import org.apache.shardingsphere.sqlfederation.optimizer.parser.rexnode.ParseRexNodeParser.ExpressionContext;
+import org.apache.shardingsphere.sqlfederation.optimizer.parser.rexnode.ParseRexNodeParser.InputContext;
+import org.apache.shardingsphere.sqlfederation.optimizer.parser.rexnode.ParseRexNodeParser.InputRefContext;
+import org.apache.shardingsphere.sqlfederation.optimizer.parser.rexnode.ParseRexNodeParser.OpContext;
+import org.apache.shardingsphere.sqlfederation.optimizer.parser.rexnode.ParseRexNodeParser.ParamWithTypeContext;
+import org.apache.shardingsphere.sqlfederation.optimizer.parser.rexnode.ParseRexNodeParser.ParameterContext;
+import org.apache.shardingsphere.sqlfederation.optimizer.parser.rexnode.ParseRexNodeParser.SearchArgsContext;
+import org.apache.shardingsphere.sqlfederation.optimizer.parser.rexnode.ParseRexNodeParser.TypeContext;
+
+import java.math.BigDecimal;
+import java.util.LinkedList;
+import java.util.List;
+
+@AllArgsConstructor
+public final class ParseRexNodeVisitorImpl extends ParseRexNodeBaseVisitor<RexNode> {
+    
+    private RexBuilder rexBuilder;
+    
+    private JavaTypeFactory typeFactory;
+    
+    @Override
+    public RexNode visitExpression(final ExpressionContext ctx) {
+        SqlOperator operator = getOp(ctx.op());
+        RexNode firstNode = visitParameter(ctx.parameter(0));
+        RexNode secondNode = visitParameter(ctx.parameter(1));
+        return rexBuilder.makeCall(operator, firstNode, secondNode);
+    }
+    
+    private SqlOperator getOp(final OpContext ctx) {
+        if (null != ctx.LIKE()) {
+            return SqlStdOperatorTable.LIKE;
+        } else if (null != ctx.SEARCH()) {
+            return SqlStdOperatorTable.SEARCH;
+        } else if (null != ctx.AND()) {
+            return SqlStdOperatorTable.AND;
+        } else if (null != ctx.OR()) {
+            return SqlStdOperatorTable.OR;
+        } else if (null != ctx.NOT()) {
+            return SqlStdOperatorTable.NOT;
+        } else if (null != ctx.EQ_()) {
+            return SqlStdOperatorTable.EQUALS;
+        } else if (null != ctx.LT_()) {
+            return SqlStdOperatorTable.LESS_THAN;
+        } else if (null != ctx.LTE_()) {
+            return SqlStdOperatorTable.LESS_THAN_OR_EQUAL;
+        } else if (null != ctx.GT_()) {
+            return SqlStdOperatorTable.GREATER_THAN;
+        } else if (null != ctx.GTE_()) {
+            return SqlStdOperatorTable.GREATER_THAN_OR_EQUAL;
+        } else if (null != ctx.NEQ_()) {
+            return SqlStdOperatorTable.NOT_EQUALS;
+        }
+        throw new OptimizationSQLRexNodeException(ctx.getText());
+    }
+    
+    @Override
+    public RexNode visitParameter(final ParameterContext ctx) {
+        if (null != ctx.expression()) {
+            return visitExpression(ctx.expression());
+        } else if (null != ctx.input()) {
+            return visitInput(ctx.input());
+        }
+        throw new OptimizationSQLRexNodeException(ctx.getText());
+    }
+    
+    @Override
+    public RexNode visitInput(final InputContext ctx) {
+        if (null != ctx.inputRef()) {
+            return visitInputRef(ctx.inputRef());
+        } else if (null != ctx.searchArgs()) {
+            return visitSearchArgs(ctx.searchArgs());
+        } else if (null != ctx.constant()) {
+            return visitConstant(ctx.constant());
+        } else if (null != ctx.cast()) {
+            return visitCast(ctx.cast());
+        } else if (null != ctx.paramWithType()) {
+            return visitParamWithType(ctx.paramWithType());
+        }
+        throw new OptimizationSQLRexNodeException(ctx.getText());
+    }
+    
+    @Override
+    public RexNode visitInputRef(final InputRefContext ctx) {
+        Integer index = Integer.valueOf(ctx.INTEGER_().getText());
+        RelDataType nonNullableInt = typeFactory.createSqlType(SqlTypeName.INTEGER);
+        return rexBuilder.makeInputRef(nonNullableInt, index);
+    }
+    
+    @Override
+    public RexNode visitSearchArgs(final SearchArgsContext ctx) {
+        Sarg<BigDecimal> sarg = null;
+        if (null != ctx.argList()) {
+            sarg = getArgList(ctx.argList());
+        } else if (null != ctx.argRange()) {
+            sarg = getArgRange(ctx.argRange());
+        } else if (null != ctx.argRangeList()) {
+            sarg = getArgRangeList(ctx.argRangeList());
+        } else {
+            throw new OptimizationSQLRexNodeException(ctx.getText());
+        }
+        RelDataType sargType = typeFactory.createSqlType(SqlTypeName.DECIMAL);
+        return rexBuilder.makeSearchArgumentLiteral(sarg, sargType);
+    }
+    
+    @Override
+    public RexNode visitConstant(final ConstantContext ctx) {
+        if (null != ctx.INTEGER_()) {
+            Integer number = Integer.valueOf(ctx.INTEGER_().getText());
+            RelDataType nonNullableInt = typeFactory.createSqlType(SqlTypeName.INTEGER);
+            return rexBuilder.makeLiteral(number, nonNullableInt, false);
+        } else if (null != ctx.STRING_()) {
+            RelDataType varchar = typeFactory.createSqlType(SqlTypeName.VARCHAR);
+            return rexBuilder.makeLiteral(ctx.STRING_().getText(), varchar, false);
+        }
+        throw new OptimizationSQLRexNodeException(ctx.getText());
+    }
+    
+    @Override
+    public RexNode visitCast(final CastContext ctx) {
+        RexNode inputRef = visitInputRef(ctx.inputRef());
+        RelDataType type = getType(ctx.type());
+        return rexBuilder.makeCast(type, inputRef);
+    }
+    
+    @Override
+    public RexNode visitParamWithType(final ParamWithTypeContext ctx) {
+        RelDataType type = getType(ctx.type());
+        if (null != ctx.INTEGER_()) {
+            return rexBuilder.makeLiteral(Integer.valueOf(ctx.INTEGER_().getText()), type);
+        } else {
+            return rexBuilder.makeLiteral(ctx.STRING_().getText(), type);
+        }
+    }
+    
+    private Sarg<BigDecimal> getArgRange(final ArgRangeContext ctx) {
+        BigDecimal lowerValue = BigDecimal.valueOf(Long.parseLong(ctx.INTEGER_(0).getText()));
+        BigDecimal upperValue = BigDecimal.valueOf(Long.parseLong(ctx.INTEGER_(1).getText()));
+        Range.range(lowerValue, BoundType.OPEN, upperValue, BoundType.OPEN);
+        if (null != ctx.LP_()) {
+            return Sarg.of(RexUnknownAs.UNKNOWN, ImmutableRangeSet.of(Range.range(lowerValue, BoundType.OPEN, upperValue, BoundType.OPEN)));
+        } else {
+            return Sarg.of(RexUnknownAs.UNKNOWN, ImmutableRangeSet.of(Range.range(lowerValue, BoundType.CLOSED, upperValue, BoundType.CLOSED)));
+        }
+    }
+    
+    private Sarg<BigDecimal> getArgList(final ArgListContext ctx) {
+        final RangeSet<BigDecimal> rangeSet = TreeRangeSet.create();
+        for (TerminalNode each : ctx.INTEGER_()) {
+            BigDecimal value = BigDecimal.valueOf(Long.parseLong(each.getText()));
+            rangeSet.add(Range.singleton(value));
+        }
+        return Sarg.of(RexUnknownAs.UNKNOWN, rangeSet);
+    }
+    
+    private Sarg<BigDecimal> getArgRangeList(final ArgRangeListContext ctx) {
+        List<Range<BigDecimal>> rangeList = new LinkedList<>();
+        for (ArgRangeContext each : ctx.argRange()) {
+            BigDecimal lowerValue = BigDecimal.valueOf(Long.MIN_VALUE);
+            BigDecimal upperValue = BigDecimal.valueOf(Long.MAX_VALUE);
+            if ((null != each.NEGETIVE_INFINITY_()) && (null != each.INTEGER_(0))) {
+                String upper = each.INTEGER_(0).getText();
+                upperValue = BigDecimal.valueOf(Long.parseLong(upper));
+            }
+            if ((null != each.POSITIVE_INFINITY_()) && (null != each.INTEGER_(0))) {
+                String lower = each.INTEGER_(0).getText();
+                lowerValue = BigDecimal.valueOf(Long.parseLong(lower));
+            }
+            if ((null == each.NEGETIVE_INFINITY_()) && (null == each.POSITIVE_INFINITY_())) {
+                String lower = each.INTEGER_(0).getText();
+                String upper = each.INTEGER_(1).getText();
+                lowerValue = BigDecimal.valueOf(Long.parseLong(lower));
+                upperValue = BigDecimal.valueOf(Long.parseLong(upper));
+            }
+            if (null != each.LP_()) {
+                Range<BigDecimal> range = Range.range(lowerValue, BoundType.OPEN, upperValue, BoundType.OPEN);
+                rangeList.add(range);
+            } else {
+                Range<BigDecimal> range = Range.range(lowerValue, BoundType.CLOSED, upperValue, BoundType.CLOSED);
+                rangeList.add(range);
+            }
+        }
+        return Sarg.of(RexUnknownAs.UNKNOWN, ImmutableRangeSet.copyOf(rangeList));
+    }
+    
+    private RelDataType getType(final TypeContext ctx) {
+        if (null != ctx.INTEGER()) {
+            return typeFactory.createSqlType(SqlTypeName.INTEGER);
+        } else {
+            return typeFactory.createSqlType(SqlTypeName.VARCHAR);
+        }
+    }
+}
diff --git a/shardingsphere-kernel/shardingsphere-sql-federation/shardingsphere-sql-federation-optimizer/src/main/java/org/apache/shardingsphere/sqlfederation/optimizer/metadata/translatable/StringToRexNodeUtil.java b/shardingsphere-kernel/shardingsphere-sql-federation/shardingsphere-sql-federation-optimizer/src/main/java/org/apache/shardingsphere/sqlfederation/optimizer/metadata/translatable/StringToRexNodeUtil.java
new file mode 100644
index 00000000000..001691efc72
--- /dev/null
+++ b/shardingsphere-kernel/shardingsphere-sql-federation/shardingsphere-sql-federation-optimizer/src/main/java/org/apache/shardingsphere/sqlfederation/optimizer/metadata/translatable/StringToRexNodeUtil.java
@@ -0,0 +1,54 @@
+/*
+ * 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.optimizer.metadata.translatable;
+
+import org.antlr.v4.runtime.CharStream;
+import org.antlr.v4.runtime.CharStreams;
+import org.antlr.v4.runtime.CommonTokenStream;
+import org.antlr.v4.runtime.tree.ParseTree;
+import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
+import org.apache.calcite.rel.type.RelDataTypeSystem;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexNode;
+import org.apache.shardingsphere.sqlfederation.optimizer.parser.rexnode.ParseRexNodeLexer;
+import org.apache.shardingsphere.sqlfederation.optimizer.parser.rexnode.ParseRexNodeParser;
+
+import java.io.IOException;
+
+/**
+ * Utility for parsing string and generate rex node.
+ */
+public final class StringToRexNodeUtil {
+    
+    /**
+     * Parse string and generate rex node.
+     * @param filterValue filter condition
+     * @param rexBuilder used to build rex node
+     * @return rex node
+     * @throws IOException io exception
+     */
+    public static RexNode buildRexNode(final String filterValue, final RexBuilder rexBuilder) throws IOException {
+        CharStream input = CharStreams.fromString(filterValue);
+        ParseRexNodeLexer lexer = new ParseRexNodeLexer(input);
+        CommonTokenStream tokens = new CommonTokenStream(lexer);
+        ParseRexNodeParser parser = new ParseRexNodeParser(tokens);
+        ParseTree tree = parser.expression();
+        ParseRexNodeVisitorImpl visitor = new ParseRexNodeVisitorImpl(rexBuilder, new JavaTypeFactoryImpl(RelDataTypeSystem.DEFAULT));
+        return visitor.visit(tree);
+    }
+}
diff --git a/shardingsphere-kernel/shardingsphere-sql-federation/shardingsphere-sql-federation-optimizer/src/main/java/org/apache/shardingsphere/sqlfederation/optimizer/metadata/translatable/TranslatableTableScan.java b/shardingsphere-kernel/shardingsphere-sql-federation/shardingsphere-sql-federation-optimizer/src/main/java/org/apache/shardingsphere/sqlfederation/optimizer/metadata/translatable/TranslatableTableScan.java
index be759e90bec..2443b5fc789 100644
--- a/shardingsphere-kernel/shardingsphere-sql-federation/shardingsphere-sql-federation-optimizer/src/main/java/org/apache/shardingsphere/sqlfederation/optimizer/metadata/translatable/TranslatableTableScan.java
+++ b/shardingsphere-kernel/shardingsphere-sql-federation/shardingsphere-sql-federation-optimizer/src/main/java/org/apache/shardingsphere/sqlfederation/optimizer/metadata/translatable/TranslatableTableScan.java
@@ -39,12 +39,7 @@ import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.rel.type.RelDataTypeField;
-import org.apache.calcite.rex.RexCall;
-import org.apache.calcite.rex.RexInputRef;
-import org.apache.calcite.rex.RexLiteral;
 import org.apache.calcite.rex.RexNode;
-import org.apache.calcite.sql.SqlKind;
-import org.codehaus.groovy.runtime.InvokerHelper;
 
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -166,32 +161,15 @@ public class TranslatableTableScan extends TableScan implements EnumerableRel {
             return implementor.result(physType, Blocks.toBlock(Expressions.call(table.getExpression(FederationTranslatableTable.class),
                     "projectAndFilter", implementor.getRootExpression(), Expressions.constant(filterValues), Expressions.constant(fields))));
         }
-        return implementor.result(physType, Blocks.toBlock(
-                Expressions.call(table.getExpression(FederationTranslatableTable.class), "project", implementor.getRootExpression(), Expressions.constant(fields))));
+        return implementor.result(physType, Blocks.toBlock(Expressions.call(table.getExpression(FederationTranslatableTable.class),
+                "project", implementor.getRootExpression(), Expressions.constant(fields))));
     }
     
-    private boolean addFilter(final List<RexNode> filters, final String[] filterValues) {
+    private void addFilter(final List<RexNode> filters, final String[] filterValues) {
+        int index = 0;
         for (RexNode filter : filters) {
-            if (filter.isA(SqlKind.AND)) {
-                ((RexCall) filter).getOperands().forEach(subFilter -> addFilter(InvokerHelper.asList(subFilter), filterValues));
-            } else if (filter.isA(SqlKind.EQUALS)) {
-                RexCall call = (RexCall) filter;
-                RexNode left = call.getOperands().get(0);
-                if (left.isA(SqlKind.CAST)) {
-                    left = ((RexCall) left).operands.get(0);
-                }
-                RexNode right = call.getOperands().get(1);
-                if (!(left instanceof RexInputRef && right instanceof RexLiteral)) {
-                    continue;
-                }
-                int index = ((RexInputRef) left).getIndex();
-                if (null == filterValues[index]) {
-                    filterValues[index] = ((RexLiteral) right).getValue2().toString();
-                    return true;
-                }
-            }
+            filterValues[index] = filter.toString();
+            index++;
         }
-        return false;
     }
-    
 }
diff --git a/shardingsphere-kernel/shardingsphere-sql-federation/shardingsphere-sql-federation-optimizer/src/test/java/org/apache/shardingsphere/sqlfederation/optimizer/ShardingSphereOptimizerTest.java b/shardingsphere-kernel/shardingsphere-sql-federation/shardingsphere-sql-federation-optimizer/src/test/java/org/apache/shardingsphere/sqlfederation/optimizer/ShardingSphereOptimizerTest.java
index eb7cf2e192b..ffb59339a9e 100644
--- a/shardingsphere-kernel/shardingsphere-sql-federation/shardingsphere-sql-federation-optimizer/src/test/java/org/apache/shardingsphere/sqlfederation/optimizer/ShardingSphereOptimizerTest.java
+++ b/shardingsphere-kernel/shardingsphere-sql-federation/shardingsphere-sql-federation-optimizer/src/test/java/org/apache/shardingsphere/sqlfederation/optimizer/ShardingSphereOptimizerTest.java
@@ -135,7 +135,7 @@ public final class ShardingSphereOptimizerTest {
                 + "    EnumerableCalc(expr#0..1=[{inputs}], expr#2=[CAST($t1):VARCHAR], proj#0..2=[{exprs}])" + LINE_SEPARATOR
                 + "      TranslatableTableScan(table=[[federate_jdbc, t_order_federate]], fields=[[0, 1]])" + LINE_SEPARATOR
                 + "    EnumerableCalc(expr#0=[{inputs}], expr#1=[CAST($t0):VARCHAR], proj#0..1=[{exprs}])" + LINE_SEPARATOR
-                + "      TranslatableTableScan(table=[[federate_jdbc, t_user_info]], fields=[[0]], filters=[[13, null]])" + LINE_SEPARATOR;
+                + "      TranslatableTableScan(table=[[federate_jdbc, t_user_info]], fields=[[0]], filters=[[=(CAST($0):INTEGER, 13), null]])" + LINE_SEPARATOR;
         assertThat(actual, is(expected));
     }
     
@@ -144,7 +144,7 @@ public final class ShardingSphereOptimizerTest {
         ShardingSphereSQLParserEngine sqlParserEngine = sqlParserRule.getSQLParserEngine(DatabaseTypeEngine.getTrunkDatabaseTypeName(new H2DatabaseType()));
         SQLStatement sqlStatement = sqlParserEngine.parse(SELECT_WHERE_ALL_FIELDS, false);
         String actual = optimizer.optimize(sqlStatement).explain();
-        String expected = "TranslatableTableScan(table=[[federate_jdbc, t_user_info]], fields=[[0, 1]], filters=[[12, null]])" + LINE_SEPARATOR;
+        String expected = "TranslatableTableScan(table=[[federate_jdbc, t_user_info]], fields=[[0, 1]], filters=[[=(CAST($0):INTEGER, 12), null]])" + LINE_SEPARATOR;
         assertThat(actual, is(expected));
     }
     
@@ -153,7 +153,7 @@ public final class ShardingSphereOptimizerTest {
         ShardingSphereSQLParserEngine sqlParserEngine = sqlParserRule.getSQLParserEngine(DatabaseTypeEngine.getTrunkDatabaseTypeName(new H2DatabaseType()));
         SQLStatement sqlStatement = sqlParserEngine.parse(SELECT_WHERE_SINGLE_FIELD, false);
         String actual = optimizer.optimize(sqlStatement).explain();
-        String expected = "TranslatableTableScan(table=[[federate_jdbc, t_user_info]], fields=[[0]], filters=[[12]])" + LINE_SEPARATOR;
+        String expected = "TranslatableTableScan(table=[[federate_jdbc, t_user_info]], fields=[[0]], filters=[[=(CAST($0):INTEGER, 12)]])" + LINE_SEPARATOR;
         assertThat(actual, is(expected));
     }
     
@@ -195,7 +195,7 @@ public final class ShardingSphereOptimizerTest {
                 + "    EnumerableCalc(expr#0..1=[{inputs}], expr#2=[CAST($t1):VARCHAR], proj#0..2=[{exprs}])" + LINE_SEPARATOR
                 + "      TranslatableTableScan(table=[[federate_jdbc, t_order_federate]], fields=[[0, 1]])" + LINE_SEPARATOR
                 + "    EnumerableCalc(expr#0=[{inputs}], expr#1=[CAST($t0):VARCHAR], proj#0..1=[{exprs}])" + LINE_SEPARATOR
-                + "      TranslatableTableScan(table=[[federate_jdbc, t_user_info]], fields=[[0]], filters=[[13, null]])" + LINE_SEPARATOR;
+                + "      TranslatableTableScan(table=[[federate_jdbc, t_user_info]], fields=[[0]], filters=[[=(CAST($0):INTEGER, 13), null]])" + LINE_SEPARATOR;
         assertThat(actual, is(expected));
     }
     
@@ -204,7 +204,7 @@ public final class ShardingSphereOptimizerTest {
         ShardingSphereSQLParserEngine sqlParserEngine = sqlParserRule.getSQLParserEngine(DatabaseTypeEngine.getTrunkDatabaseTypeName(new H2DatabaseType()));
         SQLStatement sqlStatement = sqlParserEngine.parse(SELECT_SUBQUERY_FROM, false);
         String actual = optimizer.optimize(sqlStatement).explain();
-        String expected = "TranslatableTableScan(table=[[federate_jdbc, t_user_info]], fields=[[0, 1]], filters=[[null, null]])" + LINE_SEPARATOR;
+        String expected = "TranslatableTableScan(table=[[federate_jdbc, t_user_info]], fields=[[0, 1]], filters=[[>(CAST($0):INTEGER, 1), null]])" + LINE_SEPARATOR;
         assertThat(actual, is(expected));
     }
     
@@ -218,7 +218,7 @@ public final class ShardingSphereOptimizerTest {
                 + "    TranslatableTableScan(table=[[federate_jdbc, t_order_federate]], fields=[[0, 1, 2]])" + LINE_SEPARATOR
                 + "    EnumerableAggregate(group=[{}], agg#0=[MIN($0)])" + LINE_SEPARATOR
                 + "      EnumerableCalc(expr#0..1=[{inputs}], expr#2=[true], $f0=[$t2])" + LINE_SEPARATOR
-                + "        TranslatableTableScan(table=[[federate_jdbc, t_user_info]], fields=[[0, 1]], filters=[[null, null]])" + LINE_SEPARATOR;
+                + "        TranslatableTableScan(table=[[federate_jdbc, t_user_info]], fields=[[0, 1]], filters=[[=(CAST($cor0.user_id):VARCHAR, CAST($0):VARCHAR), null]])" + LINE_SEPARATOR;
         assertThat(actual, is(expected));
     }
     
@@ -246,9 +246,9 @@ public final class ShardingSphereOptimizerTest {
                 + "      EnumerableNestedLoopJoin(condition=[>=($1, $2)], joinType=[inner])" + LINE_SEPARATOR
                 + "        TranslatableTableScan(table=[[federate_jdbc, t_order_federate]], fields=[[0, 1]])" + LINE_SEPARATOR
                 + "        EnumerableAggregate(group=[{}], agg#0=[SINGLE_VALUE($0)])" + LINE_SEPARATOR
-                + "          TranslatableTableScan(table=[[federate_jdbc, t_user_info]], fields=[[0]], filters=[[1]])" + LINE_SEPARATOR
+                + "          TranslatableTableScan(table=[[federate_jdbc, t_user_info]], fields=[[0]], filters=[[=(CAST($0):INTEGER, 1)]])" + LINE_SEPARATOR
                 + "    EnumerableAggregate(group=[{}], agg#0=[SINGLE_VALUE($0)])" + LINE_SEPARATOR
-                + "      TranslatableTableScan(table=[[federate_jdbc, t_user_info]], fields=[[0]], filters=[[3]])" + LINE_SEPARATOR;
+                + "      TranslatableTableScan(table=[[federate_jdbc, t_user_info]], fields=[[0]], filters=[[=(CAST($0):INTEGER, 3)]])" + LINE_SEPARATOR;
         assertThat(actual, is(expected));
     }
     
@@ -260,7 +260,7 @@ public final class ShardingSphereOptimizerTest {
         String expected = "EnumerableUnion(all=[false])" + LINE_SEPARATOR
                 + "  TranslatableTableScan(table=[[federate_jdbc, t_order_federate]], fields=[[0, 1]])" + LINE_SEPARATOR
                 + "  EnumerableCalc(expr#0=[{inputs}], expr#1=['1':VARCHAR], EXPR$0=[$t1], user_id=[$t0])" + LINE_SEPARATOR
-                + "    TranslatableTableScan(table=[[federate_jdbc, t_user_info]], fields=[[0]], filters=[[null, before]])" + LINE_SEPARATOR;
+                + "    TranslatableTableScan(table=[[federate_jdbc, t_user_info]], fields=[[0]], filters=[[=(CAST($1):VARCHAR, 'before'), null]])" + LINE_SEPARATOR;
         assertThat(actual, is(expected));
     }
 }