You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@ignite.apache.org by "korlov42 (via GitHub)" <gi...@apache.org> on 2023/05/29 10:49:13 UTC

[GitHub] [ignite-3] korlov42 commented on a diff in pull request #2085: IGNITE-19460 Sql. Implement missed DDL commands with using Catalog

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


##########
modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/internal/InternalSchemaTest.java:
##########
@@ -31,13 +31,16 @@
 import org.apache.ignite.sql.IgniteSql;
 import org.apache.ignite.sql.ResultSet;
 import org.apache.ignite.sql.Session;
+import org.junit.jupiter.api.Disabled;
 import org.junit.jupiter.api.Test;
 
 /** Tests for internal manipulations with schema. */
 public class InternalSchemaTest extends ClusterPerClassIntegrationTest {
     /**
      * Checks that schema version is updated even if column names are intersected.
+     * TODO Drop this test, when schema will be moved from configuration to Catalog.
      */
+    @Disabled("https://issues.apache.org/jira/browse/IGNITE-19460")

Review Comment:
   what is wrong with this test?



##########
modules/catalog/src/main/java/org/apache/ignite/internal/catalog/storage/NewIndexEntry.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.ignite.internal.catalog.storage;
+
+import org.apache.ignite.internal.catalog.descriptors.IndexDescriptor;
+import org.apache.ignite.internal.tostring.S;
+
+/**
+ * Describes addition of a new index.
+ */
+public class NewIndexEntry implements UpdateEntry {
+    private static final long serialVersionUID = 6717363577013237711L;
+
+    private final IndexDescriptor descriptor;
+
+    /**
+     * Constructs the object.
+     *
+     * @param descriptor A descriptor of a index to add.

Review Comment:
   ```suggestion
        * @param descriptor A descriptor of an index to add.
   ```



##########
modules/catalog/src/main/java/org/apache/ignite/internal/catalog/storage/DropIndexEntry.java:
##########
@@ -0,0 +1,49 @@
+/*
+ * 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.catalog.storage;
+
+import org.apache.ignite.internal.tostring.S;
+
+/**
+ * Describes deletion of a index.
+ */
+public class DropIndexEntry implements UpdateEntry {
+    private static final long serialVersionUID = -604729846502020728L;
+
+    private final int indexId;
+
+    /**
+     * Constructs the object.
+     *
+     * @param indexId An id of a index to drop.
+     */
+    public DropIndexEntry(int indexId) {
+        this.indexId = indexId;
+    }
+
+    /** Returns an id of a index to drop. */

Review Comment:
   ```suggestion
       /** Returns an id of an index to drop. */
   ```



##########
modules/catalog/src/main/java/org/apache/ignite/internal/catalog/storage/DropIndexEntry.java:
##########
@@ -0,0 +1,49 @@
+/*
+ * 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.catalog.storage;
+
+import org.apache.ignite.internal.tostring.S;
+
+/**
+ * Describes deletion of a index.
+ */
+public class DropIndexEntry implements UpdateEntry {
+    private static final long serialVersionUID = -604729846502020728L;
+
+    private final int indexId;
+
+    /**
+     * Constructs the object.
+     *
+     * @param indexId An id of a index to drop.

Review Comment:
   ```suggestion
        * @param indexId An id of an index to drop.
   ```



##########
modules/catalog/src/main/java/org/apache/ignite/internal/catalog/storage/DropIndexEntry.java:
##########
@@ -0,0 +1,49 @@
+/*
+ * 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.catalog.storage;
+
+import org.apache.ignite.internal.tostring.S;
+
+/**
+ * Describes deletion of a index.

Review Comment:
   ```suggestion
    * Describes deletion of an index.
   ```



##########
modules/catalog/src/main/java/org/apache/ignite/internal/catalog/storage/DropColumnsEntry.java:
##########
@@ -0,0 +1,58 @@
+/*
+ * 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.catalog.storage;
+
+import java.util.Set;
+import org.apache.ignite.internal.tostring.S;
+
+/**
+ * Describes addition of a new columns.

Review Comment:
   it's more like `deletion of columns from table`



##########
modules/catalog/src/main/java/org/apache/ignite/internal/catalog/storage/NewIndexEntry.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.ignite.internal.catalog.storage;
+
+import org.apache.ignite.internal.catalog.descriptors.IndexDescriptor;
+import org.apache.ignite.internal.tostring.S;
+
+/**
+ * Describes addition of a new index.
+ */
+public class NewIndexEntry implements UpdateEntry {
+    private static final long serialVersionUID = 6717363577013237711L;
+
+    private final IndexDescriptor descriptor;
+
+    /**
+     * Constructs the object.
+     *
+     * @param descriptor A descriptor of a index to add.
+     */
+    public NewIndexEntry(IndexDescriptor descriptor) {
+        this.descriptor = descriptor;
+    }
+
+    /** Returns descriptor of a index to add. */

Review Comment:
   ```suggestion
       /** Returns descriptor of an index to add. */
   ```



##########
modules/catalog/src/main/java/org/apache/ignite/internal/catalog/storage/NewColumnsEntry.java:
##########
@@ -0,0 +1,59 @@
+/*
+ * 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.catalog.storage;
+
+import java.util.List;
+import org.apache.ignite.internal.catalog.descriptors.TableColumnDescriptor;
+import org.apache.ignite.internal.tostring.S;
+
+/**
+ * Describes addition of a new columns.

Review Comment:
   ```suggestion
    * Describes addition of new columns.
   ```



##########
modules/catalog/src/main/java/org/apache/ignite/internal/catalog/storage/NewColumnsEntry.java:
##########
@@ -0,0 +1,59 @@
+/*
+ * 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.catalog.storage;
+
+import java.util.List;
+import org.apache.ignite.internal.catalog.descriptors.TableColumnDescriptor;
+import org.apache.ignite.internal.tostring.S;
+
+/**
+ * Describes addition of a new columns.
+ */
+public class NewColumnsEntry implements UpdateEntry {
+    private static final long serialVersionUID = 2970125889493580121L;
+
+    private final int tableId;
+    private final List<TableColumnDescriptor> descriptors;
+
+    /**
+     * Constructs the object.
+     *
+     * @param tableId Table id.
+     * @param descriptors A descriptors of columns to add.

Review Comment:
   ```suggestion
        * @param descriptors Descriptors of columns to add.
   ```



##########
modules/catalog/src/main/java/org/apache/ignite/internal/catalog/events/CatalogEvent.java:
##########
@@ -27,5 +27,17 @@ public enum CatalogEvent implements Event {
     TABLE_CREATE,
 
     /** This event is fired, when a table was dropped in Catalog. */
-    TABLE_DROP
+    TABLE_DROP,
+
+    /** This event is fired, when a index was created in Catalog. */
+    INDEX_CREATE,
+
+    /** This event is fired, when a index was dropped in Catalog. */
+    INDEX_DROP,
+
+    /** This event is fired, when a column was added to a table. */
+    COLUMN_ADD,
+
+    /** This event is fired, when a index was dropped from a table. */
+    COLUMN_DROP

Review Comment:
   wondering why do we need this two events? 



##########
modules/catalog/src/main/java/org/apache/ignite/internal/catalog/descriptors/IndexDescriptor.java:
##########
@@ -29,21 +32,34 @@ public abstract class IndexDescriptor extends ObjectDescriptor {
     private final int tableId;
 
     /** Unique constraint flag. */
-    private boolean unique;
+    private final boolean unique;
+
+    /** Index columns. */
+    private final List<String> columns;
 
     /** Write only flag. {@code True} when index is building. */
     private boolean writeOnly;
 
-    IndexDescriptor(int id, String name, int tableId, boolean unique) {
+    IndexDescriptor(int id, String name, int tableId, List<String> columns, boolean unique) {
         super(id, Type.INDEX, name);
         this.tableId = tableId;
         this.unique = unique;
+        this.columns = Objects.requireNonNull(columns, "columns");
+
+        if (Set.copyOf(this.columns).size() != this.columns.size()) {
+            throw new IllegalArgumentException("Indexed columns should be unique");

Review Comment:
   I believe there is no place for validation like this in an object descriptor. Any validation that is based on business rules should take place in manager, command handler, etc



##########
modules/catalog/src/main/java/org/apache/ignite/internal/catalog/storage/DropColumnsEntry.java:
##########
@@ -0,0 +1,58 @@
+/*
+ * 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.catalog.storage;
+
+import java.util.Set;
+import org.apache.ignite.internal.tostring.S;
+
+/**
+ * Describes addition of a new columns.
+ */
+public class DropColumnsEntry implements UpdateEntry {
+    private static final long serialVersionUID = 2970125889493580121L;
+
+    private final int tableId;
+    private final Set<String> columns;
+
+    /**
+     * Constructs the object.
+     *
+     * @param tableId Table id.
+     * @param columns A names of columns to drop.

Review Comment:
   ```suggestion
        * @param columns Names of columns to drop.
   ```



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