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/08/04 15:58:46 UTC

[GitHub] [ignite-3] sashapolo opened a new pull request, #979: IGNITE-17308 Rework SortedIndexMvStorage interface

sashapolo opened a new pull request, #979:
URL: https://github.com/apache/ignite-3/pull/979

   * Indices are no longer stored in PartitionMvStorage
   * Indices are no longer aware of the MVCC properties
   * Merge functionality with existing sorted indices implementation
   
   https://issues.apache.org/jira/browse/IGNITE-17308


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


[GitHub] [ignite-3] ibessonov commented on a diff in pull request #979: IGNITE-17308 Rework SortedIndexMvStorage interface

Posted by GitBox <gi...@apache.org>.
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: notifications-unsubscribe@ignite.apache.org

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


[GitHub] [ignite-3] ibessonov commented on a diff in pull request #979: IGNITE-17308 Rework SortedIndexMvStorage interface

Posted by GitBox <gi...@apache.org>.
ibessonov commented on code in PR #979:
URL: https://github.com/apache/ignite-3/pull/979#discussion_r938647875


##########
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:
   Ok, I realized that after commenting. It's fine



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


[GitHub] [ignite-3] sashapolo commented on a diff in pull request #979: IGNITE-17308 Rework SortedIndexMvStorage interface

Posted by GitBox <gi...@apache.org>.
sashapolo commented on code in PR #979:
URL: https://github.com/apache/ignite-3/pull/979#discussion_r938635889


##########
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 API is temporary and should be removed as soon as we understand how it will be integrated with the partition storage



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


[GitHub] [ignite-3] sashapolo commented on a diff in pull request #979: IGNITE-17308 Rework SortedIndexMvStorage interface

Posted by GitBox <gi...@apache.org>.
sashapolo commented on code in PR #979:
URL: https://github.com/apache/ignite-3/pull/979#discussion_r938910305


##########
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:
   oops



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


[GitHub] [ignite-3] sashapolo commented on a diff in pull request #979: IGNITE-17308 Rework SortedIndexMvStorage interface

Posted by GitBox <gi...@apache.org>.
sashapolo commented on code in PR #979:
URL: https://github.com/apache/ignite-3/pull/979#discussion_r938955613


##########
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:
   I'd rather not, because it's Friday and I'm too tired



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


[GitHub] [ignite-3] ibessonov commented on a diff in pull request #979: IGNITE-17308 Rework SortedIndexMvStorage interface

Posted by GitBox <gi...@apache.org>.
ibessonov commented on code in PR #979:
URL: https://github.com/apache/ignite-3/pull/979#discussion_r938650752


##########
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:
   I didn't realize it's a test class, sorry



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


[GitHub] [ignite-3] sashapolo commented on a diff in pull request #979: IGNITE-17308 Rework SortedIndexMvStorage interface

Posted by GitBox <gi...@apache.org>.
sashapolo commented on code in PR #979:
URL: https://github.com/apache/ignite-3/pull/979#discussion_r938639625


##########
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:
   I believe there also was a ticket about implementing a naturally sorted byte order format



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


[GitHub] [ignite-3] sashapolo commented on a diff in pull request #979: IGNITE-17308 Rework SortedIndexMvStorage interface

Posted by GitBox <gi...@apache.org>.
sashapolo commented on code in PR #979:
URL: https://github.com/apache/ignite-3/pull/979#discussion_r938638861


##########
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:
   yes, I left it for simplicity purposes



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


[GitHub] [ignite-3] sashapolo commented on a diff in pull request #979: IGNITE-17308 Rework SortedIndexMvStorage interface

Posted by GitBox <gi...@apache.org>.
sashapolo commented on code in PR #979:
URL: https://github.com/apache/ignite-3/pull/979#discussion_r938636956


##########
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:
   Yes, I also don't like them, they should be removed when indices get integrated with the storage



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


