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/20 15:19:11 UTC

[GitHub] [shardingsphere] guimingyue opened a new pull request #10889: Convert ss ast to calcite ast

guimingyue opened a new pull request #10889:
URL: https://github.com/apache/shardingsphere/pull/10889


   #9934 
   Changes proposed in this pull request:
   - add a [SqlNodeConverter](https://github.com/guimingyue/shardingsphere/blob/sql-converter/shardingsphere-infra/shardingsphere-infra-optimize/src/main/java/org/apache/shardingsphere/infra/optimize/core/convert/SqlNodeConverter.java) class to convert ss ast to calcite ast.
   - add some testcase for SqlNodeConverter.
   


-- 
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



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

Posted by GitBox <gi...@apache.org>.
tristaZero commented on a change in pull request #10889:
URL: https://github.com/apache/shardingsphere/pull/10889#discussion_r657729250



##########
File path: shardingsphere-infra/shardingsphere-infra-optimize/src/main/java/org/apache/shardingsphere/infra/optimize/core/convert/converter/impl/AbstractLimitSqlNodeConverter.java
##########
@@ -0,0 +1,50 @@
+/*
+ * 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.pagination.PaginationValueSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.pagination.limit.LimitSegment;
+
+import java.util.Optional;
+import java.util.function.Function;
+
+/**
+ * Abstract limit sql node converter.
+ */
+public abstract class AbstractLimitSqlNodeConverter implements SqlNodeConverter<LimitSegment, SqlNode> {
+
+    private Function<LimitSegment, Optional<PaginationValueSegment>> function;

Review comment:
       What's the advantage this way brings?

##########
File path: shardingsphere-infra/shardingsphere-infra-optimize/src/main/java/org/apache/shardingsphere/infra/optimize/core/convert/converter/impl/AbstractLimitSqlNodeConverter.java
##########
@@ -0,0 +1,50 @@
+/*
+ * 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.pagination.PaginationValueSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.pagination.limit.LimitSegment;
+
+import java.util.Optional;
+import java.util.function.Function;
+
+/**
+ * Abstract limit sql node converter.
+ */
+public abstract class AbstractLimitSqlNodeConverter implements SqlNodeConverter<LimitSegment, SqlNode> {
+
+    private Function<LimitSegment, Optional<PaginationValueSegment>> function;

Review comment:
       We only have two sqlNode need this function, do we?

##########
File path: shardingsphere-infra/shardingsphere-infra-optimize/src/main/java/org/apache/shardingsphere/infra/optimize/core/convert/converter/SqlNodeConverterUtil.java
##########
@@ -0,0 +1,63 @@
+/*
+ * 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;
+
+import com.google.common.collect.Lists;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.shardingsphere.infra.optimize.core.convert.converter.impl.ColumnOrderByItemSqlNodeConverter;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.order.item.ColumnOrderByItemSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.order.item.ExpressionOrderByItemSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.order.item.IndexOrderByItemSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.order.item.OrderByItemSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.order.item.TextOrderByItemSegment;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+
+/**
+ * Utility for sql node converter.
+ */
+public final class SqlNodeConverterUtil {
+    
+    /**
+     * Convert order by items.
+     * @param orderByItems order by item list.
+     * @return a collection of order by item <code>SqlNode</code>
+     */
+    public static List<SqlNode> convertOrderByItems(final Collection<OrderByItemSegment> orderByItems) {
+        List<SqlNode> sqlNodes = Lists.newArrayList();

Review comment:
       How about creating an `OrderByItemSegmentSqlNodeConverter`? We can  do `for each` in `SelectStatmentSqlNode`.




-- 
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



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

Posted by GitBox <gi...@apache.org>.
guimingyue commented on a change in pull request #10889:
URL: https://github.com/apache/shardingsphere/pull/10889#discussion_r657240308



##########
File path: shardingsphere-infra/shardingsphere-infra-optimize/src/test/java/org/apache/shardingsphere/infra/optimize/core/convert/SelectStatementSqlNodeConverterTest.java
##########
@@ -0,0 +1,138 @@
+/*
+ * 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;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertThat;
+
+/**
+ * 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 SelectStatementSqlNodeConverterTest {
+
+    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 = SqlNodeConvertEngine.convert(sqlStatement);
+        assertTrue(optional.isPresent());
+        Assert.assertThat(optional.get(), instanceOf(SqlSelect.class));
+        SqlSelect sqlSelect = (SqlSelect) optional.get();
+        assertEquals(2, sqlSelect.getSelectList().size());
+        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);
+        Optional<SqlNode> optional = SqlNodeConvertEngine.convert(sqlStatement);
+        assertTrue(optional.isPresent());
+        Assert.assertThat(optional.get(), instanceOf(SqlSelect.class));
+        SqlSelect sqlSelect = (SqlSelect) optional.get();
+        assertEquals(2, sqlSelect.getSelectList().size());
+        assertNotNull(sqlSelect.getWhere());
+    }
+
+    @Test
+    public void testConvertSimpleSelectFilterGroupBy() {
+        String sql = "select order_id, user_id from t_order where order_id = 10 group by order_id";
+        SQLStatement sqlStatement = sqlStatementParserEngine.parse(sql, false);
+        Optional<SqlNode> optional = SqlNodeConvertEngine.convert(sqlStatement);
+        assertTrue(optional.isPresent());
+        Assert.assertThat(optional.get(), instanceOf(SqlSelect.class));
+        SqlSelect sqlSelect = (SqlSelect) optional.get();
+        assertEquals(2, sqlSelect.getSelectList().size());
+        assertNotNull(sqlSelect.getWhere());
+        assertEquals(1, sqlSelect.getGroup().size());
+    }
+
+    @Test
+    public void testConvertSimpleSelectFilterOrderBy() {
+        String sql = "select order_id, user_id from t_order where user_id = 10 order by order_id desc";
+        SQLStatement sqlStatement = sqlStatementParserEngine.parse(sql, false);
+        Optional<SqlNode> optional = SqlNodeConvertEngine.convert(sqlStatement);
+        assertTrue(optional.isPresent());
+        assertThat(optional.get(), instanceOf(SqlSelect.class));
+        SqlSelect sqlSelect = (SqlSelect) optional.get();
+        assertEquals(2, sqlSelect.getSelectList().size());
+        assertNotNull(sqlSelect.getWhere());
+        assertEquals(1, sqlSelect.getOrderList().size());
+    }
+
+    @Test
+    public void testConvertInnerJoin() {
+        String sql = "select 10 + 30, o1.order_id + 10, o1.order_id, o1.user_id, o2.status from t_order o1 join t_order_item o2 on "
+                + "o1.order_id = o2.order_id where o1.status='FINISHED' and o2.order_item_id > 1024 and 1=1 order by "
+                + "o1.order_id desc";
+        SQLStatement sqlStatement = sqlStatementParserEngine.parse(sql, false);
+        Optional<SqlNode> optional = SqlNodeConvertEngine.convert(sqlStatement);
+        assertTrue(optional.isPresent());
+        assertThat(optional.get(), instanceOf(SqlSelect.class));
+        SqlSelect sqlSelect = (SqlSelect) optional.get();
+        assertThat(sqlSelect.getFrom(), instanceOf(SqlJoin.class));
+        assertEquals(1, sqlSelect.getOrderList().size());
+    }
+
+    @Test
+    public void testConvertLeftOuterJoin() {
+        String sql = "select 10 + 30, o1.order_id + 10, o1.order_id, o1.user_id, o2.status from t_order o1 left outer join t_order_item o2 on "
+                + "o1.order_id = o2.order_id where o1.status='FINISHED' and o2.order_item_id > 1024 and 1=1 order by "
+                + "o1.order_id desc";
+        SQLStatement sqlStatement = sqlStatementParserEngine.parse(sql, false);
+        Optional<SqlNode> optional = SqlNodeConvertEngine.convert(sqlStatement);
+        assertTrue(optional.isPresent());
+        // TODO outer join is not supported by parser of ShardingSphere 
+    }
+

Review comment:
       done




-- 
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



[GitHub] [shardingsphere] guimingyue commented on pull request #10889: Convert ss ast to calcite ast

Posted by GitBox <gi...@apache.org>.
guimingyue commented on pull request #10889:
URL: https://github.com/apache/shardingsphere/pull/10889#issuecomment-869379872


   > involved
   
   @tristaZero Thanks for your help.
   For the UTs, I think it is not difficult for contributors who are familiar with AST of SQL. 
   And the TODO of these class if left for that I think we need someone who is good with both antlr and ss ast. I know how to convert ss ast to calcite ast, but it is difficult for me to parse sql text to ss ast.
   BTW, what's your plan and roadmap for integrating with calcite?


-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@shardingsphere.apache.org

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



[GitHub] [shardingsphere] tristaZero merged pull request #10889: Convert ss ast to calcite ast

Posted by GitBox <gi...@apache.org>.
tristaZero merged pull request #10889:
URL: https://github.com/apache/shardingsphere/pull/10889


   


-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@shardingsphere.apache.org

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



[GitHub] [shardingsphere] guimingyue commented on pull request #10889: Convert ss ast to calcite ast

Posted by GitBox <gi...@apache.org>.
guimingyue commented on pull request #10889:
URL: https://github.com/apache/shardingsphere/pull/10889#issuecomment-864976572


   > Hi @guimingyue ,
   > 
   > That's exactly for our first step. The content is fine with me, but I assume we need an `engine` or `mechanism` to do converting work.
   > Instead of `all in one`, I prefer splitting `sqlNodeConverter` into many child ones, each of which just focuses its function and avoids interfering with others.
   > I just raised a #10895 to help us do such things. How do you think? It is possible to fill #10895 with what you did in this PR?
   > 
   > If you like, we still have other contributors who are willing to give a hand.
   
   It's fine with me, I'll finish this today or tomorrow. I think need some help.


-- 
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



[GitHub] [shardingsphere] tristaZero edited a comment on pull request #10889: Convert ss ast to calcite ast

Posted by GitBox <gi...@apache.org>.
tristaZero edited a comment on pull request #10889:
URL: https://github.com/apache/shardingsphere/pull/10889#issuecomment-864955327


   Hi @guimingyue ,
   
   That's exactly for our first step. The content is fine with me, but I assume we need an `engine` or `mechanism` to do converting work. 
   Instead of `all in one`, I prefer splitting `sqlNodeConverter` into many child ones, each of which just focuses its function and avoids interfering with others. 
   I just raised a #10895 to help us do such things. How do you think? It is possible to fill #10895 with what you did in this PR?
   
   If you like, we still have other contributors who are willing to give a hand.


-- 
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



[GitHub] [shardingsphere] guimingyue commented on pull request #10889: Convert ss ast to calcite ast

Posted by GitBox <gi...@apache.org>.
guimingyue commented on pull request #10889:
URL: https://github.com/apache/shardingsphere/pull/10889#issuecomment-869771789


   > HI @guimingyue ,
   > 
   > The next step is to make sure `ShardingSphereOptimizer` can optimize with ShardingSphere parser, please refer to #11042.
   > Do you think `ShardingSphereOptimizer` can work well?
   
   Actually, I don't understand what is the `ShardingSphereOptimizer`  is created for. Where the `sql` parameter of `optimize` is from and how the `RelNode` generated by another `optimize` method will be executed. So, I don't know whether `ShardingSphereOptimizer` can work well.


-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@shardingsphere.apache.org

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



[GitHub] [shardingsphere] guimingyue commented on pull request #10889: Convert ss ast to calcite ast

Posted by GitBox <gi...@apache.org>.
guimingyue commented on pull request #10889:
URL: https://github.com/apache/shardingsphere/pull/10889#issuecomment-866421447


   @tristaZero I finished this branch,please give it a code review and comment your suggestions. 


-- 
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



[GitHub] [shardingsphere] guimingyue commented on pull request #10889: Convert ss ast to calcite ast

Posted by GitBox <gi...@apache.org>.
guimingyue commented on pull request #10889:
URL: https://github.com/apache/shardingsphere/pull/10889#issuecomment-866965098


   @tristaZero As for offset and row count sql node converter, I add an abstract class to decrease duplicate code line.


-- 
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



[GitHub] [shardingsphere] guimingyue commented on pull request #10889: Convert ss ast to calcite ast

Posted by GitBox <gi...@apache.org>.
guimingyue commented on pull request #10889:
URL: https://github.com/apache/shardingsphere/pull/10889#issuecomment-865532494


   > Hi @guimingyue Thanks for your prompt update.
   > It looks our PR begins to come into shape. ;-) Please give a look at these comments to make it better. Thanks.
   
   I haven't finished this branch. I'll keep improving it today.


-- 
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



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

Posted by GitBox <gi...@apache.org>.
tristaZero commented on pull request #10889:
URL: https://github.com/apache/shardingsphere/pull/10889#issuecomment-870151290


   @guimingyue 
   
   ### UT
   That will be fine. Do you think `SelectStatementSqlNodeConverterTest` still needs more assertion?
   
   ### ShardingSphereOptimizer
   Just to get the best physical plan. Please look from `optimize()` of `ShardingSphereOptimizer` which contains parsing, sqlNode converting and optimizing. 
   


-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@shardingsphere.apache.org

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



[GitHub] [shardingsphere] guimingyue commented on pull request #10889: Convert ss ast to calcite ast

Posted by GitBox <gi...@apache.org>.
guimingyue commented on pull request #10889:
URL: https://github.com/apache/shardingsphere/pull/10889#issuecomment-869686853


   > @guimingyue If we add a UT for converter, which assertions are expected?
   > 
   > ```java
   > assertEquals(2, sqlSelect.getSelectList().size());
   > assertNull(sqlSelect.getWhere());
   > assertThat(optional.get(), instanceOf(SqlSelect.class))
   > ```
   > 
   > Do you think these items are enough?
   > 
   > Could you firstly create a few UT issues for these `converter`, like #10904? We can assign these issues you created to other contributors.
   
   These UT asserts are not enough, we should compare the result with Calcite parser result.
   I will create some ideal UT first. And then create issues to assign to other contributors.


-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@shardingsphere.apache.org

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



[GitHub] [shardingsphere] guimingyue commented on pull request #10889: Convert ss ast to calcite ast

Posted by GitBox <gi...@apache.org>.
guimingyue commented on pull request #10889:
URL: https://github.com/apache/shardingsphere/pull/10889#issuecomment-872210248


   > @guimingyue
   > Do you think we still need to pass SQL parameters to `ShardingSphereOptimizer` to get the best physical executing plan?
   
   It depends on the design of the federated SQL query.


-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@shardingsphere.apache.org

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



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

Posted by GitBox <gi...@apache.org>.
tristaZero commented on pull request #10889:
URL: https://github.com/apache/shardingsphere/pull/10889#issuecomment-872133550


   @guimingyue 
   Do you think we still need to pass SQL parameters to `ShardingSphereOptimizer` to  get the best physical executing plan?


-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@shardingsphere.apache.org

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



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

Posted by GitBox <gi...@apache.org>.
tristaZero commented on a change in pull request #10889:
URL: https://github.com/apache/shardingsphere/pull/10889#discussion_r658585204



##########
File path: shardingsphere-infra/shardingsphere-infra-optimize/src/main/java/org/apache/shardingsphere/infra/optimize/core/convert/converter/SqlNodeConverterUtil.java
##########
@@ -0,0 +1,63 @@
+/*
+ * 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;
+
+import com.google.common.collect.Lists;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.shardingsphere.infra.optimize.core.convert.converter.impl.ColumnOrderByItemSqlNodeConverter;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.order.item.ColumnOrderByItemSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.order.item.ExpressionOrderByItemSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.order.item.IndexOrderByItemSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.order.item.OrderByItemSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.order.item.TextOrderByItemSegment;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+
+/**
+ * Utility for sql node converter.
+ */
+public final class SqlNodeConverterUtil {
+    
+    /**
+     * Convert order by items.
+     * @param orderByItems order by item list.
+     * @return a collection of order by item <code>SqlNode</code>
+     */
+    public static List<SqlNode> convertOrderByItems(final Collection<OrderByItemSegment> orderByItems) {
+        List<SqlNode> sqlNodes = Lists.newArrayList();

Review comment:
       I suppose you still have this sense that this class is supposed to implement the converting interface, just because its result is not a sqlNode, we have this util class. If there are only two classes calling this one, I guess it is acceptable to do two `for each`  loops. How do you think?




-- 
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



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

Posted by GitBox <gi...@apache.org>.
tristaZero commented on a change in pull request #10889:
URL: https://github.com/apache/shardingsphere/pull/10889#discussion_r658580003



##########
File path: shardingsphere-infra/shardingsphere-infra-optimize/src/main/java/org/apache/shardingsphere/infra/optimize/core/convert/converter/SqlNodeConverterUtil.java
##########
@@ -0,0 +1,63 @@
+/*
+ * 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;
+
+import com.google.common.collect.Lists;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.shardingsphere.infra.optimize.core.convert.converter.impl.ColumnOrderByItemSqlNodeConverter;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.order.item.ColumnOrderByItemSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.order.item.ExpressionOrderByItemSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.order.item.IndexOrderByItemSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.order.item.OrderByItemSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.order.item.TextOrderByItemSegment;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+
+/**
+ * Utility for sql node converter.
+ */
+public final class SqlNodeConverterUtil {
+    
+    /**
+     * Convert order by items.
+     * @param orderByItems order by item list.
+     * @return a collection of order by item <code>SqlNode</code>
+     */
+    public static List<SqlNode> convertOrderByItems(final Collection<OrderByItemSegment> orderByItems) {
+        List<SqlNode> sqlNodes = Lists.newArrayList();

Review comment:
       This PR now looks great, after we fix this one, it is time to merge it, I suppose.

##########
File path: shardingsphere-infra/shardingsphere-infra-optimize/src/main/java/org/apache/shardingsphere/infra/optimize/core/convert/converter/SqlNodeConverterUtil.java
##########
@@ -0,0 +1,63 @@
+/*
+ * 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;
+
+import com.google.common.collect.Lists;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.shardingsphere.infra.optimize.core.convert.converter.impl.ColumnOrderByItemSqlNodeConverter;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.order.item.ColumnOrderByItemSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.order.item.ExpressionOrderByItemSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.order.item.IndexOrderByItemSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.order.item.OrderByItemSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.order.item.TextOrderByItemSegment;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+
+/**
+ * Utility for sql node converter.
+ */
+public final class SqlNodeConverterUtil {
+    
+    /**
+     * Convert order by items.
+     * @param orderByItems order by item list.
+     * @return a collection of order by item <code>SqlNode</code>
+     */
+    public static List<SqlNode> convertOrderByItems(final Collection<OrderByItemSegment> orderByItems) {
+        List<SqlNode> sqlNodes = Lists.newArrayList();

Review comment:
       Hey, there only two places to call this class, right? Do you think there will be more classes calling this one later on?




-- 
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



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

Posted by GitBox <gi...@apache.org>.
guimingyue commented on a change in pull request #10889:
URL: https://github.com/apache/shardingsphere/pull/10889#discussion_r657239894



##########
File path: shardingsphere-infra/shardingsphere-infra-optimize/src/main/java/org/apache/shardingsphere/infra/optimize/core/convert/converter/impl/SelectStatementSqlNodeConverter.java
##########
@@ -22,20 +22,66 @@
 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) {
+        if (limitSegment == null) {
+            return new AbstractMap.SimpleEntry<>(null, null);
+        }
+
+        Optional<SqlNode> offsetSqlNode = Optional.empty();
+        Optional<SqlNode> fetchSqlNode = Optional.empty();
+        Optional<PaginationValueSegment> offset = limitSegment.getOffset();
+        Optional<PaginationValueSegment> fetch = limitSegment.getRowCount();
+        if (offset.isPresent()) {
+            offsetSqlNode = new PaginationValueSqlConverter().convert(offset.get());
+        }
+        if (fetch.isPresent()) {
+            fetchSqlNode = new PaginationValueSqlConverter().convert(fetch.get());
+        }
+        return new AbstractMap.SimpleEntry<>(offsetSqlNode.orElse(null), fetchSqlNode.orElse(null));
+    }
+

Review comment:
       done




-- 
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



[GitHub] [shardingsphere] codecov-commenter commented on pull request #10889: Convert ss ast to calcite ast

Posted by GitBox <gi...@apache.org>.
codecov-commenter commented on pull request #10889:
URL: https://github.com/apache/shardingsphere/pull/10889#issuecomment-864574578


   # [Codecov](https://codecov.io/gh/apache/shardingsphere/pull/10889?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#10889](https://codecov.io/gh/apache/shardingsphere/pull/10889?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (dec9dd7) into [master](https://codecov.io/gh/apache/shardingsphere/commit/b08767cbe237956734f1acc1cf8a49241542d478?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (b08767c) will **decrease** coverage by `0.04%`.
   > The diff coverage is `56.72%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/shardingsphere/pull/10889/graphs/tree.svg?width=650&height=150&src=pr&token=ZvlXpWa7so&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/shardingsphere/pull/10889?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master   #10889      +/-   ##
   ============================================
   - Coverage     65.85%   65.80%   -0.05%     
   - Complexity      707      709       +2     
   ============================================
     Files          1802     1803       +1     
     Lines         30798    30968     +170     
     Branches       5550     5595      +45     
   ============================================
   + Hits          20283    20380      +97     
   - Misses         8911     8968      +57     
   - Partials       1604     1620      +16     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/shardingsphere/pull/10889?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [.../infra/optimize/core/convert/SqlNodeConverter.java](https://codecov.io/gh/apache/shardingsphere/pull/10889/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2hhcmRpbmdzcGhlcmUtaW5mcmEvc2hhcmRpbmdzcGhlcmUtaW5mcmEtb3B0aW1pemUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NoYXJkaW5nc3BoZXJlL2luZnJhL29wdGltaXplL2NvcmUvY29udmVydC9TcWxOb2RlQ29udmVydGVyLmphdmE=) | `53.24% <53.24%> (+53.24%)` | :arrow_up: |
   | [...nfra/optimize/core/operator/BinarySqlOperator.java](https://codecov.io/gh/apache/shardingsphere/pull/10889/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2hhcmRpbmdzcGhlcmUtaW5mcmEvc2hhcmRpbmdzcGhlcmUtaW5mcmEtb3B0aW1pemUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NoYXJkaW5nc3BoZXJlL2luZnJhL29wdGltaXplL2NvcmUvb3BlcmF0b3IvQmluYXJ5U3FsT3BlcmF0b3IuamF2YQ==) | `88.23% <88.23%> (ø)` | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/shardingsphere/pull/10889?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/shardingsphere/pull/10889?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [b08767c...dec9dd7](https://codecov.io/gh/apache/shardingsphere/pull/10889?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


-- 
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



[GitHub] [shardingsphere] guimingyue commented on pull request #10889: Convert ss ast to calcite ast

Posted by GitBox <gi...@apache.org>.
guimingyue commented on pull request #10889:
URL: https://github.com/apache/shardingsphere/pull/10889#issuecomment-869686853






-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@shardingsphere.apache.org

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



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

Posted by GitBox <gi...@apache.org>.
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



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

Posted by GitBox <gi...@apache.org>.
tristaZero commented on a change in pull request #10889:
URL: https://github.com/apache/shardingsphere/pull/10889#discussion_r658580874



##########
File path: shardingsphere-infra/shardingsphere-infra-optimize/src/main/java/org/apache/shardingsphere/infra/optimize/core/convert/converter/SqlNodeConverterUtil.java
##########
@@ -0,0 +1,63 @@
+/*
+ * 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;
+
+import com.google.common.collect.Lists;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.shardingsphere.infra.optimize.core.convert.converter.impl.ColumnOrderByItemSqlNodeConverter;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.order.item.ColumnOrderByItemSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.order.item.ExpressionOrderByItemSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.order.item.IndexOrderByItemSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.order.item.OrderByItemSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.order.item.TextOrderByItemSegment;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+
+/**
+ * Utility for sql node converter.
+ */
+public final class SqlNodeConverterUtil {
+    
+    /**
+     * Convert order by items.
+     * @param orderByItems order by item list.
+     * @return a collection of order by item <code>SqlNode</code>
+     */
+    public static List<SqlNode> convertOrderByItems(final Collection<OrderByItemSegment> orderByItems) {
+        List<SqlNode> sqlNodes = Lists.newArrayList();

Review comment:
       Please give a look at the failed ci tests.




-- 
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



[GitHub] [shardingsphere] guimingyue commented on pull request #10889: Convert ss ast to calcite ast

Posted by GitBox <gi...@apache.org>.
guimingyue commented on pull request #10889:
URL: https://github.com/apache/shardingsphere/pull/10889#issuecomment-871163721


   > @guimingyue
   > 
   > ### UT
   > That will be fine. Do you think `SelectStatementSqlNodeConverterTest` still needs more assertion?
   > 
   > ### ShardingSphereOptimizer
   > Just to get the best physical plan. Please look from `optimize()` of `ShardingSphereOptimizer` which contains parsing, sqlNode converting and optimizing.
   
   Is the SQL parameter of the `optimize` method the result of the federated router? if so, what is the boundary of the federated router? 


-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@shardingsphere.apache.org

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



[GitHub] [shardingsphere] guimingyue commented on pull request #10889: Convert ss ast to calcite ast

Posted by GitBox <gi...@apache.org>.
guimingyue commented on pull request #10889:
URL: https://github.com/apache/shardingsphere/pull/10889#issuecomment-864976572






-- 
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



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

Posted by GitBox <gi...@apache.org>.
guimingyue commented on a change in pull request #10889:
URL: https://github.com/apache/shardingsphere/pull/10889#discussion_r656379280



##########
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:
       done




-- 
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



[GitHub] [shardingsphere] tristaZero edited a comment on pull request #10889: Convert ss ast to calcite ast

Posted by GitBox <gi...@apache.org>.
tristaZero edited a comment on pull request #10889:
URL: https://github.com/apache/shardingsphere/pull/10889#issuecomment-869406519


   @guimingyue If we add a UT for converter, which assertions are expected? 
   
   ```java
   assertEquals(2, sqlSelect.getSelectList().size());
   assertNull(sqlSelect.getWhere());
   assertThat(optional.get(), instanceOf(SqlSelect.class))
   ```
   Do you think these items are enough?
   
   Could you firstly create a few UT issues for these `converter`, like #10904? We can assign these issues you created to other contributors.


-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@shardingsphere.apache.org

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



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

Posted by GitBox <gi...@apache.org>.
tristaZero commented on a change in pull request #10889:
URL: https://github.com/apache/shardingsphere/pull/10889#discussion_r659402556



##########
File path: shardingsphere-infra/shardingsphere-infra-optimize/src/main/java/org/apache/shardingsphere/infra/optimize/core/convert/converter/SqlNodeConverterUtil.java
##########
@@ -0,0 +1,63 @@
+/*
+ * 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;
+
+import com.google.common.collect.Lists;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.shardingsphere.infra.optimize.core.convert.converter.impl.ColumnOrderByItemSqlNodeConverter;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.order.item.ColumnOrderByItemSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.order.item.ExpressionOrderByItemSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.order.item.IndexOrderByItemSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.order.item.OrderByItemSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.order.item.TextOrderByItemSegment;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+
+/**
+ * Utility for sql node converter.
+ */
+public final class SqlNodeConverterUtil {
+    
+    /**
+     * Convert order by items.
+     * @param orderByItems order by item list.
+     * @return a collection of order by item <code>SqlNode</code>
+     */
+    public static List<SqlNode> convertOrderByItems(final Collection<OrderByItemSegment> orderByItems) {
+        List<SqlNode> sqlNodes = Lists.newArrayList();

Review comment:
       Ok, It's about ones' design thinking, you can keep your style.




-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@shardingsphere.apache.org

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



[GitHub] [shardingsphere] tristaZero edited a comment on pull request #10889: Convert ss ast to calcite ast

Posted by GitBox <gi...@apache.org>.
tristaZero edited a comment on pull request #10889:
URL: https://github.com/apache/shardingsphere/pull/10889#issuecomment-864955327


   Hi @guimingyue ,
   
   That's exactly for our first step. The content is fine with me, but I assume we need an `engine` or `mechanism` to do converting work. 
   Instead of `all in one`, I prefer splitting `sqlNodeConverter` into many child ones, each of which just focuses its function and avoids interfering with others. 
   I just raised a #10895 to help us do such things. How do you think? It is possible to fill #10895 with what you did in this PR?
   
   If you like, we still have other contributors who are willing to give a hand.


-- 
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



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

Posted by GitBox <gi...@apache.org>.
tristaZero commented on pull request #10889:
URL: https://github.com/apache/shardingsphere/pull/10889#issuecomment-864955327


   Hi @guimingyue ,
   
   That's exactly for our first step. The content is fine with me, but I assume we need an `engine` or `mechanism` to do converting work. 
   Instead of `all in one`, I prefer splitting `sqlNodeConverter` into many child ones, each of which just focuses its function and avoids interfering with others. 
   I just raised a #10895 to help us do such things. How do you think? It is possible to fill #10895 with what you did in this PR?


-- 
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



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

Posted by GitBox <gi...@apache.org>.
tristaZero commented on pull request #10889:
URL: https://github.com/apache/shardingsphere/pull/10889#issuecomment-869548274


   HI @guimingyue ,
   
   The next step is to make sure `ShardingSphereOptimizer` can optimize with ShardingSphere parser, please refer to #11042.
   Do you think `ShardingSphereOptimizer` can work well?


-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@shardingsphere.apache.org

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



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

Posted by GitBox <gi...@apache.org>.
tristaZero commented on a change in pull request #10889:
URL: https://github.com/apache/shardingsphere/pull/10889#discussion_r657037370



##########
File path: shardingsphere-infra/shardingsphere-infra-optimize/src/test/java/org/apache/shardingsphere/infra/optimize/core/convert/SelectStatementSqlNodeConverterTest.java
##########
@@ -0,0 +1,138 @@
+/*
+ * 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;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertThat;
+
+/**
+ * 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 SelectStatementSqlNodeConverterTest {
+
+    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 = SqlNodeConvertEngine.convert(sqlStatement);
+        assertTrue(optional.isPresent());
+        Assert.assertThat(optional.get(), instanceOf(SqlSelect.class));
+        SqlSelect sqlSelect = (SqlSelect) optional.get();
+        assertEquals(2, sqlSelect.getSelectList().size());
+        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);
+        Optional<SqlNode> optional = SqlNodeConvertEngine.convert(sqlStatement);
+        assertTrue(optional.isPresent());
+        Assert.assertThat(optional.get(), instanceOf(SqlSelect.class));
+        SqlSelect sqlSelect = (SqlSelect) optional.get();
+        assertEquals(2, sqlSelect.getSelectList().size());
+        assertNotNull(sqlSelect.getWhere());
+    }
+
+    @Test
+    public void testConvertSimpleSelectFilterGroupBy() {
+        String sql = "select order_id, user_id from t_order where order_id = 10 group by order_id";
+        SQLStatement sqlStatement = sqlStatementParserEngine.parse(sql, false);
+        Optional<SqlNode> optional = SqlNodeConvertEngine.convert(sqlStatement);
+        assertTrue(optional.isPresent());
+        Assert.assertThat(optional.get(), instanceOf(SqlSelect.class));
+        SqlSelect sqlSelect = (SqlSelect) optional.get();
+        assertEquals(2, sqlSelect.getSelectList().size());
+        assertNotNull(sqlSelect.getWhere());
+        assertEquals(1, sqlSelect.getGroup().size());
+    }
+
+    @Test
+    public void testConvertSimpleSelectFilterOrderBy() {
+        String sql = "select order_id, user_id from t_order where user_id = 10 order by order_id desc";
+        SQLStatement sqlStatement = sqlStatementParserEngine.parse(sql, false);
+        Optional<SqlNode> optional = SqlNodeConvertEngine.convert(sqlStatement);
+        assertTrue(optional.isPresent());
+        assertThat(optional.get(), instanceOf(SqlSelect.class));
+        SqlSelect sqlSelect = (SqlSelect) optional.get();
+        assertEquals(2, sqlSelect.getSelectList().size());
+        assertNotNull(sqlSelect.getWhere());
+        assertEquals(1, sqlSelect.getOrderList().size());
+    }
+
+    @Test
+    public void testConvertInnerJoin() {
+        String sql = "select 10 + 30, o1.order_id + 10, o1.order_id, o1.user_id, o2.status from t_order o1 join t_order_item o2 on "
+                + "o1.order_id = o2.order_id where o1.status='FINISHED' and o2.order_item_id > 1024 and 1=1 order by "
+                + "o1.order_id desc";
+        SQLStatement sqlStatement = sqlStatementParserEngine.parse(sql, false);
+        Optional<SqlNode> optional = SqlNodeConvertEngine.convert(sqlStatement);
+        assertTrue(optional.isPresent());
+        assertThat(optional.get(), instanceOf(SqlSelect.class));
+        SqlSelect sqlSelect = (SqlSelect) optional.get();
+        assertThat(sqlSelect.getFrom(), instanceOf(SqlJoin.class));
+        assertEquals(1, sqlSelect.getOrderList().size());
+    }
+
+    @Test
+    public void testConvertLeftOuterJoin() {
+        String sql = "select 10 + 30, o1.order_id + 10, o1.order_id, o1.user_id, o2.status from t_order o1 left outer join t_order_item o2 on "
+                + "o1.order_id = o2.order_id where o1.status='FINISHED' and o2.order_item_id > 1024 and 1=1 order by "
+                + "o1.order_id desc";
+        SQLStatement sqlStatement = sqlStatementParserEngine.parse(sql, false);
+        Optional<SqlNode> optional = SqlNodeConvertEngine.convert(sqlStatement);
+        assertTrue(optional.isPresent());
+        // TODO outer join is not supported by parser of ShardingSphere 
+    }
+

Review comment:
       Could you remove this redundant line?

##########
File path: shardingsphere-infra/shardingsphere-infra-optimize/src/main/java/org/apache/shardingsphere/infra/optimize/core/convert/converter/impl/SelectStatementSqlNodeConverter.java
##########
@@ -22,20 +22,66 @@
 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) {
+        if (limitSegment == null) {
+            return new AbstractMap.SimpleEntry<>(null, null);
+        }
+
+        Optional<SqlNode> offsetSqlNode = Optional.empty();
+        Optional<SqlNode> fetchSqlNode = Optional.empty();
+        Optional<PaginationValueSegment> offset = limitSegment.getOffset();
+        Optional<PaginationValueSegment> fetch = limitSegment.getRowCount();
+        if (offset.isPresent()) {
+            offsetSqlNode = new PaginationValueSqlConverter().convert(offset.get());
+        }
+        if (fetch.isPresent()) {
+            fetchSqlNode = new PaginationValueSqlConverter().convert(fetch.get());
+        }
+        return new AbstractMap.SimpleEntry<>(offsetSqlNode.orElse(null), fetchSqlNode.orElse(null));
+    }
+

Review comment:
       Hi could your remove this blank line?

##########
File path: shardingsphere-infra/shardingsphere-infra-optimize/src/main/java/org/apache/shardingsphere/infra/optimize/core/convert/converter/SqlNodeConverter.java
##########
@@ -17,20 +17,55 @@
 
 package org.apache.shardingsphere.infra.optimize.core.convert.converter;
 
+import com.google.common.collect.Lists;
 import org.apache.calcite.sql.SqlNode;
+import org.apache.shardingsphere.infra.optimize.core.convert.converter.impl.ColumnOrderByItemSqlNodeConverter;
 import org.apache.shardingsphere.sql.parser.api.visitor.ASTNode;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.order.item.ColumnOrderByItemSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.order.item.ExpressionOrderByItemSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.order.item.IndexOrderByItemSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.order.item.OrderByItemSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.order.item.TextOrderByItemSegment;
 
+import java.util.Collection;
+import java.util.List;
 import java.util.Optional;
 
 /**
  * SqlNode converter.
  */
-public interface SqlNodeConverter<T extends ASTNode> {
+public interface SqlNodeConverter<T extends ASTNode, R extends SqlNode> {
     
     /**
      *  Convert.
      * @param astNode ast node
      * @return sqlNode optional
      */
-    Optional<SqlNode> convert(T astNode);
+    Optional<R> convert(T astNode);
+
+    /**
+     * Convert order by items.
+     * @param orderByItems order by item list.
+     * @return a collection of order by item <code>SqlNode</code>
+     */
+    default List<SqlNode> convertOrderByItems(final Collection<OrderByItemSegment> orderByItems) {

Review comment:
       It looks `convertOrderByItems` is like `PaginationValueSqlConverter`. Do you think it will better not to provide  `convertOrderByItems` as a default implement? I mean, `default` is much general for most of the classes. But `orderByItem` does not apply to many simple queries, like `select * from tb`;

##########
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:
       Hi do you think #10949 is feasible to fix this issue?




-- 
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



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

Posted by GitBox <gi...@apache.org>.
tristaZero commented on pull request #10889:
URL: https://github.com/apache/shardingsphere/pull/10889#issuecomment-869392829


   Firstly, our convert package is filled with plenty of implementations, nice.
   ![image](https://user-images.githubusercontent.com/27757146/123589148-654a0b00-d81b-11eb-8585-0273f14b8397.png)
   


-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@shardingsphere.apache.org

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



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

Posted by GitBox <gi...@apache.org>.
tristaZero commented on pull request #10889:
URL: https://github.com/apache/shardingsphere/pull/10889#issuecomment-864980293


   Hi @guimingyue Nice,
   So, which part needs more contributors? Maybe create more `converter` like `DistinctSqlNodeConverter`


-- 
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



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

Posted by GitBox <gi...@apache.org>.
tristaZero commented on pull request #10889:
URL: https://github.com/apache/shardingsphere/pull/10889#issuecomment-864955327






-- 
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



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

Posted by GitBox <gi...@apache.org>.
guimingyue commented on a change in pull request #10889:
URL: https://github.com/apache/shardingsphere/pull/10889#discussion_r657752664



##########
File path: shardingsphere-infra/shardingsphere-infra-optimize/src/main/java/org/apache/shardingsphere/infra/optimize/core/convert/converter/SqlNodeConverterUtil.java
##########
@@ -0,0 +1,63 @@
+/*
+ * 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;
+
+import com.google.common.collect.Lists;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.shardingsphere.infra.optimize.core.convert.converter.impl.ColumnOrderByItemSqlNodeConverter;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.order.item.ColumnOrderByItemSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.order.item.ExpressionOrderByItemSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.order.item.IndexOrderByItemSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.order.item.OrderByItemSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.order.item.TextOrderByItemSegment;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+
+/**
+ * Utility for sql node converter.
+ */
+public final class SqlNodeConverterUtil {
+    
+    /**
+     * Convert order by items.
+     * @param orderByItems order by item list.
+     * @return a collection of order by item <code>SqlNode</code>
+     */
+    public static List<SqlNode> convertOrderByItems(final Collection<OrderByItemSegment> orderByItems) {
+        List<SqlNode> sqlNodes = Lists.newArrayList();

Review comment:
       group by and order by both have OrderByItemSegment list, so I prefer to convert OrderByItemSegment list in a method, or we have to write duplicate `for each` statement.




-- 
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



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

Posted by GitBox <gi...@apache.org>.
guimingyue commented on a change in pull request #10889:
URL: https://github.com/apache/shardingsphere/pull/10889#discussion_r658862843



##########
File path: shardingsphere-infra/shardingsphere-infra-optimize/src/main/java/org/apache/shardingsphere/infra/optimize/core/convert/converter/SqlNodeConverterUtil.java
##########
@@ -0,0 +1,63 @@
+/*
+ * 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;
+
+import com.google.common.collect.Lists;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.shardingsphere.infra.optimize.core.convert.converter.impl.ColumnOrderByItemSqlNodeConverter;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.order.item.ColumnOrderByItemSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.order.item.ExpressionOrderByItemSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.order.item.IndexOrderByItemSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.order.item.OrderByItemSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.order.item.TextOrderByItemSegment;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+
+/**
+ * Utility for sql node converter.
+ */
+public final class SqlNodeConverterUtil {
+    
+    /**
+     * Convert order by items.
+     * @param orderByItems order by item list.
+     * @return a collection of order by item <code>SqlNode</code>
+     */
+    public static List<SqlNode> convertOrderByItems(final Collection<OrderByItemSegment> orderByItems) {
+        List<SqlNode> sqlNodes = Lists.newArrayList();

Review comment:
       By now, there are only two places calling this method, `SqlNodeConverterUtil.convertOrderByItems `. 
   I don't like duplicate code snippets. If a code block should be invoked in more than one method, I prefer to move this code block into a single method in the same class or in another util class. 




-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@shardingsphere.apache.org

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



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

Posted by GitBox <gi...@apache.org>.
guimingyue commented on a change in pull request #10889:
URL: https://github.com/apache/shardingsphere/pull/10889#discussion_r656391039



##########
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:
       done

##########
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:
       done




-- 
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



[GitHub] [shardingsphere] codecov-commenter edited a comment on pull request #10889: Convert ss ast to calcite ast

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #10889:
URL: https://github.com/apache/shardingsphere/pull/10889#issuecomment-864574578


   # [Codecov](https://codecov.io/gh/apache/shardingsphere/pull/10889?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#10889](https://codecov.io/gh/apache/shardingsphere/pull/10889?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (97bb9aa) into [master](https://codecov.io/gh/apache/shardingsphere/commit/f15e1f296e51e6a1cb59bcf344a2103b6a619baf?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (f15e1f2) will **increase** coverage by `0.06%`.
   > The diff coverage is `63.90%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/shardingsphere/pull/10889/graphs/tree.svg?width=650&height=150&src=pr&token=ZvlXpWa7so&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/shardingsphere/pull/10889?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master   #10889      +/-   ##
   ============================================
   + Coverage     65.76%   65.83%   +0.06%     
   - Complexity      710      717       +7     
   ============================================
     Files          1805     1826      +21     
     Lines         30753    30935     +182     
     Branches       5549     5583      +34     
   ============================================
   + Hits          20225    20365     +140     
   - Misses         8927     8953      +26     
   - Partials       1601     1617      +16     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/shardingsphere/pull/10889?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...er/impl/ExpressionOrderByItemSqlNodeConverter.java](https://codecov.io/gh/apache/shardingsphere/pull/10889/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2hhcmRpbmdzcGhlcmUtaW5mcmEvc2hhcmRpbmdzcGhlcmUtaW5mcmEtb3B0aW1pemUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NoYXJkaW5nc3BoZXJlL2luZnJhL29wdGltaXplL2NvcmUvY29udmVydC9jb252ZXJ0ZXIvaW1wbC9FeHByZXNzaW9uT3JkZXJCeUl0ZW1TcWxOb2RlQ29udmVydGVyLmphdmE=) | `0.00% <0.00%> (ø)` | |
   | [...converter/impl/ListExpressionSqlNodeConverter.java](https://codecov.io/gh/apache/shardingsphere/pull/10889/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2hhcmRpbmdzcGhlcmUtaW5mcmEvc2hhcmRpbmdzcGhlcmUtaW5mcmEtb3B0aW1pemUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NoYXJkaW5nc3BoZXJlL2luZnJhL29wdGltaXplL2NvcmUvY29udmVydC9jb252ZXJ0ZXIvaW1wbC9MaXN0RXhwcmVzc2lvblNxbE5vZGVDb252ZXJ0ZXIuamF2YQ==) | `0.00% <0.00%> (ø)` | |
   | [...rt/converter/impl/PaginationValueSqlConverter.java](https://codecov.io/gh/apache/shardingsphere/pull/10889/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2hhcmRpbmdzcGhlcmUtaW5mcmEvc2hhcmRpbmdzcGhlcmUtaW5mcmEtb3B0aW1pemUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NoYXJkaW5nc3BoZXJlL2luZnJhL29wdGltaXplL2NvcmUvY29udmVydC9jb252ZXJ0ZXIvaW1wbC9QYWdpbmF0aW9uVmFsdWVTcWxDb252ZXJ0ZXIuamF2YQ==) | `0.00% <0.00%> (ø)` | |
   | [.../converter/impl/SubqueryTableSqlNodeConverter.java](https://codecov.io/gh/apache/shardingsphere/pull/10889/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2hhcmRpbmdzcGhlcmUtaW5mcmEvc2hhcmRpbmdzcGhlcmUtaW5mcmEtb3B0aW1pemUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NoYXJkaW5nc3BoZXJlL2luZnJhL29wdGltaXplL2NvcmUvY29udmVydC9jb252ZXJ0ZXIvaW1wbC9TdWJxdWVyeVRhYmxlU3FsTm9kZUNvbnZlcnRlci5qYXZh) | `0.00% <0.00%> (ø)` | |
   | [...ert/converter/impl/ExpressionSqlNodeConverter.java](https://codecov.io/gh/apache/shardingsphere/pull/10889/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2hhcmRpbmdzcGhlcmUtaW5mcmEvc2hhcmRpbmdzcGhlcmUtaW5mcmEtb3B0aW1pemUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NoYXJkaW5nc3BoZXJlL2luZnJhL29wdGltaXplL2NvcmUvY29udmVydC9jb252ZXJ0ZXIvaW1wbC9FeHByZXNzaW9uU3FsTm9kZUNvbnZlcnRlci5qYXZh) | `42.85% <42.85%> (ø)` | |
   | [...imize/core/convert/converter/SqlNodeConverter.java](https://codecov.io/gh/apache/shardingsphere/pull/10889/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2hhcmRpbmdzcGhlcmUtaW5mcmEvc2hhcmRpbmdzcGhlcmUtaW5mcmEtb3B0aW1pemUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NoYXJkaW5nc3BoZXJlL2luZnJhL29wdGltaXplL2NvcmUvY29udmVydC9jb252ZXJ0ZXIvU3FsTm9kZUNvbnZlcnRlci5qYXZh) | `46.66% <46.66%> (ø)` | |
   | [...convert/converter/impl/HavingSqlNodeConverter.java](https://codecov.io/gh/apache/shardingsphere/pull/10889/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2hhcmRpbmdzcGhlcmUtaW5mcmEvc2hhcmRpbmdzcGhlcmUtaW5mcmEtb3B0aW1pemUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NoYXJkaW5nc3BoZXJlL2luZnJhL29wdGltaXplL2NvcmUvY29udmVydC9jb252ZXJ0ZXIvaW1wbC9IYXZpbmdTcWxOb2RlQ29udmVydGVyLmphdmE=) | `50.00% <50.00%> (ø)` | |
   | [.../convert/converter/impl/TableSqlNodeConverter.java](https://codecov.io/gh/apache/shardingsphere/pull/10889/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2hhcmRpbmdzcGhlcmUtaW5mcmEvc2hhcmRpbmdzcGhlcmUtaW5mcmEtb3B0aW1pemUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NoYXJkaW5nc3BoZXJlL2luZnJhL29wdGltaXplL2NvcmUvY29udmVydC9jb252ZXJ0ZXIvaW1wbC9UYWJsZVNxbE5vZGVDb252ZXJ0ZXIuamF2YQ==) | `50.00% <50.00%> (ø)` | |
   | [...vert/converter/impl/JoinTableSqlNodeConverter.java](https://codecov.io/gh/apache/shardingsphere/pull/10889/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2hhcmRpbmdzcGhlcmUtaW5mcmEvc2hhcmRpbmdzcGhlcmUtaW5mcmEtb3B0aW1pemUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NoYXJkaW5nc3BoZXJlL2luZnJhL29wdGltaXplL2NvcmUvY29udmVydC9jb252ZXJ0ZXIvaW1wbC9Kb2luVGFibGVTcWxOb2RlQ29udmVydGVyLmphdmE=) | `52.17% <52.17%> (ø)` | |
   | [...onverter/impl/SelectStatementSqlNodeConverter.java](https://codecov.io/gh/apache/shardingsphere/pull/10889/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2hhcmRpbmdzcGhlcmUtaW5mcmEvc2hhcmRpbmdzcGhlcmUtaW5mcmEtb3B0aW1pemUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NoYXJkaW5nc3BoZXJlL2luZnJhL29wdGltaXplL2NvcmUvY29udmVydC9jb252ZXJ0ZXIvaW1wbC9TZWxlY3RTdGF0ZW1lbnRTcWxOb2RlQ29udmVydGVyLmphdmE=) | `67.74% <67.74%> (+67.74%)` | :arrow_up: |
   | ... and [60 more](https://codecov.io/gh/apache/shardingsphere/pull/10889/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/shardingsphere/pull/10889?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/shardingsphere/pull/10889?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [f15e1f2...97bb9aa](https://codecov.io/gh/apache/shardingsphere/pull/10889?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


-- 
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



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

Posted by GitBox <gi...@apache.org>.
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



[GitHub] [shardingsphere] guimingyue edited a comment on pull request #10889: Convert ss ast to calcite ast

Posted by GitBox <gi...@apache.org>.
guimingyue edited a comment on pull request #10889:
URL: https://github.com/apache/shardingsphere/pull/10889#issuecomment-869379872


   > involved
   
   @tristaZero Thanks for your help.
   For the UTs, I think it is not difficult for contributors who are familiar with AST of SQL. 
   And the TODO of these classes is left for that I think we need someone who is good with both antlr and ss ast. I know how to convert ss ast to calcite ast, but it is difficult for me to parse sql text to ss ast.
   BTW, what's your plan and roadmap for integrating with calcite?


-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@shardingsphere.apache.org

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



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

Posted by GitBox <gi...@apache.org>.
tristaZero commented on pull request #10889:
URL: https://github.com/apache/shardingsphere/pull/10889#issuecomment-870151290


   @guimingyue 
   
   ### UT
   That will be fine. Do you think `SelectStatementSqlNodeConverterTest` still needs more assertion?
   
   ### ShardingSphereOptimizer
   Just to get the best physical plan. Please look from `optimize()` of `ShardingSphereOptimizer` which contains parsing, sqlNode converting and optimizing. 
   


-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@shardingsphere.apache.org

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



[GitHub] [shardingsphere] codecov-commenter edited a comment on pull request #10889: Convert ss ast to calcite ast

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #10889:
URL: https://github.com/apache/shardingsphere/pull/10889#issuecomment-864574578


   # [Codecov](https://codecov.io/gh/apache/shardingsphere/pull/10889?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#10889](https://codecov.io/gh/apache/shardingsphere/pull/10889?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (74db509) into [master](https://codecov.io/gh/apache/shardingsphere/commit/915d4572b7ee72a77891e8d130a1a5ca4b20ea04?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (915d457) will **decrease** coverage by `0.26%`.
   > The diff coverage is `70.44%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/shardingsphere/pull/10889/graphs/tree.svg?width=650&height=150&src=pr&token=ZvlXpWa7so&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/shardingsphere/pull/10889?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master   #10889      +/-   ##
   ============================================
   - Coverage     65.61%   65.35%   -0.27%     
   + Complexity      713      704       -9     
   ============================================
     Files          1810     1830      +20     
     Lines         30815    30976     +161     
     Branches       5554     5595      +41     
   ============================================
   + Hits          20219    20243      +24     
   - Misses         9001     9122     +121     
   - Partials       1595     1611      +16     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/shardingsphere/pull/10889?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...er/impl/ExpressionOrderByItemSqlNodeConverter.java](https://codecov.io/gh/apache/shardingsphere/pull/10889/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2hhcmRpbmdzcGhlcmUtaW5mcmEvc2hhcmRpbmdzcGhlcmUtaW5mcmEtb3B0aW1pemUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NoYXJkaW5nc3BoZXJlL2luZnJhL29wdGltaXplL2NvcmUvY29udmVydC9jb252ZXJ0ZXIvaW1wbC9FeHByZXNzaW9uT3JkZXJCeUl0ZW1TcWxOb2RlQ29udmVydGVyLmphdmE=) | `0.00% <0.00%> (ø)` | |
   | [...converter/impl/ListExpressionSqlNodeConverter.java](https://codecov.io/gh/apache/shardingsphere/pull/10889/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2hhcmRpbmdzcGhlcmUtaW5mcmEvc2hhcmRpbmdzcGhlcmUtaW5mcmEtb3B0aW1pemUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NoYXJkaW5nc3BoZXJlL2luZnJhL29wdGltaXplL2NvcmUvY29udmVydC9jb252ZXJ0ZXIvaW1wbC9MaXN0RXhwcmVzc2lvblNxbE5vZGVDb252ZXJ0ZXIuamF2YQ==) | `0.00% <0.00%> (ø)` | |
   | [.../converter/impl/SubqueryTableSqlNodeConverter.java](https://codecov.io/gh/apache/shardingsphere/pull/10889/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2hhcmRpbmdzcGhlcmUtaW5mcmEvc2hhcmRpbmdzcGhlcmUtaW5mcmEtb3B0aW1pemUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NoYXJkaW5nc3BoZXJlL2luZnJhL29wdGltaXplL2NvcmUvY29udmVydC9jb252ZXJ0ZXIvaW1wbC9TdWJxdWVyeVRhYmxlU3FsTm9kZUNvbnZlcnRlci5qYXZh) | `0.00% <0.00%> (ø)` | |
   | [...ert/converter/impl/ExpressionSqlNodeConverter.java](https://codecov.io/gh/apache/shardingsphere/pull/10889/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2hhcmRpbmdzcGhlcmUtaW5mcmEvc2hhcmRpbmdzcGhlcmUtaW5mcmEtb3B0aW1pemUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NoYXJkaW5nc3BoZXJlL2luZnJhL29wdGltaXplL2NvcmUvY29udmVydC9jb252ZXJ0ZXIvaW1wbC9FeHByZXNzaW9uU3FsTm9kZUNvbnZlcnRlci5qYXZh) | `42.85% <42.85%> (ø)` | |
   | [...e/core/convert/converter/SqlNodeConverterUtil.java](https://codecov.io/gh/apache/shardingsphere/pull/10889/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2hhcmRpbmdzcGhlcmUtaW5mcmEvc2hhcmRpbmdzcGhlcmUtaW5mcmEtb3B0aW1pemUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NoYXJkaW5nc3BoZXJlL2luZnJhL29wdGltaXplL2NvcmUvY29udmVydC9jb252ZXJ0ZXIvU3FsTm9kZUNvbnZlcnRlclV0aWwuamF2YQ==) | `43.75% <43.75%> (ø)` | |
   | [...convert/converter/impl/HavingSqlNodeConverter.java](https://codecov.io/gh/apache/shardingsphere/pull/10889/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2hhcmRpbmdzcGhlcmUtaW5mcmEvc2hhcmRpbmdzcGhlcmUtaW5mcmEtb3B0aW1pemUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NoYXJkaW5nc3BoZXJlL2luZnJhL29wdGltaXplL2NvcmUvY29udmVydC9jb252ZXJ0ZXIvaW1wbC9IYXZpbmdTcWxOb2RlQ29udmVydGVyLmphdmE=) | `50.00% <50.00%> (ø)` | |
   | [.../convert/converter/impl/TableSqlNodeConverter.java](https://codecov.io/gh/apache/shardingsphere/pull/10889/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2hhcmRpbmdzcGhlcmUtaW5mcmEvc2hhcmRpbmdzcGhlcmUtaW5mcmEtb3B0aW1pemUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NoYXJkaW5nc3BoZXJlL2luZnJhL29wdGltaXplL2NvcmUvY29udmVydC9jb252ZXJ0ZXIvaW1wbC9UYWJsZVNxbE5vZGVDb252ZXJ0ZXIuamF2YQ==) | `50.00% <50.00%> (ø)` | |
   | [...vert/converter/impl/JoinTableSqlNodeConverter.java](https://codecov.io/gh/apache/shardingsphere/pull/10889/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2hhcmRpbmdzcGhlcmUtaW5mcmEvc2hhcmRpbmdzcGhlcmUtaW5mcmEtb3B0aW1pemUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NoYXJkaW5nc3BoZXJlL2luZnJhL29wdGltaXplL2NvcmUvY29udmVydC9jb252ZXJ0ZXIvaW1wbC9Kb2luVGFibGVTcWxOb2RlQ29udmVydGVyLmphdmE=) | `52.17% <52.17%> (ø)` | |
   | [...verter/impl/LiteralExpressionSqlNodeConverter.java](https://codecov.io/gh/apache/shardingsphere/pull/10889/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2hhcmRpbmdzcGhlcmUtaW5mcmEvc2hhcmRpbmdzcGhlcmUtaW5mcmEtb3B0aW1pemUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NoYXJkaW5nc3BoZXJlL2luZnJhL29wdGltaXplL2NvcmUvY29udmVydC9jb252ZXJ0ZXIvaW1wbC9MaXRlcmFsRXhwcmVzc2lvblNxbE5vZGVDb252ZXJ0ZXIuamF2YQ==) | `71.42% <71.42%> (ø)` | |
   | [...verter/impl/ColumnOrderByItemSqlNodeConverter.java](https://codecov.io/gh/apache/shardingsphere/pull/10889/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c2hhcmRpbmdzcGhlcmUtaW5mcmEvc2hhcmRpbmdzcGhlcmUtaW5mcmEtb3B0aW1pemUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL3NoYXJkaW5nc3BoZXJlL2luZnJhL29wdGltaXplL2NvcmUvY29udmVydC9jb252ZXJ0ZXIvaW1wbC9Db2x1bW5PcmRlckJ5SXRlbVNxbE5vZGVDb252ZXJ0ZXIuamF2YQ==) | `80.00% <80.00%> (ø)` | |
   | ... and [104 more](https://codecov.io/gh/apache/shardingsphere/pull/10889/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/shardingsphere/pull/10889?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/shardingsphere/pull/10889?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [915d457...74db509](https://codecov.io/gh/apache/shardingsphere/pull/10889?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@shardingsphere.apache.org

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



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

Posted by GitBox <gi...@apache.org>.
guimingyue commented on a change in pull request #10889:
URL: https://github.com/apache/shardingsphere/pull/10889#discussion_r659089352



##########
File path: shardingsphere-infra/shardingsphere-infra-optimize/src/main/java/org/apache/shardingsphere/infra/optimize/core/convert/converter/SqlNodeConverterUtil.java
##########
@@ -0,0 +1,63 @@
+/*
+ * 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;
+
+import com.google.common.collect.Lists;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.shardingsphere.infra.optimize.core.convert.converter.impl.ColumnOrderByItemSqlNodeConverter;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.order.item.ColumnOrderByItemSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.order.item.ExpressionOrderByItemSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.order.item.IndexOrderByItemSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.order.item.OrderByItemSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.order.item.TextOrderByItemSegment;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+
+/**
+ * Utility for sql node converter.
+ */
+public final class SqlNodeConverterUtil {
+    
+    /**
+     * Convert order by items.
+     * @param orderByItems order by item list.
+     * @return a collection of order by item <code>SqlNode</code>
+     */
+    public static List<SqlNode> convertOrderByItems(final Collection<OrderByItemSegment> orderByItems) {
+        List<SqlNode> sqlNodes = Lists.newArrayList();

Review comment:
       > Please give a look at the failed ci tests.
   
   @tristaZero  I fixed the failed ci test.




-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@shardingsphere.apache.org

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



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

Posted by GitBox <gi...@apache.org>.
guimingyue commented on a change in pull request #10889:
URL: https://github.com/apache/shardingsphere/pull/10889#discussion_r657186553



##########
File path: shardingsphere-infra/shardingsphere-infra-optimize/src/main/java/org/apache/shardingsphere/infra/optimize/core/convert/converter/SqlNodeConverter.java
##########
@@ -17,20 +17,55 @@
 
 package org.apache.shardingsphere.infra.optimize.core.convert.converter;
 
+import com.google.common.collect.Lists;
 import org.apache.calcite.sql.SqlNode;
+import org.apache.shardingsphere.infra.optimize.core.convert.converter.impl.ColumnOrderByItemSqlNodeConverter;
 import org.apache.shardingsphere.sql.parser.api.visitor.ASTNode;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.order.item.ColumnOrderByItemSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.order.item.ExpressionOrderByItemSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.order.item.IndexOrderByItemSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.order.item.OrderByItemSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.order.item.TextOrderByItemSegment;
 
+import java.util.Collection;
+import java.util.List;
 import java.util.Optional;
 
 /**
  * SqlNode converter.
  */
-public interface SqlNodeConverter<T extends ASTNode> {
+public interface SqlNodeConverter<T extends ASTNode, R extends SqlNode> {
     
     /**
      *  Convert.
      * @param astNode ast node
      * @return sqlNode optional
      */
-    Optional<SqlNode> convert(T astNode);
+    Optional<R> convert(T astNode);
+
+    /**
+     * Convert order by items.
+     * @param orderByItems order by item list.
+     * @return a collection of order by item <code>SqlNode</code>
+     */
+    default List<SqlNode> convertOrderByItems(final Collection<OrderByItemSegment> orderByItems) {

Review comment:
       I placed this method in both abstract super class and util class two days ago,  and use a default method at last but I was not sure. Static util method is much better? 




-- 
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



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

Posted by GitBox <gi...@apache.org>.
tristaZero commented on pull request #10889:
URL: https://github.com/apache/shardingsphere/pull/10889#issuecomment-869406519


   @guimingyue If we add a UT for converter, which assertions are expected? 
   
   ```java
   assertEquals(2, sqlSelect.getSelectList().size());
   assertNull(sqlSelect.getWhere());
   assertThat(optional.get(), instanceOf(SqlSelect.class))
   ```
   Do you think these items are enough?
   
   Could you firstly create a few UT issues for these `converter`, like #10904? We can assign theses issues you created to other contributors.


-- 
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.

To unsubscribe, e-mail: notifications-unsubscribe@shardingsphere.apache.org

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



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

Posted by GitBox <gi...@apache.org>.
guimingyue commented on a change in pull request #10889:
URL: https://github.com/apache/shardingsphere/pull/10889#discussion_r656392204



##########
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:
       I tried, but failed. There should be some code snippet to handle nullable of LimitSegment.




-- 
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



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

Posted by GitBox <gi...@apache.org>.
guimingyue commented on a change in pull request #10889:
URL: https://github.com/apache/shardingsphere/pull/10889#discussion_r657755900



##########
File path: shardingsphere-infra/shardingsphere-infra-optimize/src/main/java/org/apache/shardingsphere/infra/optimize/core/convert/converter/impl/AbstractLimitSqlNodeConverter.java
##########
@@ -0,0 +1,50 @@
+/*
+ * 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.pagination.PaginationValueSegment;
+import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.pagination.limit.LimitSegment;
+
+import java.util.Optional;
+import java.util.function.Function;
+
+/**
+ * Abstract limit sql node converter.
+ */
+public abstract class AbstractLimitSqlNodeConverter implements SqlNodeConverter<LimitSegment, SqlNode> {
+
+    private Function<LimitSegment, Optional<PaginationValueSegment>> function;

Review comment:
       This `Function` method is used to get `PaginationValueSegment` object in `AbstractLimitSqlNodeConverter.convert` method. I think this can simplify the `OffsetSqlNodeConverter` and `RowCountSqlNodeConverter`. And the invoker of `OffsetSqlNodeConverter` and `RowCountSqlNodeConverter` does not have do any extra work.




-- 
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