PHILO-HE commented on code in PR #9893: URL: https://github.com/apache/incubator-gluten/pull/9893#discussion_r2154032951
########## gluten-flink/ut/src/test/java/org/apache/gluten/table/runtime/stream/custom/NexmarkTest.java: ########## @@ -0,0 +1,158 @@ +/* + * 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.gluten.table.runtime.stream.custom; + +import org.apache.gluten.table.runtime.stream.common.Velox4jEnvironment; + +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.TableResult; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; + +import org.junit.jupiter.api.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.net.URISyntaxException; +import java.net.URL; +import java.nio.file.DirectoryStream; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +import static org.assertj.core.api.Assertions.assertThat; + +public class NexmarkTest { + + private static final Logger LOG = LoggerFactory.getLogger(NexmarkTest.class); + private static final String NEXMARK_RESOURCE_DIR = "nexmark"; + + @Test + void testAllNexmarkQueries() { + LOG.info("NexmarkTest setup"); + Velox4jEnvironment.initializeOnce(); + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); + + EnvironmentSettings settings = EnvironmentSettings.newInstance().inStreamingMode().build(); + StreamTableEnvironment tEnv = StreamTableEnvironment.create(env, settings); + + setupNexmarkEnvironment(tEnv); + + List<String> queryFiles = discoverQueryFiles(); + assertThat(queryFiles).isNotEmpty(); + + LOG.info("Found {} Nexmark query files: {}", queryFiles.size(), queryFiles); + + for (String queryFile : queryFiles) { + LOG.info("Executing query from file: {}", queryFile); + executeNexmarkQuery(tEnv, queryFile); + } + } + + private void setupNexmarkEnvironment(StreamTableEnvironment tEnv) { + Map<String, String> variables = createNexmarkVariables(); + String createNexmarkSource = readSqlFromFile(NEXMARK_RESOURCE_DIR + "/ddl_gen.sql"); + createNexmarkSource = replaceVariables(createNexmarkSource, variables); + tEnv.executeSql(createNexmarkSource); + + String createTableView = readSqlFromFile(NEXMARK_RESOURCE_DIR + "/ddl_views.sql"); + String[] sqlTableView = createTableView.split(";"); + for (String sql : sqlTableView) { + String trimmedSql = sql.trim(); + if (!trimmedSql.isEmpty()) { + tEnv.executeSql(trimmedSql); + } + } + } + + private Map<String, String> createNexmarkVariables() { + Map<String, String> variables = new HashMap<>(); + variables.put("TPS", "10"); + variables.put("EVENTS_NUM", "100"); + variables.put("PERSON_PROPORTION", "1"); + variables.put("AUCTION_PROPORTION", "3"); + variables.put("BID_PROPORTION", "46"); Review Comment: Seems we can directly create a static map variable for this class and initialize it when declaring? ########## gluten-flink/runtime/src/main/java/org/apache/gluten/vectorized/ArrowVectorAccessor.java: ########## @@ -244,3 +247,17 @@ protected Object getImpl(int rowIndex) { return new GenericMapData(mapEntries); } } + +class TimeStampMicroVectorAccessor extends BaseArrowVectorAccessor<TimeStampMicroVector> { + + public TimeStampMicroVectorAccessor(FieldVector vector) { + super(vector); + } + + @Override + public Object getImpl(int rowIndex) { + long microseconds = typedVector.get(rowIndex); + long milliseconds = microseconds / 1000; Review Comment: Nit: combine the above two lines. ########## gluten-flink/ut/src/test/java/org/apache/gluten/table/runtime/stream/custom/NexmarkTest.java: ########## @@ -0,0 +1,158 @@ +/* + * 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.gluten.table.runtime.stream.custom; + +import org.apache.gluten.table.runtime.stream.common.Velox4jEnvironment; + +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.TableResult; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; + +import org.junit.jupiter.api.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.net.URISyntaxException; +import java.net.URL; +import java.nio.file.DirectoryStream; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +import static org.assertj.core.api.Assertions.assertThat; + +public class NexmarkTest { + + private static final Logger LOG = LoggerFactory.getLogger(NexmarkTest.class); + private static final String NEXMARK_RESOURCE_DIR = "nexmark"; + + @Test + void testAllNexmarkQueries() { + LOG.info("NexmarkTest setup"); + Velox4jEnvironment.initializeOnce(); + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); + + EnvironmentSettings settings = EnvironmentSettings.newInstance().inStreamingMode().build(); + StreamTableEnvironment tEnv = StreamTableEnvironment.create(env, settings); + + setupNexmarkEnvironment(tEnv); + + List<String> queryFiles = discoverQueryFiles(); + assertThat(queryFiles).isNotEmpty(); + + LOG.info("Found {} Nexmark query files: {}", queryFiles.size(), queryFiles); + + for (String queryFile : queryFiles) { + LOG.info("Executing query from file: {}", queryFile); + executeNexmarkQuery(tEnv, queryFile); + } + } + + private void setupNexmarkEnvironment(StreamTableEnvironment tEnv) { + Map<String, String> variables = createNexmarkVariables(); + String createNexmarkSource = readSqlFromFile(NEXMARK_RESOURCE_DIR + "/ddl_gen.sql"); + createNexmarkSource = replaceVariables(createNexmarkSource, variables); + tEnv.executeSql(createNexmarkSource); + + String createTableView = readSqlFromFile(NEXMARK_RESOURCE_DIR + "/ddl_views.sql"); + String[] sqlTableView = createTableView.split(";"); + for (String sql : sqlTableView) { + String trimmedSql = sql.trim(); + if (!trimmedSql.isEmpty()) { + tEnv.executeSql(trimmedSql); + } + } + } + + private Map<String, String> createNexmarkVariables() { + Map<String, String> variables = new HashMap<>(); + variables.put("TPS", "10"); + variables.put("EVENTS_NUM", "100"); + variables.put("PERSON_PROPORTION", "1"); + variables.put("AUCTION_PROPORTION", "3"); + variables.put("BID_PROPORTION", "46"); + return variables; + } + + private String replaceVariables(String sql, Map<String, String> variables) { + String result = sql; + for (Map.Entry<String, String> entry : variables.entrySet()) { + result = result.replace("${" + entry.getKey() + "}", entry.getValue()); + } + return result; + } + + private void executeNexmarkQuery(StreamTableEnvironment tEnv, String queryFileName) { Review Comment: name suggestion for simplicity: executeQuery. ########## gluten-flink/ut/pom.xml: ########## @@ -90,6 +90,11 @@ <artifactId>flink-clients</artifactId> <version>${flink.version}</version> </dependency> + <dependency> + <groupId>com.github.nexmark</groupId> + <artifactId>nexmark-flink</artifactId> + <version>0.3-SNAPSHOT</version> Review Comment: Should we also add test scope for junit-jupiter-api, etc? ########## gluten-flink/planner/src/main/java/org/apache/gluten/rexnode/functions/BaseRexCallConverters.java: ########## @@ -84,3 +86,30 @@ public TypedExpr toTypedExpr(RexCall callNode, RexConversionContext context) { return new CallTypedExpr(resultType, alignedParams, functionName); } } + +class SubtractRexCallConverter extends BaseRexCallConverter { + + public SubtractRexCallConverter() { + super("subtract"); + } + + @Override + public TypedExpr toTypedExpr(RexCall callNode, RexConversionContext context) { + List<TypedExpr> params = getParams(callNode, context); + + if (params.size() == 2 + && params.get(0).getReturnType() instanceof TimestampType + && params.get(1).getReturnType() instanceof BigIntType) { + + Type bigIntType = new BigIntType(); + TypedExpr castedParam0 = new CallTypedExpr(bigIntType, List.of(params.get(0)), "cast"); Review Comment: name suggestion: castedParam0->castExpr ########## gluten-flink/ut/src/test/java/org/apache/gluten/table/runtime/stream/custom/NexmarkTest.java: ########## @@ -0,0 +1,158 @@ +/* + * 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.gluten.table.runtime.stream.custom; + +import org.apache.gluten.table.runtime.stream.common.Velox4jEnvironment; + +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.TableResult; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; + +import org.junit.jupiter.api.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.net.URISyntaxException; +import java.net.URL; +import java.nio.file.DirectoryStream; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +import static org.assertj.core.api.Assertions.assertThat; + +public class NexmarkTest { + + private static final Logger LOG = LoggerFactory.getLogger(NexmarkTest.class); + private static final String NEXMARK_RESOURCE_DIR = "nexmark"; + + @Test + void testAllNexmarkQueries() { + LOG.info("NexmarkTest setup"); + Velox4jEnvironment.initializeOnce(); + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); + + EnvironmentSettings settings = EnvironmentSettings.newInstance().inStreamingMode().build(); + StreamTableEnvironment tEnv = StreamTableEnvironment.create(env, settings); + + setupNexmarkEnvironment(tEnv); + + List<String> queryFiles = discoverQueryFiles(); + assertThat(queryFiles).isNotEmpty(); + + LOG.info("Found {} Nexmark query files: {}", queryFiles.size(), queryFiles); + + for (String queryFile : queryFiles) { + LOG.info("Executing query from file: {}", queryFile); + executeNexmarkQuery(tEnv, queryFile); + } + } + + private void setupNexmarkEnvironment(StreamTableEnvironment tEnv) { + Map<String, String> variables = createNexmarkVariables(); + String createNexmarkSource = readSqlFromFile(NEXMARK_RESOURCE_DIR + "/ddl_gen.sql"); + createNexmarkSource = replaceVariables(createNexmarkSource, variables); + tEnv.executeSql(createNexmarkSource); + + String createTableView = readSqlFromFile(NEXMARK_RESOURCE_DIR + "/ddl_views.sql"); + String[] sqlTableView = createTableView.split(";"); + for (String sql : sqlTableView) { + String trimmedSql = sql.trim(); + if (!trimmedSql.isEmpty()) { + tEnv.executeSql(trimmedSql); + } + } + } + + private Map<String, String> createNexmarkVariables() { + Map<String, String> variables = new HashMap<>(); + variables.put("TPS", "10"); + variables.put("EVENTS_NUM", "100"); + variables.put("PERSON_PROPORTION", "1"); + variables.put("AUCTION_PROPORTION", "3"); + variables.put("BID_PROPORTION", "46"); + return variables; + } + + private String replaceVariables(String sql, Map<String, String> variables) { + String result = sql; + for (Map.Entry<String, String> entry : variables.entrySet()) { + result = result.replace("${" + entry.getKey() + "}", entry.getValue()); + } + return result; + } + + private void executeNexmarkQuery(StreamTableEnvironment tEnv, String queryFileName) { + String queryContent = readSqlFromFile(NEXMARK_RESOURCE_DIR + "/" + queryFileName); + + String[] sqlStatements = queryContent.split(";"); + assertThat(sqlStatements.length).isGreaterThanOrEqualTo(2); + + String createResultTable = sqlStatements[0].trim(); + if (!createResultTable.isEmpty()) { + TableResult createResult = tEnv.executeSql(createResultTable); + assertThat(createResult.getJobClient().isPresent()).isFalse(); + } + + String insertQuery = sqlStatements[1].trim(); + if (!insertQuery.isEmpty()) { + TableResult insertResult = tEnv.executeSql(insertQuery); + assertThat(insertResult.getJobClient().isPresent()).isTrue(); + } + } + + private List<String> discoverQueryFiles() { Review Comment: name suggestion for simplicity: getQueries ########## gluten-flink/planner/src/main/java/org/apache/gluten/rexnode/functions/BaseRexCallConverters.java: ########## @@ -84,3 +86,30 @@ public TypedExpr toTypedExpr(RexCall callNode, RexConversionContext context) { return new CallTypedExpr(resultType, alignedParams, functionName); } } + +class SubtractRexCallConverter extends BaseRexCallConverter { + + public SubtractRexCallConverter() { + super("subtract"); + } + + @Override + public TypedExpr toTypedExpr(RexCall callNode, RexConversionContext context) { + List<TypedExpr> params = getParams(callNode, context); + + if (params.size() == 2 Review Comment: Seems the `subtract` expression can only have two inputs? Do we need this check? ########## gluten-flink/ut/src/test/java/org/apache/gluten/table/runtime/stream/custom/NexmarkTest.java: ########## @@ -0,0 +1,158 @@ +/* + * 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.gluten.table.runtime.stream.custom; + +import org.apache.gluten.table.runtime.stream.common.Velox4jEnvironment; + +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.TableResult; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; + +import org.junit.jupiter.api.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.net.URISyntaxException; +import java.net.URL; +import java.nio.file.DirectoryStream; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +import static org.assertj.core.api.Assertions.assertThat; + +public class NexmarkTest { + + private static final Logger LOG = LoggerFactory.getLogger(NexmarkTest.class); + private static final String NEXMARK_RESOURCE_DIR = "nexmark"; + + @Test + void testAllNexmarkQueries() { + LOG.info("NexmarkTest setup"); + Velox4jEnvironment.initializeOnce(); + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); + + EnvironmentSettings settings = EnvironmentSettings.newInstance().inStreamingMode().build(); + StreamTableEnvironment tEnv = StreamTableEnvironment.create(env, settings); + + setupNexmarkEnvironment(tEnv); Review Comment: Can we move the above setup to a static method called `setup` with `@BeforeAll` annotation? -- 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: [email protected] For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