[GitHub] [ignite-3] ibessonov commented on a diff in pull request #979: IGNITE-17308 Rework SortedIndexMvStorage interface

Posted by GitBox <gi...@apache.org>.
ibessonov commented on code in PR #979:
URL: https://github.com/apache/ignite-3/pull/979#discussion_r938652716


##########
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:
   Ok, I guess we'll do it in https://issues.apache.org/jira/browse/IGNITE-17310



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


[GitHub] [ignite-3] ibessonov merged pull request #979: IGNITE-17308 Rework SortedIndexMvStorage interface

Posted by GitBox <gi...@apache.org>.
ibessonov merged PR #979:
URL: https://github.com/apache/ignite-3/pull/979


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


[GitHub] [ignite-3] ibessonov commented on a diff in pull request #979: IGNITE-17308 Rework SortedIndexMvStorage interface

Posted by GitBox <gi...@apache.org>.
ibessonov commented on code in PR #979:
URL: https://github.com/apache/ignite-3/pull/979#discussion_r938651647


##########
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:
   Yes, but they're slow



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


[GitHub] [ignite-3] ibessonov commented on a diff in pull request #979: IGNITE-17308 Rework SortedIndexMvStorage interface

Posted by GitBox <gi...@apache.org>.
ibessonov commented on code in PR #979:
URL: https://github.com/apache/ignite-3/pull/979#discussion_r938650040


##########
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:
   I think it would be better. I'd prefer InternalTuple for convenience.
   Anyway, there should be a common way of converting a InternalTuple into IEP-92 data format, otherwise such API wouldn't work. I realize that, and I believe that we should provide this "common way"



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


[GitHub] [ignite-3] ibessonov commented on a diff in pull request #979: IGNITE-17308 Rework SortedIndexMvStorage interface

Posted by GitBox <gi...@apache.org>.
ibessonov commented on code in PR #979:
URL: https://github.com/apache/ignite-3/pull/979#discussion_r940049315


