diff --git a/dinky-client/dinky-client-1.14/src/main/java/org/dinky/executor/CustomTableEnvironmentImpl.java b/dinky-client/dinky-client-1.14/src/main/java/org/dinky/executor/CustomTableEnvironmentImpl.java index 6ee1e92fec..8671fb9f81 100644 --- a/dinky-client/dinky-client-1.14/src/main/java/org/dinky/executor/CustomTableEnvironmentImpl.java +++ b/dinky-client/dinky-client-1.14/src/main/java/org/dinky/executor/CustomTableEnvironmentImpl.java @@ -22,6 +22,7 @@ import org.dinky.assertion.Asserts; import org.dinky.data.model.LineageRel; import org.dinky.data.result.SqlExplainResult; +import org.dinky.parser.CustomParserImpl; import org.dinky.utils.JsonUtils; import org.dinky.utils.LineageContext; @@ -119,6 +120,8 @@ public CustomTableEnvironmentImpl( isStreamingMode, userClassLoader)); this.executor = executor; + injectParser(new CustomParserImpl(getPlanner().getParser())); + injectExtendedExecutor(new CustomExtendedOperationExecutorImpl(this)); } public static CustomTableEnvironmentImpl create( diff --git a/dinky-client/dinky-client-1.15/src/main/java/org/dinky/executor/CustomTableEnvironmentImpl.java b/dinky-client/dinky-client-1.15/src/main/java/org/dinky/executor/CustomTableEnvironmentImpl.java index 04f88810b5..f46520a374 100644 --- a/dinky-client/dinky-client-1.15/src/main/java/org/dinky/executor/CustomTableEnvironmentImpl.java +++ b/dinky-client/dinky-client-1.15/src/main/java/org/dinky/executor/CustomTableEnvironmentImpl.java @@ -24,6 +24,7 @@ import org.dinky.assertion.Asserts; import org.dinky.data.model.LineageRel; import org.dinky.data.result.SqlExplainResult; +import org.dinky.parser.CustomParserImpl; import org.dinky.utils.LineageContext; import org.apache.flink.api.common.RuntimeExecutionMode; @@ -108,6 +109,8 @@ public CustomTableEnvironmentImpl( executor, isStreamingMode, userClassLoader)); + injectParser(new CustomParserImpl(getPlanner().getParser())); + injectExtendedExecutor(new CustomExtendedOperationExecutorImpl(this)); } public static CustomTableEnvironmentImpl create( diff --git a/dinky-client/dinky-client-1.16/src/main/java/org/dinky/executor/CustomTableEnvironmentImpl.java b/dinky-client/dinky-client-1.16/src/main/java/org/dinky/executor/CustomTableEnvironmentImpl.java index f1ad9ce279..a69349e489 100644 --- a/dinky-client/dinky-client-1.16/src/main/java/org/dinky/executor/CustomTableEnvironmentImpl.java +++ b/dinky-client/dinky-client-1.16/src/main/java/org/dinky/executor/CustomTableEnvironmentImpl.java @@ -22,6 +22,7 @@ import org.dinky.assertion.Asserts; import org.dinky.data.model.LineageRel; import org.dinky.data.result.SqlExplainResult; +import org.dinky.parser.CustomParserImpl; import org.dinky.trans.ddl.CustomSetOperation; import org.dinky.utils.JsonUtils; import org.dinky.utils.LineageContext; @@ -76,6 +77,8 @@ public class CustomTableEnvironmentImpl extends AbstractCustomTableEnvironment { public CustomTableEnvironmentImpl(StreamTableEnvironment streamTableEnvironment) { super(streamTableEnvironment); + injectParser(new CustomParserImpl(getPlanner().getParser())); + injectExtendedExecutor(new CustomExtendedOperationExecutorImpl(this)); } public static CustomTableEnvironmentImpl create( diff --git a/dinky-client/dinky-client-1.17/src/main/java/org/dinky/executor/CustomTableEnvironmentImpl.java b/dinky-client/dinky-client-1.17/src/main/java/org/dinky/executor/CustomTableEnvironmentImpl.java index 52782c94b4..4d8a1bd9c9 100644 --- a/dinky-client/dinky-client-1.17/src/main/java/org/dinky/executor/CustomTableEnvironmentImpl.java +++ b/dinky-client/dinky-client-1.17/src/main/java/org/dinky/executor/CustomTableEnvironmentImpl.java @@ -22,6 +22,7 @@ import org.dinky.assertion.Asserts; import org.dinky.data.model.LineageRel; import org.dinky.data.result.SqlExplainResult; +import org.dinky.parser.CustomParserImpl; import org.dinky.utils.LineageContext; import org.apache.flink.api.dag.Transformation; @@ -79,6 +80,8 @@ public class CustomTableEnvironmentImpl extends AbstractCustomTableEnvironment { public CustomTableEnvironmentImpl(StreamTableEnvironment streamTableEnvironment) { super(streamTableEnvironment); + injectParser(new CustomParserImpl(getPlanner().getParser())); + injectExtendedExecutor(new CustomExtendedOperationExecutorImpl(this)); } public static CustomTableEnvironmentImpl create( diff --git a/dinky-client/dinky-client-1.18/src/main/java/org/dinky/executor/CustomTableEnvironmentImpl.java b/dinky-client/dinky-client-1.18/src/main/java/org/dinky/executor/CustomTableEnvironmentImpl.java index 33c62ced62..a9aa8dcc36 100644 --- a/dinky-client/dinky-client-1.18/src/main/java/org/dinky/executor/CustomTableEnvironmentImpl.java +++ b/dinky-client/dinky-client-1.18/src/main/java/org/dinky/executor/CustomTableEnvironmentImpl.java @@ -22,6 +22,7 @@ import org.dinky.assertion.Asserts; import org.dinky.data.model.LineageRel; import org.dinky.data.result.SqlExplainResult; +import org.dinky.operations.CustomNewParserImpl; import org.dinky.utils.LineageContext; import org.apache.flink.api.dag.Transformation; @@ -81,6 +82,7 @@ public class CustomTableEnvironmentImpl extends AbstractCustomTableEnvironment { public CustomTableEnvironmentImpl(StreamTableEnvironment streamTableEnvironment) { super(streamTableEnvironment); + injectParser(new CustomNewParserImpl(this, getPlanner().getParser())); } public static CustomTableEnvironmentImpl create( diff --git a/dinky-client/dinky-client-1.18/src/main/java/org/dinky/operations/CustomNewParserImpl.java b/dinky-client/dinky-client-1.18/src/main/java/org/dinky/operations/CustomNewParserImpl.java new file mode 100644 index 0000000000..58b8099d2e --- /dev/null +++ b/dinky-client/dinky-client-1.18/src/main/java/org/dinky/operations/CustomNewParserImpl.java @@ -0,0 +1,41 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.dinky.operations; + +import org.dinky.parser.CustomParserImpl; + +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.delegation.Parser; +import org.apache.flink.table.planner.parse.ExtendedParser; + +public class CustomNewParserImpl extends CustomParserImpl { + + private final DinkyParser dinkyParser; + + public CustomNewParserImpl(TableEnvironment tableEnvironment, Parser parser) { + super(parser); + this.dinkyParser = new DinkyParser(tableEnvironment); + } + + @Override + public ExtendedParser getDinkyParser() { + return this.dinkyParser; + } +} diff --git a/dinky-client/dinky-client-1.18/src/main/java/org/dinky/operations/DinkyExecutableOperation.java b/dinky-client/dinky-client-1.18/src/main/java/org/dinky/operations/DinkyExecutableOperation.java new file mode 100644 index 0000000000..f30641265d --- /dev/null +++ b/dinky-client/dinky-client-1.18/src/main/java/org/dinky/operations/DinkyExecutableOperation.java @@ -0,0 +1,51 @@ +/* + * + * 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.dinky.operations; + +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.internal.TableResultInternal; +import org.apache.flink.table.operations.ExecutableOperation; +import org.apache.flink.table.operations.Operation; + +public class DinkyExecutableOperation implements ExecutableOperation { + + private final Operation innerOperation; + private final TableEnvironment tableEnvironment; + + public DinkyExecutableOperation(TableEnvironment tableEnvironment, Operation innerOperation) { + this.tableEnvironment = tableEnvironment; + this.innerOperation = innerOperation; + } + + @Override + public TableResultInternal execute(Context ctx) { + DinkyOperationExecutor operationExecutor = new DinkyOperationExecutor(tableEnvironment, ctx); + return operationExecutor.executeOperation(innerOperation).get(); + } + + public Operation getInnerOperation() { + return innerOperation; + } + + @Override + public String asSummaryString() { + return innerOperation.asSummaryString(); + } +} diff --git a/dinky-client/dinky-client-1.18/src/main/java/org/dinky/operations/DinkyOperationExecutor.java b/dinky-client/dinky-client-1.18/src/main/java/org/dinky/operations/DinkyOperationExecutor.java new file mode 100644 index 0000000000..4892fea153 --- /dev/null +++ b/dinky-client/dinky-client-1.18/src/main/java/org/dinky/operations/DinkyOperationExecutor.java @@ -0,0 +1,48 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.dinky.operations; + +import org.dinky.executor.CustomTableEnvironment; +import org.dinky.trans.ExtendOperation; + +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.internal.TableResultInternal; +import org.apache.flink.table.operations.ExecutableOperation; +import org.apache.flink.table.operations.Operation; + +import java.util.Optional; + +public class DinkyOperationExecutor { + private final ExecutableOperation.Context context; + + private final TableEnvironment tableEnvironment; + + public DinkyOperationExecutor(TableEnvironment tableEnvironment, ExecutableOperation.Context context) { + this.tableEnvironment = tableEnvironment; + this.context = context; + } + + public Optional executeOperation(Operation operation) { + ExtendOperation extendOperation = (ExtendOperation) operation; + return Optional.of((TableResultInternal) extendOperation + .execute((CustomTableEnvironment) tableEnvironment) + .get()); + } +} diff --git a/dinky-client/dinky-client-1.18/src/main/java/org/dinky/operations/DinkyParser.java b/dinky-client/dinky-client-1.18/src/main/java/org/dinky/operations/DinkyParser.java new file mode 100644 index 0000000000..94ec72e4ae --- /dev/null +++ b/dinky-client/dinky-client-1.18/src/main/java/org/dinky/operations/DinkyParser.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.dinky.operations; + +import org.dinky.parser.DinkyExtendedParser; + +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.operations.Operation; +import org.apache.flink.table.planner.parse.ExtendedParseStrategy; + +import java.util.Optional; + +public class DinkyParser extends DinkyExtendedParser { + private final TableEnvironment tableEnvironment; + + public DinkyParser(TableEnvironment tableEnvironment) { + this.tableEnvironment = tableEnvironment; + } + + @Override + public Optional parse(String statement) { + for (ExtendedParseStrategy strategy : PARSE_STRATEGIES) { + if (strategy.match(statement)) { + return Optional.of(new DinkyExecutableOperation(this.tableEnvironment, strategy.convert(statement))); + } + } + return Optional.empty(); + } +} diff --git a/dinky-core/src/main/java/org/dinky/executor/CustomExtendedOperationExecutorImpl.java b/dinky-client/dinky-client-base/src/main/java/org/dinky/executor/CustomExtendedOperationExecutorImpl.java similarity index 100% rename from dinky-core/src/main/java/org/dinky/executor/CustomExtendedOperationExecutorImpl.java rename to dinky-client/dinky-client-base/src/main/java/org/dinky/executor/CustomExtendedOperationExecutorImpl.java diff --git a/dinky-client/dinky-client-base/src/main/java/org/dinky/executor/CustomParser.java b/dinky-client/dinky-client-base/src/main/java/org/dinky/executor/CustomParser.java index ad2a896851..c872f94c4b 100644 --- a/dinky-client/dinky-client-base/src/main/java/org/dinky/executor/CustomParser.java +++ b/dinky-client/dinky-client-base/src/main/java/org/dinky/executor/CustomParser.java @@ -22,6 +22,7 @@ import org.apache.calcite.sql.SqlNode; import org.apache.flink.table.delegation.Parser; import org.apache.flink.table.operations.Operation; +import org.apache.flink.table.planner.parse.ExtendedParser; import java.util.List; @@ -30,6 +31,8 @@ public interface CustomParser { Parser getParser(); + ExtendedParser getDinkyParser(); + SqlNode parseExpression(String sqlExpression); /** diff --git a/dinky-client/dinky-client-base/src/main/java/org/dinky/parser/CustomParserImpl.java b/dinky-client/dinky-client-base/src/main/java/org/dinky/parser/CustomParserImpl.java index b96df52f8c..964baf4b3a 100644 --- a/dinky-client/dinky-client-base/src/main/java/org/dinky/parser/CustomParserImpl.java +++ b/dinky-client/dinky-client-base/src/main/java/org/dinky/parser/CustomParserImpl.java @@ -20,10 +20,6 @@ package org.dinky.parser; import org.dinky.executor.CustomParser; -import org.dinky.trans.parse.AddJarSqlParseStrategy; -import org.dinky.trans.parse.CreateAggTableSelectSqlParseStrategy; -import org.dinky.trans.parse.CreateTemporalTableFunctionParseStrategy; -import org.dinky.trans.parse.SetSqlParseStrategy; import org.apache.calcite.sql.SqlNode; import org.apache.flink.table.delegation.Parser; @@ -31,10 +27,8 @@ import org.apache.flink.table.planner.calcite.FlinkPlannerImpl; import org.apache.flink.table.planner.delegation.ParserImpl; import org.apache.flink.table.planner.parse.CalciteParser; -import org.apache.flink.table.planner.parse.ExtendedParseStrategy; import org.apache.flink.table.planner.parse.ExtendedParser; -import java.util.Arrays; import java.util.Collections; import java.util.List; import java.util.Optional; @@ -44,7 +38,6 @@ public class CustomParserImpl implements CustomParser { - private static final DinkyExtendedParser DINKY_EXTENDED_PARSER = DinkyExtendedParser.INSTANCE; private final Parser parser; private final Supplier validatorSupplier; private final Supplier calciteParserSupplier; @@ -77,7 +70,7 @@ public static Supplier getValidatorSupplier(Parser parser) { @Override public List parse(String statement) { - Optional command = DINKY_EXTENDED_PARSER.parse(statement); + Optional command = getDinkyParser().parse(statement); // note: null represent not custom parser; return command.map(Collections::singletonList).orElse(null); @@ -88,6 +81,11 @@ public Parser getParser() { return parser; } + @Override + public ExtendedParser getDinkyParser() { + return DinkyExtendedParser.INSTANCE; + } + @Override public SqlNode parseExpression(String sqlExpression) { return calciteParserSupplier.get().parseExpression(sqlExpression); @@ -103,24 +101,4 @@ public SqlNode validate(SqlNode sqlNode) { FlinkPlannerImpl flinkPlanner = validatorSupplier.get(); return flinkPlanner.validate(sqlNode); } - - public static class DinkyExtendedParser extends ExtendedParser { - public static final DinkyExtendedParser INSTANCE = new DinkyExtendedParser(); - - private static final List PARSE_STRATEGIES = Arrays.asList( - AddJarSqlParseStrategy.INSTANCE, - CreateAggTableSelectSqlParseStrategy.INSTANCE, - SetSqlParseStrategy.INSTANCE, - CreateTemporalTableFunctionParseStrategy.INSTANCE); - - @Override - public Optional parse(String statement) { - for (ExtendedParseStrategy strategy : PARSE_STRATEGIES) { - if (strategy.match(statement)) { - return Optional.of(strategy.convert(statement)); - } - } - return Optional.empty(); - } - } } diff --git a/dinky-client/dinky-client-base/src/main/java/org/dinky/parser/DinkyExtendedParser.java b/dinky-client/dinky-client-base/src/main/java/org/dinky/parser/DinkyExtendedParser.java new file mode 100644 index 0000000000..119cc1cb5e --- /dev/null +++ b/dinky-client/dinky-client-base/src/main/java/org/dinky/parser/DinkyExtendedParser.java @@ -0,0 +1,53 @@ +/* + * + * 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.dinky.parser; + +import org.dinky.trans.parse.AddJarSqlParseStrategy; +import org.dinky.trans.parse.CreateAggTableSelectSqlParseStrategy; +import org.dinky.trans.parse.CreateTemporalTableFunctionParseStrategy; +import org.dinky.trans.parse.SetSqlParseStrategy; + +import org.apache.flink.table.operations.Operation; +import org.apache.flink.table.planner.parse.ExtendedParseStrategy; +import org.apache.flink.table.planner.parse.ExtendedParser; + +import java.util.Arrays; +import java.util.List; +import java.util.Optional; + +public class DinkyExtendedParser extends ExtendedParser { + public static final DinkyExtendedParser INSTANCE = new DinkyExtendedParser(); + + public static final List PARSE_STRATEGIES = Arrays.asList( + AddJarSqlParseStrategy.INSTANCE, + CreateAggTableSelectSqlParseStrategy.INSTANCE, + SetSqlParseStrategy.INSTANCE, + CreateTemporalTableFunctionParseStrategy.INSTANCE); + + @Override + public Optional parse(String statement) { + for (ExtendedParseStrategy strategy : PARSE_STRATEGIES) { + if (strategy.match(statement)) { + return Optional.of(strategy.convert(statement)); + } + } + return Optional.empty(); + } +} diff --git a/dinky-core/src/main/java/org/dinky/executor/Executor.java b/dinky-core/src/main/java/org/dinky/executor/Executor.java index a813ea5cd6..6dbdd904df 100644 --- a/dinky-core/src/main/java/org/dinky/executor/Executor.java +++ b/dinky-core/src/main/java/org/dinky/executor/Executor.java @@ -26,7 +26,6 @@ import org.dinky.data.result.SqlExplainResult; import org.dinky.interceptor.FlinkInterceptor; import org.dinky.interceptor.FlinkInterceptorResult; -import org.dinky.parser.CustomParserImpl; import org.dinky.utils.KerberosUtil; import org.apache.flink.api.common.ExecutionConfig; @@ -140,12 +139,6 @@ protected void init(DinkyClassLoader classLoader) { tableEnvironment = createCustomTableEnvironment(classLoader); CustomTableEnvironmentContext.set(tableEnvironment); - // after 1.18 version, this two injection should be removed - tableEnvironment.injectParser( - new CustomParserImpl(tableEnvironment.getPlanner().getParser())); - tableEnvironment.injectExtendedExecutor( - new CustomExtendedOperationExecutorImpl(this.getCustomTableEnvironment())); - Configuration configuration = tableEnvironment.getConfig().getConfiguration(); if (executorConfig.isValidJobName()) { configuration.setString(PipelineOptions.NAME.key(), executorConfig.getJobName()); diff --git a/dinky-core/src/main/java/org/dinky/interceptor/FlinkInterceptor.java b/dinky-core/src/main/java/org/dinky/interceptor/FlinkInterceptor.java index b54a6ea0ee..5d8ddb7e74 100644 --- a/dinky-core/src/main/java/org/dinky/interceptor/FlinkInterceptor.java +++ b/dinky-core/src/main/java/org/dinky/interceptor/FlinkInterceptor.java @@ -50,7 +50,7 @@ public static FlinkInterceptorResult build(Executor executor, String statement) TableResult tableResult = null; Operation operation = Operations.buildOperation(statement); if (Asserts.isNotNull(operation)) { - tableResult = operation.build(executor); + tableResult = operation.execute(executor); noExecute = operation.noExecute(); } return FlinkInterceptorResult.build(noExecute, tableResult); diff --git a/dinky-core/src/main/java/org/dinky/trans/Operation.java b/dinky-core/src/main/java/org/dinky/trans/Operation.java index 0792d28df8..d477a62749 100644 --- a/dinky-core/src/main/java/org/dinky/trans/Operation.java +++ b/dinky-core/src/main/java/org/dinky/trans/Operation.java @@ -34,7 +34,7 @@ public interface Operation extends org.apache.flink.table.operations.Operation { Operation create(String statement); - TableResult build(Executor executor); + TableResult execute(Executor executor); boolean noExecute(); diff --git a/dinky-core/src/main/java/org/dinky/trans/ddl/CreateCDCSourceOperation.java b/dinky-core/src/main/java/org/dinky/trans/ddl/CreateCDCSourceOperation.java index 9c25ee7293..a04352b7b6 100644 --- a/dinky-core/src/main/java/org/dinky/trans/ddl/CreateCDCSourceOperation.java +++ b/dinky-core/src/main/java/org/dinky/trans/ddl/CreateCDCSourceOperation.java @@ -72,7 +72,7 @@ public Operation create(String statement) { } @Override - public TableResult build(Executor executor) { + public TableResult execute(Executor executor) { logger.info("Start build CDCSOURCE Task..."); CDCSource cdcSource = CDCSource.build(statement); FlinkCDCConfig config = new FlinkCDCConfig( diff --git a/dinky-core/src/main/java/org/dinky/trans/show/ShowFragmentOperation.java b/dinky-core/src/main/java/org/dinky/trans/show/ShowFragmentOperation.java index 59983ef124..83aa495ccb 100644 --- a/dinky-core/src/main/java/org/dinky/trans/show/ShowFragmentOperation.java +++ b/dinky-core/src/main/java/org/dinky/trans/show/ShowFragmentOperation.java @@ -57,7 +57,7 @@ public Operation create(String statement) { } @Override - public TableResult build(Executor executor) { + public TableResult execute(Executor executor) { Map> map = SingleSqlParserFactory.generateParser(statement); if (Asserts.isNotNullMap(map)) { if (map.containsKey("FRAGMENT")) { diff --git a/dinky-core/src/main/java/org/dinky/trans/show/ShowFragmentsOperation.java b/dinky-core/src/main/java/org/dinky/trans/show/ShowFragmentsOperation.java index af88e14f79..57ef8b3f69 100644 --- a/dinky-core/src/main/java/org/dinky/trans/show/ShowFragmentsOperation.java +++ b/dinky-core/src/main/java/org/dinky/trans/show/ShowFragmentsOperation.java @@ -51,7 +51,7 @@ public Operation create(String statement) { } @Override - public TableResult build(Executor executor) { + public TableResult execute(Executor executor) { return executor.getVariableManager().getVariables(); } }