You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by GitBox <gi...@apache.org> on 2021/06/22 04:31:21 UTC

[GitHub] [shardingsphere] tristaZero commented on a change in pull request #10889: Convert ss ast to calcite ast

tristaZero commented on a change in pull request #10889:
URL: https://github.com/apache/shardingsphere/pull/10889#discussion_r655861368



##########
File path: shardingsphere-infra/shardingsphere-infra-optimize/src/main/java/org/apache/shardingsphere/infra/optimize/core/convert/converter/impl/ExpressionSqlNodeConverter.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.infra.optimize.core.convert.converter.impl;
+
+import org.apache.calcite.sql.SqlNode;
+import org.apache.shardingsphere.infra.optimize.core.convert.converter.SqlNodeConverter;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.column.ColumnSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.BinaryOperationExpression;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.ExpressionSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.ListExpression;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.complex.CommonExpressionSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.simple.LiteralExpressionSegment;
+
+import java.util.Optional;
+
+/**
+ * Expression converter entry.
+ */
+public final class ExpressionSqlNodeConverter implements SqlNodeConverter<ExpressionSegment, SqlNode> {
+    @Override
+    public Optional<SqlNode> convert(ExpressionSegment expression) {
+        if (expression == null) {
+            return null;

Review comment:
       return Optional.empty()?

##########
File path: shardingsphere-infra/shardingsphere-infra-optimize/src/main/java/org/apache/shardingsphere/infra/optimize/core/convert/converter/impl/OrderBySqlNodeConverter.java
##########
@@ -0,0 +1,42 @@
+/*
+ * 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.infra.optimize.core.convert.converter.impl;
+
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlNodeList;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.shardingsphere.infra.optimize.core.convert.converter.SqlNodeConverter;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.order.OrderBySegment;
+
+import java.util.List;
+import java.util.Optional;
+
+/**
+ * Order by converter. 
+ */
+public final class OrderBySqlNodeConverter extends AbstractOrderBySqlNodeConverter implements SqlNodeConverter<OrderBySegment, SqlNodeList> {
+    

Review comment:
       Please keep the same check style for other classes.

##########
File path: shardingsphere-infra/shardingsphere-infra-optimize/src/main/java/org/apache/shardingsphere/infra/optimize/core/convert/converter/impl/ExpressionSqlNodeConverter.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.infra.optimize.core.convert.converter.impl;
+
+import org.apache.calcite.sql.SqlNode;
+import org.apache.shardingsphere.infra.optimize.core.convert.converter.SqlNodeConverter;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.column.ColumnSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.BinaryOperationExpression;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.ExpressionSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.ListExpression;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.complex.CommonExpressionSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.simple.LiteralExpressionSegment;
+
+import java.util.Optional;
+
+/**
+ * Expression converter entry.
+ */
+public final class ExpressionSqlNodeConverter implements SqlNodeConverter<ExpressionSegment, SqlNode> {
+    @Override

Review comment:
       A new line above this line is expected to add.

##########
File path: shardingsphere-infra/shardingsphere-infra-optimize/src/test/java/org/apache/shardingsphere/infra/optimize/core/convert/SqlNodeConverterTest.java
##########
@@ -0,0 +1,139 @@
+/*
+ * 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.infra.optimize.core.convert;
+
+import org.apache.calcite.sql.SqlJoin;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlSelect;
+import org.apache.shardingsphere.infra.database.type.DatabaseTypeRegistry;
+import org.apache.shardingsphere.infra.database.type.dialect.MySQLDatabaseType;
+import org.apache.shardingsphere.infra.parser.ShardingSphereSQLParserEngine;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.Optional;
+
+import static org.hamcrest.CoreMatchers.instanceOf;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * testcase of converting shardingshphere ast to calcite ast.
+ *
+ * <p>after converting phrase finished, the next phrase is comparing  the converted result with the
+ * result of calcite parser.
+ * </p>
+ */
+public final class SqlNodeConverterTest {
+
+    private ShardingSphereSQLParserEngine sqlStatementParserEngine;
+
+    @Before
+    public void init() {
+        sqlStatementParserEngine = new ShardingSphereSQLParserEngine(DatabaseTypeRegistry.getTrunkDatabaseTypeName(
+                new MySQLDatabaseType()));
+    }
+
+    @Test
+    public void testConvertSimpleSelect() {
+        String sql = "select order_id, user_id from t_order";
+        SQLStatement sqlStatement = sqlStatementParserEngine.parse(sql, false);
+        Optional<SqlNode> optional = SqlNodeConverter.convert(sqlStatement);
+        assertTrue(optional.isPresent());
+        Assert.assertThat(optional.get(), instanceOf(SqlSelect.class));
+        SqlSelect sqlSelect = (SqlSelect) optional.get();
+        Assert.assertEquals(2, sqlSelect.getSelectList().size());
+        Assert.assertNull(sqlSelect.getWhere());
+        /* 
+         TODO compare ast from calcite parser and ast converted from ss ast if possible
+        SqlParser parser = SqlParser.create(sql);
+        SqlNode calciteSqlNode = parser.parseQuery();
+        Assert.assertNotNull(calciteSqlNode);
+        */
+    }
+
+    @Test
+    public void testConvertSimpleSelectFilter() {
+        String sql = "select order_id, user_id from t_order where order_id = 10";
+        SQLStatement sqlStatement = sqlStatementParserEngine.parse(sql, false);
+

Review comment:
       Could you remove these blank lines? If you really want to make your coding descriptive, please consider creating functions for them.

##########
File path: shardingsphere-infra/shardingsphere-infra-optimize/src/main/java/org/apache/shardingsphere/infra/optimize/core/convert/converter/impl/JoinTableSqlNodeConverter.java
##########
@@ -0,0 +1,75 @@
+/*
+ * 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.infra.optimize.core.convert.converter.impl;
+
+import org.apache.calcite.sql.JoinConditionType;
+import org.apache.calcite.sql.JoinType;
+import org.apache.calcite.sql.SqlJoin;
+import org.apache.calcite.sql.SqlLiteral;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.shardingsphere.infra.optimize.core.convert.converter.SqlNodeConverter;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.expr.ExpressionSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.generic.table.JoinTableSegment;
+
+import java.util.Optional;
+
+/**
+ * Join converter.
+ */
+public final class JoinTableSqlNodeConverter implements SqlNodeConverter<JoinTableSegment, SqlNode> {
+
+    private static final String JOIN_TYPE_INNER = "INNER";
+
+    private static final String JOIN_TYPE_LEFT = "LEFT";
+
+    private static final String JOIN_TYPE_RIGHT = "RIGHT";
+
+    private static final String JOIN_TYPE_FULL = "FULL";
+    
+    @Override
+    public Optional<SqlNode> convert(JoinTableSegment join) {
+        String joinType = join.getJoinType();
+        SqlNode left = new TableSqlNodeConverter().convert(join.getLeft()).get();
+        SqlNode right = new TableSqlNodeConverter().convert(join.getRight()).get();
+        ExpressionSegment expressionSegment = join.getCondition();
+        Optional<SqlNode> condition = new ExpressionSqlNodeConverter().convert(expressionSegment);
+

Review comment:
       These blank lines are suggested to remove. ### If you want to isolate `SqlLiteral conditionType`, a new function will make it better.

##########
File path: shardingsphere-infra/shardingsphere-infra-optimize/src/main/java/org/apache/shardingsphere/infra/optimize/core/convert/converter/impl/GroupBySqlNodeConverter.java
##########
@@ -0,0 +1,46 @@
+/*
+ * 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.infra.optimize.core.convert.converter.impl;
+
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlNodeList;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.shardingsphere.infra.optimize.core.convert.converter.SqlNodeConverter;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.order.GroupBySegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.order.item.OrderByItemSegment;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+
+/**
+ * Group by converter.
+ */
+public final class GroupBySqlNodeConverter extends AbstractOrderBySqlNodeConverter implements SqlNodeConverter<GroupBySegment, SqlNodeList> {
+    @Override

Review comment:
       A new line above this line is expected to add.

##########
File path: shardingsphere-infra/shardingsphere-infra-optimize/src/test/java/org/apache/shardingsphere/infra/optimize/core/convert/SqlNodeConverterTest.java
##########
@@ -0,0 +1,139 @@
+/*
+ * 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.infra.optimize.core.convert;
+
+import org.apache.calcite.sql.SqlJoin;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlSelect;
+import org.apache.shardingsphere.infra.database.type.DatabaseTypeRegistry;
+import org.apache.shardingsphere.infra.database.type.dialect.MySQLDatabaseType;
+import org.apache.shardingsphere.infra.parser.ShardingSphereSQLParserEngine;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.Optional;
+
+import static org.hamcrest.CoreMatchers.instanceOf;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * testcase of converting shardingshphere ast to calcite ast.
+ *
+ * <p>after converting phrase finished, the next phrase is comparing  the converted result with the
+ * result of calcite parser.
+ * </p>
+ */
+public final class SqlNodeConverterTest {

Review comment:
       This class is suggested to rename as SelectStatementConverterTest, since it just mentions `SelectStatment` coverting.

##########
File path: shardingsphere-infra/shardingsphere-infra-optimize/src/test/java/org/apache/shardingsphere/infra/optimize/core/convert/SqlNodeConverterTest.java
##########
@@ -0,0 +1,139 @@
+/*
+ * 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.infra.optimize.core.convert;
+
+import org.apache.calcite.sql.SqlJoin;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlSelect;
+import org.apache.shardingsphere.infra.database.type.DatabaseTypeRegistry;
+import org.apache.shardingsphere.infra.database.type.dialect.MySQLDatabaseType;
+import org.apache.shardingsphere.infra.parser.ShardingSphereSQLParserEngine;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.Optional;
+
+import static org.hamcrest.CoreMatchers.instanceOf;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * testcase of converting shardingshphere ast to calcite ast.
+ *
+ * <p>after converting phrase finished, the next phrase is comparing  the converted result with the
+ * result of calcite parser.
+ * </p>
+ */
+public final class SqlNodeConverterTest {
+
+    private ShardingSphereSQLParserEngine sqlStatementParserEngine;
+
+    @Before
+    public void init() {
+        sqlStatementParserEngine = new ShardingSphereSQLParserEngine(DatabaseTypeRegistry.getTrunkDatabaseTypeName(
+                new MySQLDatabaseType()));
+    }
+
+    @Test
+    public void testConvertSimpleSelect() {
+        String sql = "select order_id, user_id from t_order";
+        SQLStatement sqlStatement = sqlStatementParserEngine.parse(sql, false);
+        Optional<SqlNode> optional = SqlNodeConverter.convert(sqlStatement);
+        assertTrue(optional.isPresent());
+        Assert.assertThat(optional.get(), instanceOf(SqlSelect.class));
+        SqlSelect sqlSelect = (SqlSelect) optional.get();
+        Assert.assertEquals(2, sqlSelect.getSelectList().size());

Review comment:
       Could you import `Assert.assertEquals` to avoid `Assert.` expression? Plus, please check other calls. Thanks.

##########
File path: shardingsphere-infra/shardingsphere-infra-optimize/src/test/java/org/apache/shardingsphere/infra/optimize/core/convert/SqlNodeConverterTest.java
##########
@@ -0,0 +1,139 @@
+/*
+ * 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.infra.optimize.core.convert;
+
+import org.apache.calcite.sql.SqlJoin;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlSelect;
+import org.apache.shardingsphere.infra.database.type.DatabaseTypeRegistry;
+import org.apache.shardingsphere.infra.database.type.dialect.MySQLDatabaseType;
+import org.apache.shardingsphere.infra.parser.ShardingSphereSQLParserEngine;
+import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.Optional;
+
+import static org.hamcrest.CoreMatchers.instanceOf;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * testcase of converting shardingshphere ast to calcite ast.
+ *
+ * <p>after converting phrase finished, the next phrase is comparing  the converted result with the
+ * result of calcite parser.
+ * </p>
+ */
+public final class SqlNodeConverterTest {

Review comment:
       Hey, it is better to be `final` class.

##########
File path: shardingsphere-infra/shardingsphere-infra-optimize/src/main/java/org/apache/shardingsphere/infra/optimize/core/convert/converter/impl/SelectStatementSqlNodeConverter.java
##########
@@ -22,20 +22,69 @@
 import org.apache.calcite.sql.SqlSelect;
 import org.apache.calcite.sql.parser.SqlParserPos;
 import org.apache.shardingsphere.infra.optimize.core.convert.converter.SqlNodeConverter;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.pagination.PaginationValueSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.pagination.limit.LimitSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.SelectStatement;
+import org.apache.shardingsphere.sql.parser.sql.dialect.handler.dml.SelectStatementHandler;
 
+import java.util.AbstractMap;
+import java.util.Map;
 import java.util.Optional;
 
 /**
  * Select statement sql node converter.
  */
-public final class SelectStatementSqlNodeConverter implements SqlNodeConverter<SelectStatement> {
+public final class SelectStatementSqlNodeConverter implements SqlNodeConverter<SelectStatement, SqlNode> {
     
     @Override
     public Optional<SqlNode> convert(final SelectStatement selectStatement) {
-        Optional<SqlNode> distinct = new DistinctSqlNodeConverter().convert(selectStatement.getProjections());
-        // TODO : prepare other sqlNodes referring to `distinct`.
-        return Optional.of(new SqlSelect(SqlParserPos.ZERO, (SqlNodeList) distinct.orElse(null), null, null, null, null, null,
-                null, null, null, null, null));
+        Optional<SqlNodeList> distinct = new DistinctSqlNodeConverter().convert(selectStatement.getProjections());
+        Optional<SqlNodeList> projections = new ProjectionsSqlNodeConverter().convert(selectStatement.getProjections());
+        Optional<SqlNode> from = new TableSqlNodeConverter().convert(selectStatement.getFrom());
+        Optional<SqlNode> where = new WhereSqlNodeConverter().convert(selectStatement.getWhere().orElse(null));
+        Optional<SqlNodeList> groupBy = new GroupBySqlNodeConverter().convert(selectStatement.getGroupBy().orElse(null));
+        Optional<SqlNode> having = new HavingSqlNodeConverter().convert(selectStatement.getHaving().orElse(null));
+        Optional<SqlNodeList> orderBy = new OrderBySqlNodeConverter().convert(selectStatement.getOrderBy().orElse(null));
+        Optional<LimitSegment> limit = SelectStatementHandler.getLimitSegment(selectStatement);
+        Map.Entry<SqlNode, SqlNode> offsetRowCount = convertPagination(limit.orElse(null));
+        return Optional.of(new SqlSelect(SqlParserPos.ZERO, 
+                distinct.orElse(null), 
+                projections.orElse(null), 
+                from.orElse(null), 
+                where.orElse(null), 
+                groupBy.orElse(null), 
+                having.orElse(null),
+                null, 
+                orderBy.orElse(null), 
+                offsetRowCount.getKey(), 
+                offsetRowCount.getValue(), 
+                null));
     }
+
+    /**
+     * convert pagination.
+     * @param limitSegment pagination clause
+     * @return offset and fetch <code>SqlNode</code>.
+     */
+    public static Map.Entry<SqlNode, SqlNode> convertPagination(final LimitSegment limitSegment) {

Review comment:
       Do you think whether we can split it into two classes, i.e.,`covertOffsetSqlNode` and c`overtFetchSqlnode`?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org