##########
modules/storage-api/src/main/java/org/apache/ignite/internal/storage/index/IndexRow.java:
##########
@@ -26,12 +26,12 @@
  */
 public interface IndexRow {
     /**
-     * Returns the serialized presentation of indexed columns as a byte array in
+     * Returns the serialized presentation of indexed columns as a {@link BinaryTuple} 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.
+     * @return Binary tuple representation of indexed columns.
      */
-    ByteBuffer indexBytes();
+    BinaryTuple indexColumns();

Review Comment:
   I wonder why it's a BinaryTuple instead of InternalTuple. Maybe we don't even need an interface, only the implementation? Maybe we should remove it in the future, idk



##########
modules/storage-api/src/main/java/org/apache/ignite/internal/storage/index/IndexRow.java:
##########
@@ -26,12 +26,12 @@
  */
 public interface IndexRow {
     /**
-     * Returns the serialized presentation of indexed columns as a byte array in
+     * Returns the serialized presentation of indexed columns as a {@link BinaryTuple} 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.
+     * @return Binary tuple representation of indexed columns.
      */
-    ByteBuffer indexBytes();
+    BinaryTuple indexColumns();

Review Comment:
   You see, the problem with BinaryTuple is that it contains its own schema for no good reason.
   Well, there is a reason, but it's artificial and can be worked around



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


[GitHub] [ignite-3] sashapolo commented on a diff in pull request #979: IGNITE-17308 Rework SortedIndexMvStorage interface

Posted by GitBox <gi...@apache.org>.
sashapolo commented on code in PR #979:
URL: https://github.com/apache/ignite-3/pull/979#discussion_r938638268


##########
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:
   It's a test class, why should we bother? I expect the production implementation to be different



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


[GitHub] [ignite-3] sashapolo commented on a diff in pull request #979: IGNITE-17308 Rework SortedIndexMvStorage interface

Posted by GitBox <gi...@apache.org>.
sashapolo commented on code in PR #979:
URL: https://github.com/apache/ignite-3/pull/979#discussion_r938637313


##########
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:
   This is a test class, should we bother?



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


[GitHub] [ignite-3] sashapolo commented on a diff in pull request #979: IGNITE-17308 Rework SortedIndexMvStorage interface

Posted by GitBox <gi...@apache.org>.
sashapolo commented on code in PR #979:
URL: https://github.com/apache/ignite-3/pull/979#discussion_r938641534


##########
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:
   Because it is impossible to deserialize an `InternalTuple`, while a `ByteBuffer` can be wrapped into a `BinaryTuple`. Returning a `BinaryTuple` might be a good alternative, but I'm not sure if it's actually better



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


[GitHub] [ignite-3] sashapolo commented on a diff in pull request #979: IGNITE-17308 Rework SortedIndexMvStorage interface

Posted by GitBox <gi...@apache.org>.
sashapolo commented on code in PR #979:
URL: https://github.com/apache/ignite-3/pull/979#discussion_r938908169


##########
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:
   Replaced with BinaryTuple



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

Review Comment:
   Removed this class



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


[GitHub] [ignite-3] ibessonov commented on a diff in pull request #979: IGNITE-17308 Rework SortedIndexMvStorage interface

Posted by GitBox <gi...@apache.org>.
ibessonov commented on code in PR #979:
URL: https://github.com/apache/ignite-3/pull/979#discussion_r938651995


##########
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:
   Ok, again, I didn't realize it was in a "test" folder



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


[GitHub] [ignite-3] sashapolo commented on a diff in pull request #979: IGNITE-17308 Rework SortedIndexMvStorage interface

Posted by GitBox <gi...@apache.org>.
sashapolo commented on code in PR #979:
URL: https://github.com/apache/ignite-3/pull/979#discussion_r938637889


##########
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:
   you can't use all those functional things, like `comparing` and stuff, because they return a Comparator



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


[GitHub] [ignite-3] ibessonov commented on a diff in pull request #979: IGNITE-17308 Rework SortedIndexMvStorage interface

Posted by GitBox <gi...@apache.org>.
ibessonov commented on code in PR #979:
URL: https://github.com/apache/ignite-3/pull/979#discussion_r938651159


##########
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:
   Naturally sorted byte order format will not be implemented, as far as I know, we abandoned this idea



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


[GitHub] [ignite-3] sashapolo commented on a diff in pull request #979: IGNITE-17308 Rework SortedIndexMvStorage interface

Posted by GitBox <gi...@apache.org>.
sashapolo commented on code in PR #979:
URL: https://github.com/apache/ignite-3/pull/979#discussion_r938642675


##########
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:
   `getDeclaredField` doesn't look into fields that are declared in superclasses. It could be fixed by manually traversing the hierarchy, but I don't see anything wrong with simply declaring these variables as `public`



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


[GitHub] [ignite-3] sashapolo commented on a diff in pull request #979: IGNITE-17308 Rework SortedIndexMvStorage interface

Posted by GitBox <gi...@apache.org>.
sashapolo commented on code in PR #979:
URL: https://github.com/apache/ignite-3/pull/979#discussion_r938771123


##########
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:
   Yes, but this is a test class, so I don't think it matters. This comparator should be heavily refactored for production use anyway



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


[GitHub] [ignite-3] sashapolo commented on a diff in pull request #979: IGNITE-17308 Rework SortedIndexMvStorage interface

Posted by GitBox <gi...@apache.org>.
sashapolo commented on code in PR #979:
URL: https://github.com/apache/ignite-3/pull/979#discussion_r938908767


##########
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 removed these streams



##########
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 removed this class



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


[GitHub] [ignite-3] sashapolo commented on a diff in pull request #979: IGNITE-17308 Rework SortedIndexMvStorage interface

Posted by GitBox <gi...@apache.org>.
sashapolo commented on code in PR #979:
URL: https://github.com/apache/ignite-3/pull/979#discussion_r938908584


##########
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:
   I removed the delegate



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