ibessonov commented on code in PR #979:
URL: https://github.com/apache/ignite-3/pull/979#discussion_r938612786


##########
modules/core/src/test/java/org/apache/ignite/internal/testframework/VariableArgumentsProvider.java:
##########
@@ -45,7 +45,7 @@ public void accept(VariableSource variableSource) {
     public Stream<? extends Arguments> provideArguments(ExtensionContext 
context) throws Exception {
         Class<?> testClass = context.getRequiredTestClass();
 
-        Field source = testClass.getDeclaredField(variableName);
+        Field source = testClass.getField(variableName);

Review Comment:
   Very interesting, but what was wrong with non-public fields?



##########
modules/storage-api/src/main/java/org/apache/ignite/internal/storage/index/SortedIndexDescriptor.java:
##########
@@ -17,72 +17,58 @@
 
 package org.apache.ignite.internal.storage.index;
 
-import static java.util.stream.Collectors.toList;
 import static java.util.stream.Collectors.toUnmodifiableList;
 
-import java.util.Arrays;
-import java.util.Comparator;
 import java.util.List;
-import java.util.stream.Stream;
 import org.apache.ignite.configuration.NamedListView;
 import org.apache.ignite.configuration.schemas.table.ColumnView;
 import org.apache.ignite.configuration.schemas.table.IndexColumnView;
 import org.apache.ignite.configuration.schemas.table.SortedIndexView;
 import org.apache.ignite.configuration.schemas.table.TableIndexView;
 import org.apache.ignite.configuration.schemas.table.TableView;
-import org.apache.ignite.internal.schema.Column;
-import org.apache.ignite.internal.schema.SchemaDescriptor;
+import org.apache.ignite.internal.schema.NativeType;
 import 
org.apache.ignite.internal.schema.configuration.SchemaConfigurationConverter;
 import 
org.apache.ignite.internal.schema.configuration.SchemaDescriptorConverter;
 import org.apache.ignite.internal.storage.StorageException;
 import org.apache.ignite.internal.tostring.S;
-import org.apache.ignite.schema.definition.ColumnDefinition;
 
 /**
  * Descriptor for creating a Sorted Index Storage.
  *
- * @see SortedIndexStorage
+ * @see SortedIndexMvStorage
  */
 public class SortedIndexDescriptor {
     /**
      * Descriptor of a Sorted Index column (column name and column sort order).
      */
     public static class ColumnDescriptor {
-        private final Column column;
+        private final String name;
 
-        private final boolean asc;
-
-        private final boolean indexedColumn;
+        private final NativeType type;
 
         private final boolean nullable;
 
-        ColumnDescriptor(Column column, boolean asc, boolean indexedColumn, 
boolean nullable) {
-            this.column = column;
-            this.asc = asc;
-            this.indexedColumn = indexedColumn;
-            this.nullable = nullable;
-        }
+        private final boolean asc;

Review Comment:
   Since we're here, you can add "nullFirst" flag, but only if it's easy and 
won't take much time.



##########
modules/storage-api/src/main/java/org/apache/ignite/internal/storage/index/IndexRow.java:
##########
@@ -17,26 +17,26 @@
 
 package org.apache.ignite.internal.storage.index;
 
-import org.apache.ignite.internal.storage.PartitionStorage;
-import org.apache.ignite.internal.storage.SearchRow;
+import java.nio.ByteBuffer;
+import org.apache.ignite.internal.storage.MvPartitionStorage;
+import org.apache.ignite.internal.storage.RowId;
 
 /**
- * Represents an Index Row - a set of indexed columns and Primary Key columns 
(for key uniqueness).
+ * Represents an Index Row - a set of indexed columns and a corresponding Row 
ID.
  */
 public interface IndexRow {
     /**
-     * Returns the serialized presentation of this row as a byte array.
+     * Returns the serialized presentation of indexed columns as a byte array 
in
+     * <a 
href="https://cwiki.apache.org/confluence/display/IGNITE/IEP-92%3A+Binary+Tuple+Format";>IEP-92</a>
 format.
      *
      * @return Serialized byte array value.
      */
-    byte[] rowBytes();
+    ByteBuffer indexBytes();

Review Comment:
   It's weird that here we operate with bytes instead of InternalTuple or 
something. What's the reason of such decision? I find it non-intuitive



##########
modules/storage-api/src/test/java/org/apache/ignite/internal/storage/index/impl/BinaryTupleComparator.java:
##########
@@ -0,0 +1,149 @@
+/*
+ * 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.storage.index.impl;
+
+import static java.util.Comparator.comparing;
+import static java.util.Comparator.comparingDouble;
+import static java.util.Comparator.comparingInt;
+import static java.util.Comparator.comparingLong;
+import static 
org.apache.ignite.internal.storage.index.impl.ComparatorUtils.comparingNull;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.stream.IntStream;
+import org.apache.ignite.internal.schema.BinaryTuple;
+import org.apache.ignite.internal.schema.BinaryTupleSchema;
+import org.apache.ignite.internal.schema.NativeType;
+import org.apache.ignite.internal.storage.index.SortedIndexDescriptor;
+import 
org.apache.ignite.internal.storage.index.SortedIndexDescriptor.ColumnDescriptor;
+
+/**
+ * Comparator implementation for comparting {@link BinaryTuple}s on a 
per-column basis.
+ */
+class BinaryTupleComparator implements Comparator<ByteBuffer> {

Review Comment:
   Please add TODO for https://issues.apache.org/jira/browse/IGNITE-17325



##########
modules/storage-api/src/test/java/org/apache/ignite/internal/storage/index/impl/PrefixComparator.java:
##########
@@ -15,77 +15,67 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.storage.index;
+package org.apache.ignite.internal.storage.index.impl;
 
 import java.util.Arrays;
 import java.util.BitSet;
-import org.apache.ignite.internal.schema.BinaryRow;
-import org.apache.ignite.internal.schema.Column;
-import org.apache.ignite.internal.schema.NativeTypeSpec;
-import org.apache.ignite.internal.schema.row.Row;
+import org.apache.ignite.internal.schema.BinaryTuple;
+import org.apache.ignite.internal.schema.NativeType;
+import org.apache.ignite.internal.storage.index.IndexRowPrefix;
+import org.apache.ignite.internal.storage.index.SortedIndexDescriptor;
 import 
org.apache.ignite.internal.storage.index.SortedIndexDescriptor.ColumnDescriptor;
 import org.jetbrains.annotations.Nullable;
 
 /**
- * Class for comparing a {@link BinaryRow} representing an Index Key with a 
given prefix of index columns.
+ * Class for comparing a {@link BinaryTuple} representing an Index Row with a 
given prefix of index columns.
  */
-public class PrefixComparator {
+class PrefixComparator {

Review Comment:
   I expect this class to disappear in the future, is this a reasonable 
expectation? There should be a single place with comparison implementation



##########
modules/storage-api/src/main/java/org/apache/ignite/internal/storage/index/IndexRowPrefix.java:
##########
@@ -23,7 +23,7 @@
 public interface IndexRowPrefix {

Review Comment:
   I believe that we should use InternalTuple for prefixes as well, there's no 
point in having additional interface now



##########
modules/storage-api/src/main/java/org/apache/ignite/internal/storage/index/SortedIndexMvStorage.java:
##########
@@ -51,68 +50,47 @@ public interface SortedIndexMvStorage {
     int BACKWARDS = 1 << 2;
 
     /**
-     * The sole purpose of this class is to avoid massive refactoring while 
changing the original IndexRow.
+     * Returns the Index Descriptor of this storage.
      */
-    interface IndexRowEx {
-        /**
-         * Key-only binary row if index-only scan is supported, full binary 
row otherwise.
-         */
-        BinaryRow row();
+    SortedIndexDescriptor indexDescriptor();
 
-        /**
-         * Returns indexed column value.
-         *
-         * @param idx PK column index.
-         * @return Indexed column value.
-         */
-        Object value(int idx);
-    }
+    /**
+     * Returns a factory for creating index rows for this storage.
+     */
+    IndexRowSerializer indexRowSerializer();
 
-    boolean supportsBackwardsScan();
+    /**
+     * Returns a class deserializing index columns.
+     */
+    IndexRowDeserializer indexRowDeserializer();

Review Comment:
   Again, we should discuss the decision of having separate 
serializer/deserializer in index. It's not clear, maybe we should remove them



##########
modules/storage-api/src/test/java/org/apache/ignite/internal/storage/index/impl/BinaryTupleComparator.java:
##########
@@ -0,0 +1,149 @@
+/*
+ * 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.storage.index.impl;
+
+import static java.util.Comparator.comparing;
+import static java.util.Comparator.comparingDouble;
+import static java.util.Comparator.comparingInt;
+import static java.util.Comparator.comparingLong;
+import static 
org.apache.ignite.internal.storage.index.impl.ComparatorUtils.comparingNull;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.stream.IntStream;
+import org.apache.ignite.internal.schema.BinaryTuple;
+import org.apache.ignite.internal.schema.BinaryTupleSchema;
+import org.apache.ignite.internal.schema.NativeType;
+import org.apache.ignite.internal.storage.index.SortedIndexDescriptor;
+import 
org.apache.ignite.internal.storage.index.SortedIndexDescriptor.ColumnDescriptor;
+
+/**
+ * Comparator implementation for comparting {@link BinaryTuple}s on a 
per-column basis.
+ */
+class BinaryTupleComparator implements Comparator<ByteBuffer> {
+    /**
+     * Actual comparator implementation.
+     */
+    private final Comparator<ByteBuffer> delegate;

Review Comment:
   What's the point of having a delegate if we could just implement compareTo? 
What are the benefits?



##########
modules/storage-api/src/test/java/org/apache/ignite/internal/storage/index/impl/BinaryTupleComparator.java:
##########
@@ -0,0 +1,149 @@
+/*
+ * 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.storage.index.impl;
+
+import static java.util.Comparator.comparing;
+import static java.util.Comparator.comparingDouble;
+import static java.util.Comparator.comparingInt;
+import static java.util.Comparator.comparingLong;
+import static 
org.apache.ignite.internal.storage.index.impl.ComparatorUtils.comparingNull;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.stream.IntStream;
+import org.apache.ignite.internal.schema.BinaryTuple;
+import org.apache.ignite.internal.schema.BinaryTupleSchema;
+import org.apache.ignite.internal.schema.NativeType;
+import org.apache.ignite.internal.storage.index.SortedIndexDescriptor;
+import 
org.apache.ignite.internal.storage.index.SortedIndexDescriptor.ColumnDescriptor;
+
+/**
+ * Comparator implementation for comparting {@link BinaryTuple}s on a 
per-column basis.
+ */
+class BinaryTupleComparator implements Comparator<ByteBuffer> {
+    /**
+     * Actual comparator implementation.
+     */
+    private final Comparator<ByteBuffer> delegate;
+
+    /** Index descriptor. */
+    private final SortedIndexDescriptor indexDescriptor;
+
+    /**
+     * Creates a comparator for a Sorted Index identified by the given 
descriptor.
+     */
+    BinaryTupleComparator(SortedIndexDescriptor descriptor, BinaryTupleSchema 
schema) {
+        this.indexDescriptor = descriptor;
+
+        this.delegate = comparing(bytes -> new BinaryTuple(schema, bytes), 
binaryTupleComparator());
+    }
+
+    /**
+     * Creates a comparator that compares two {@link BinaryTuple}s by 
comparing individual columns.
+     */
+    private Comparator<BinaryTuple> binaryTupleComparator() {
+        return IntStream.range(0, indexDescriptor.indexColumns().size())

Review Comment:
   I expect comparison to be a very hot method. Let's avoid all these streams 
and lambdas, ok?



##########
modules/storage-api/src/main/java/org/apache/ignite/internal/storage/index/IndexRowSerializer.java:
##########
@@ -17,17 +17,17 @@
 
 package org.apache.ignite.internal.storage.index;
 
-import org.apache.ignite.internal.storage.SearchRow;
+import org.apache.ignite.internal.storage.RowId;
 
 /**
  * Temporary API for creating Index rows from a list of column values. All 
columns must be sorted according to the index columns order,
- * specified by the {@link SortedIndexDescriptor#indexRowColumns()}.
+ * specified by the {@link SortedIndexDescriptor#indexColumns()}.
  */
-public interface IndexRowFactory {
+public interface IndexRowSerializer {
     /**
      * Creates an Index row from a list of column values.
      */
-    IndexRow createIndexRow(Object[] columnValues, SearchRow primaryKey);
+    IndexRow createIndexRow(Object[] columnValues, RowId rowId);

Review Comment:
   This approach is inherently ineffective, most of column values could be 
copied directly from BinaryRow's payload. Serialization to Object and back is 
redundant



##########
modules/storage-api/src/main/java/org/apache/ignite/internal/storage/index/SortedIndexMvStorage.java:
##########
@@ -51,68 +50,47 @@ public interface SortedIndexMvStorage {
     int BACKWARDS = 1 << 2;
 
     /**
-     * The sole purpose of this class is to avoid massive refactoring while 
changing the original IndexRow.
+     * Returns the Index Descriptor of this storage.
      */
-    interface IndexRowEx {
-        /**
-         * Key-only binary row if index-only scan is supported, full binary 
row otherwise.
-         */
-        BinaryRow row();
+    SortedIndexDescriptor indexDescriptor();
 
-        /**
-         * Returns indexed column value.
-         *
-         * @param idx PK column index.
-         * @return Indexed column value.
-         */
-        Object value(int idx);
-    }
+    /**
+     * Returns a factory for creating index rows for this storage.
+     */
+    IndexRowSerializer indexRowSerializer();
 
-    boolean supportsBackwardsScan();
+    /**
+     * Returns a class deserializing index columns.
+     */
+    IndexRowDeserializer indexRowDeserializer();
 
-    boolean supportsIndexOnlyScan();
+    /**
+     * Adds the given index row to the index.
+     */
+    void put(IndexRow row);
 
     /**
-     * Returns a range of index values between the lower bound and the upper 
bound, consistent with the passed transaction id.
+     * Removes the given key from the index.
      *
-     * @param lowerBound Lower bound. Exclusivity is controlled by a {@link 
#GREATER_OR_EQUAL} or {@link #GREATER} flag.
-     *      {@code null} means unbounded.
-     * @param upperBound Upper bound. Exclusivity is controlled by a {@link 
#LESS} or {@link #LESS_OR_EQUAL} flag.
-     *      {@code null} means unbounded.
-     * @param flags Control flags. {@link #GREATER} | {@link #LESS} | {@link 
#FORWARD} by default. Other available values
-     *      are {@link #GREATER_OR_EQUAL}, {@link #LESS_OR_EQUAL} and {@link 
#BACKWARDS}.
-     * @param txId Transaction id for consistent multi-versioned index scan.
-     * @param partitionFilter Partition filter predicate. {@code null} means 
returning data from all partitions.
-     * @return Cursor with fetched index rows.
-     * @throws IllegalArgumentException If backwards flag is passed and 
backwards iteration is not supported by the storage.
+     * <p>Removing a non-existent key is a no-op.
      */
-    Cursor<IndexRowEx> scan(
-            @Nullable IndexRowPrefix lowerBound,
-            @Nullable IndexRowPrefix upperBound,
-            @MagicConstant(flagsFromClass = SortedIndexStorage.class) int 
flags,
-            UUID txId,
-            @Nullable IntPredicate partitionFilter
-    );
+    void remove(IndexRow row);
 
     /**
-     * Returns a range of index values between the lower bound and the upper 
bound, consistent with the passed timestamp.
+     * Returns a range of index values between the lower bound and the upper 
bound, consistent with the passed transaction id.

Review Comment:
   I'd remove the "consistent with" part, there's no such parameter now



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

Reply via email to