You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@ignite.apache.org by GitBox <gi...@apache.org> on 2022/12/01 10:53:00 UTC

[GitHub] [ignite-3] korlov42 commented on a diff in pull request #1385: IGNITE-18157 Sql. Provide commands and handlers for distributed zones related operations

korlov42 commented on code in PR #1385:
URL: https://github.com/apache/ignite-3/pull/1385#discussion_r1036896365


##########
modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/prepare/ddl/DdlCommandOptionUpdater.java:
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.ignite.internal.sql.engine.prepare.ddl;
+
+import static org.apache.ignite.lang.ErrorGroups.Sql.DDL_OPTION_ERR;
+
+import java.util.function.BiConsumer;
+import java.util.function.Consumer;
+import org.apache.calcite.sql.SqlLiteral;
+import org.apache.ignite.lang.IgniteException;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * DDL command updater.
+ */
+class DdlCommandOptionUpdater<S, T> {
+    private final Class<T> type;
+
+    @Nullable private final Consumer<T> validator;
+
+    private final BiConsumer<S, T> setter;

Review Comment:
   let's introduce a specific interface instead of abstract BiConsumer



##########
modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/prepare/ddl/DdlCommandOptionUpdater.java:
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.ignite.internal.sql.engine.prepare.ddl;
+
+import static org.apache.ignite.lang.ErrorGroups.Sql.DDL_OPTION_ERR;
+
+import java.util.function.BiConsumer;
+import java.util.function.Consumer;
+import org.apache.calcite.sql.SqlLiteral;
+import org.apache.ignite.lang.IgniteException;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * DDL command updater.

Review Comment:
   Could you please extend the javadoc with more details?



##########
modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/prepare/ddl/DdlSqlToCommandConverter.java:
##########
@@ -97,16 +108,23 @@ public class DdlSqlToCommandConverter {
     private final Map<String, String> dataStorageNames;
 
     /**
-     * Mapping: Table option ID -> table option info.
+     * Mapping: Table option ID -> DDL command updater.
      *
      * <p>Example for "replicas": {@code Map.of("REPLICAS", TableOptionInfo@123)}.
      */
-    private final Map<String, TableOptionInfo<?>> tableOptionInfos;
+    private final Map<String, DdlCommandOptionUpdater<CreateTableCommand, ?>> tableOptionUpdaters;
 
     /**
-     * Like {@link #tableOptionInfos}, but for each data storage name.
+     * Like {@link #tableOptionUpdaters}, but for each data storage name.
      */
-    private final Map<String, Map<String, TableOptionInfo<?>>> dataStorageOptionInfos;
+    private final Map<String, Map<String, DdlCommandOptionUpdater<CreateTableCommand, ?>>> dataStorageOptionUpdaters;
+
+    /**
+     * Mapping: Zone option ID -> DDL command updater.
+     *
+     * <p>Example for "replicas": {@code Map.of("REPLICAS", TableOptionInfo@123)}.

Review Comment:
   what is `TableOptionInfo`?



##########
modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/exec/MockedStructuresTest.java:
##########
@@ -275,6 +285,58 @@ private void mockMetastore() throws Exception {
         when(cursorMocked.iterator()).thenReturn(itMock);
     }
 
+    @Test

Review Comment:
   does it make sense to verify that options properly passed to the dzManager? I'm mean if we put `replicas=4`, it will  causes an invocation of dzManager with proper config



##########
modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/prepare/ddl/DdlSqlToCommandConverterTest.java:
##########
@@ -246,9 +210,95 @@ public void tableWithAutogenPkColumn() throws SqlParseException {
         );
     }
 
-    private TableOptionInfo tableOptionInfo(String name) {
-        return new TableOptionInfo<>(name, Object.class, null, (createTableCommand, o) -> {
-        });
+    @Test

Review Comment:
   the same question about moving to a separate class



##########
modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/exec/MockedStructuresTest.java:
##########
@@ -275,6 +285,58 @@ private void mockMetastore() throws Exception {
         when(cursorMocked.iterator()).thenReturn(itMock);
     }
 
+    @Test
+    public void testCreateZone() {

Review Comment:
   does it make sense to move all zone-related methods to a separate class?



##########
modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/prepare/ddl/DdlCommandOptionUpdater.java:
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.ignite.internal.sql.engine.prepare.ddl;
+
+import static org.apache.ignite.lang.ErrorGroups.Sql.DDL_OPTION_ERR;
+
+import java.util.function.BiConsumer;
+import java.util.function.Consumer;
+import org.apache.calcite.sql.SqlLiteral;
+import org.apache.ignite.lang.IgniteException;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * DDL command updater.
+ */
+class DdlCommandOptionUpdater<S, T> {
+    private final Class<T> type;
+
+    @Nullable private final Consumer<T> validator;
+
+    private final BiConsumer<S, T> setter;
+
+    /**
+     * Constructor.
+     *
+     * @param type DDL option type.
+     * @param validator DDL option value validator.
+     * @param setter DDL option value setter.
+     */
+    DdlCommandOptionUpdater(Class<T> type, @Nullable Consumer<T> validator, BiConsumer<S, T> setter) {
+        this.type = type;
+        this.validator = validator;
+        this.setter = setter;
+    }
+
+    /**
+     * Updates target object.
+     *
+     * @param name Option name.
+     * @param value Option value.
+     * @param query Sql query.
+     * @param target Object to update.
+     */
+    void update(Object name, SqlLiteral value, String query, S target) {
+        T value0;
+
+        try {
+            value0 = value.getValueAs(type);
+        } catch (AssertionError | ClassCastException e) {
+            throw new IgniteException(DDL_OPTION_ERR, String.format(

Review Comment:
   I'm not sure if this utility class should know anything about context of operation. But you've got originating query string leaked from the outside just to create an exception



##########
modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/prepare/ddl/AbstractZoneDdlCommand.java:
##########
@@ -0,0 +1,64 @@
+/*
+ * 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.ignite.internal.sql.engine.prepare.ddl;
+
+/**
+ * Abstract zone ddl command.
+ */
+public class AbstractZoneDdlCommand implements DdlCommand {
+    /** Table zone. */
+    private String zoneName;
+
+    /** Quietly ignore this command if zone is not exists. */
+    protected boolean ifZoneExists;

Review Comment:
   this field doesn't make sense for CREATE cmd



##########
modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/prepare/ddl/DdlCommandOptionUpdater.java:
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.ignite.internal.sql.engine.prepare.ddl;
+
+import static org.apache.ignite.lang.ErrorGroups.Sql.DDL_OPTION_ERR;
+
+import java.util.function.BiConsumer;
+import java.util.function.Consumer;
+import org.apache.calcite.sql.SqlLiteral;
+import org.apache.ignite.lang.IgniteException;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * DDL command updater.
+ */
+class DdlCommandOptionUpdater<S, T> {
+    private final Class<T> type;
+
+    @Nullable private final Consumer<T> validator;
+
+    private final BiConsumer<S, T> setter;
+
+    /**
+     * Constructor.
+     *
+     * @param type DDL option type.
+     * @param validator DDL option value validator.
+     * @param setter DDL option value setter.
+     */
+    DdlCommandOptionUpdater(Class<T> type, @Nullable Consumer<T> validator, BiConsumer<S, T> setter) {
+        this.type = type;
+        this.validator = validator;
+        this.setter = setter;
+    }
+
+    /**
+     * Updates target object.
+     *
+     * @param name Option name.
+     * @param value Option value.
+     * @param query Sql query.
+     * @param target Object to update.
+     */
+    void update(Object name, SqlLiteral value, String query, S target) {

Review Comment:
   another questionable decision is to pass option's name as argument. So, the consistency of error message and actual validation depends on the caller



##########
modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/exec/MockedStructuresTest.java:
##########
@@ -275,6 +285,58 @@ private void mockMetastore() throws Exception {
         when(cursorMocked.iterator()).thenReturn(itMock);
     }
 
+    @Test
+    public void testCreateZone() {
+        String mtdName = getCurrentMethodName();
+
+        String query = String.format("CREATE ZONE %s", mtdName);
+
+        // Create new distribution zone.
+        readFirst(queryProc.queryAsync("PUBLIC", query));
+
+        // Create distribution zone with existing name.
+        IgniteException ex = assertThrows(IgniteException.class, () -> readFirst(queryProc.queryAsync("PUBLIC", query)));
+        assertTrue(hasCause(ex, DistributionZoneAlreadyExistsException.class, null));
+
+        // Check ifNotExists flag.
+        readFirst(queryProc.queryAsync("PUBLIC", String.format("CREATE ZONE IF NOT EXISTS %s", mtdName)));
+    }
+
+    @Test
+    public void testCreateZoneOptions() {
+        String mtdName = getCurrentMethodName();
+
+        // Check for conflicting options.
+        String qry0 = String.format("CREATE ZONE %s WITH DATA_NODES_AUTO_ADJUST=10, DATA_NODES_AUTO_ADJUST_SCALE_UP=5", mtdName);

Review Comment:
   what if some options will be presented several times? I believe we need such a test



##########
modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/prepare/ddl/DdlSqlToCommandConverter.java:
##########
@@ -438,6 +480,61 @@ private DropIndexCommand convertDropIndex(IgniteSqlDropIndex sqlCmd, PlanningCon
         return dropCmd;
     }
 
+    /**
+     * Converts a given CreateZone AST to a CreateZone command.
+     *
+     * @param createZoneNode Root node of the given AST.
+     * @param ctx           Planning context.
+     */
+    private CreateZoneCommand convertCreateZone(IgniteSqlCreateZone createZoneNode, PlanningContext ctx) {
+        CreateZoneCommand createZoneCmd = new CreateZoneCommand();
+
+        createZoneCmd.schemaName(deriveSchemaName(createZoneNode.name(), ctx));
+        createZoneCmd.zoneName(deriveObjectName(createZoneNode.name(), ctx, "zoneName"));
+        createZoneCmd.ifZoneExists(createZoneNode.ifNotExists());

Review Comment:
   that looks weird `ifExists = ifNotExists`



-- 
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@ignite.apache.org

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