ibessonov commented on code in PR #1076: URL: https://github.com/apache/ignite-3/pull/1076#discussion_r974194163
########## modules/schema/src/main/java/org/apache/ignite/internal/schema/BinaryTuplePrefix.java: ########## @@ -0,0 +1,71 @@ +/* + * 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.schema; + +import java.math.BigDecimal; +import java.nio.ByteBuffer; +import org.apache.ignite.internal.binarytuple.BinaryTuplePrefixBuilder; +import org.apache.ignite.internal.binarytuple.BinaryTupleReader; +import org.apache.ignite.internal.schema.row.InternalTuple; + +/** + * Class that represents a Binary Tuple Prefix. + * + * @see BinaryTuplePrefixBuilder BinaryTuplePrefixBuilder for information about the Binary Tuple Prefix format. + */ +public class BinaryTuplePrefix extends BinaryTupleReader implements InternalTuple { + /** Tuple schema. */ + private final BinaryTupleSchema schema; + + /** + * Constructor. + * + * @param schema Full Tuple schema. + * @param bytes Serialized representation of a Binary Tuple Prefix. + */ + public BinaryTuplePrefix(BinaryTupleSchema schema, byte[] bytes) { + super(schema.elementCount() + 1, bytes); + this.schema = schema; + } + + /** + * Constructor. + * + * @param schema Full Tuple schema. + * @param buffer Serialized representation of a Binary Tuple Prefix. + */ + public BinaryTuplePrefix(BinaryTupleSchema schema, ByteBuffer buffer) { + super(schema.elementCount() + 1, buffer); + this.schema = schema; + } + + @Override + public int count() { + return elementCount(); + } + + @Override + public BigDecimal decimalValue(int index) { Review Comment: Not related to your code, but we need to fix the situation with decimal scale ASAP ########## modules/client/src/main/java/org/apache/ignite/internal/client/table/ClientKeyValueView.java: ########## @@ -458,7 +458,7 @@ private void writeKeyValue(ClientSchema s, PayloadOutputChannel w, @Nullable Tra } private void writeKeyValueRaw(ClientSchema s, PayloadOutputChannel w, @NotNull K key, V val) { - var builder = BinaryTupleBuilder.create(s.columns().length, true); + var builder = new BinaryTupleBuilder(s.columns().length, true); Review Comment: What was the exact reason for removing the static constructor? ########## modules/storage-api/src/main/java/org/apache/ignite/internal/storage/index/BinaryTupleComparator.java: ########## @@ -15,80 +15,82 @@ * limitations under the License. */ -package org.apache.ignite.internal.storage.index.impl; +package org.apache.ignite.internal.storage.index; +import static org.apache.ignite.internal.binarytuple.BinaryTupleCommon.isPrefix; + +import java.nio.ByteBuffer; +import java.nio.ByteOrder; import java.util.Arrays; import java.util.Comparator; import org.apache.ignite.internal.schema.BinaryTuple; +import org.apache.ignite.internal.schema.BinaryTuplePrefix; +import org.apache.ignite.internal.schema.BinaryTupleSchema; import org.apache.ignite.internal.schema.NativeTypeSpec; -import org.apache.ignite.internal.storage.index.SortedIndexDescriptor; +import org.apache.ignite.internal.schema.row.InternalTuple; import org.apache.ignite.internal.storage.index.SortedIndexDescriptor.ColumnDescriptor; /** - * Comparator implementation for comparting {@link BinaryTuple}s on a per-column basis. + * Comparator implementation for comparing {@link BinaryTuple}s on a per-column basis. */ -class BinaryTupleComparator implements Comparator<BinaryTuple> { +public class BinaryTupleComparator implements Comparator<ByteBuffer> { private final SortedIndexDescriptor descriptor; - private final int prefixLength; - - private BinaryTupleComparator(SortedIndexDescriptor descriptor, int prefixLength) { - if (prefixLength > descriptor.indexColumns().size()) { - throw new IllegalArgumentException("Invalid prefix length: " + prefixLength); - } - - this.descriptor = descriptor; - this.prefixLength = prefixLength; - } - /** * Creates a comparator for a Sorted Index identified by the given descriptor. */ - static BinaryTupleComparator newComparator(SortedIndexDescriptor descriptor) { - return new BinaryTupleComparator(descriptor, descriptor.indexColumns().size()); - } - - /** - * Similar to {@link #newComparator} but creates a comparator that only compares first {@code prefixLength} index columns. - */ - static BinaryTupleComparator newPrefixComparator(SortedIndexDescriptor descriptor, int prefixLength) { - return new BinaryTupleComparator(descriptor, prefixLength); + public BinaryTupleComparator(SortedIndexDescriptor descriptor) { + this.descriptor = descriptor; } @Override - public int compare(BinaryTuple tuple1, BinaryTuple tuple2) { - return compare(tuple1, tuple2, 1, 0); - } + public int compare(ByteBuffer buffer1, ByteBuffer buffer2) { + assert buffer1.order() == ByteOrder.LITTLE_ENDIAN; + assert buffer2.order() == ByteOrder.LITTLE_ENDIAN; + + BinaryTupleSchema schema = descriptor.binaryTupleSchema(); + + InternalTuple tuple1; + InternalTuple tuple2; + int columnsToCompare; + + if (isPrefix(buffer1)) { Review Comment: It may sound picky, but this looks too bloated for me. Can't we have, like, this: ``` InternalTuple tuple1 = isPrefix(buffer1) ? new BinaryTuplePrefix(schema, buffer1) new BinaryTuple(schema, buffer1); ... int columnsToCompare = Math.min(tuple1.count(), tuple2.count()); ``` ########## modules/schema/src/main/java/org/apache/ignite/internal/schema/BinaryTuplePrefix.java: ########## @@ -0,0 +1,71 @@ +/* + * 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.schema; + +import java.math.BigDecimal; +import java.nio.ByteBuffer; +import org.apache.ignite.internal.binarytuple.BinaryTuplePrefixBuilder; +import org.apache.ignite.internal.binarytuple.BinaryTupleReader; +import org.apache.ignite.internal.schema.row.InternalTuple; + +/** + * Class that represents a Binary Tuple Prefix. + * + * @see BinaryTuplePrefixBuilder BinaryTuplePrefixBuilder for information about the Binary Tuple Prefix format. + */ +public class BinaryTuplePrefix extends BinaryTupleReader implements InternalTuple { + /** Tuple schema. */ + private final BinaryTupleSchema schema; + + /** + * Constructor. + * + * @param schema Full Tuple schema. + * @param bytes Serialized representation of a Binary Tuple Prefix. + */ + public BinaryTuplePrefix(BinaryTupleSchema schema, byte[] bytes) { + super(schema.elementCount() + 1, bytes); + this.schema = schema; + } + + /** + * Constructor. + * + * @param schema Full Tuple schema. + * @param buffer Serialized representation of a Binary Tuple Prefix. + */ + public BinaryTuplePrefix(BinaryTupleSchema schema, ByteBuffer buffer) { + super(schema.elementCount() + 1, buffer); + this.schema = schema; + } + + @Override + public int count() { + return elementCount(); + } + + @Override + public BigDecimal decimalValue(int index) { + return decimalValue(index, schema.element(index).decimalScale); + } + + @Override + public int elementCount() { Review Comment: What's the difference between `count()` and `elementCount()`? ########## modules/schema/src/main/java/org/apache/ignite/internal/schema/BinaryTuplePrefix.java: ########## @@ -0,0 +1,71 @@ +/* + * 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.schema; + +import java.math.BigDecimal; +import java.nio.ByteBuffer; +import org.apache.ignite.internal.binarytuple.BinaryTuplePrefixBuilder; +import org.apache.ignite.internal.binarytuple.BinaryTupleReader; +import org.apache.ignite.internal.schema.row.InternalTuple; + +/** + * Class that represents a Binary Tuple Prefix. + * + * @see BinaryTuplePrefixBuilder BinaryTuplePrefixBuilder for information about the Binary Tuple Prefix format. + */ +public class BinaryTuplePrefix extends BinaryTupleReader implements InternalTuple { Review Comment: Do you really need to read data from it? ########## modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/SortedIndices.java: ########## @@ -0,0 +1,55 @@ +/* + * 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.rocksdb; + +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import org.apache.ignite.internal.rocksdb.ColumnFamily; +import org.apache.ignite.internal.storage.StorageException; +import org.apache.ignite.internal.storage.index.SortedIndexDescriptor; +import org.apache.ignite.internal.storage.index.SortedIndexStorage; +import org.apache.ignite.internal.storage.rocksdb.index.RocksDbSortedIndexStorage; +import org.rocksdb.RocksDBException; + +class SortedIndices { + private final SortedIndexDescriptor descriptor; + + private final ColumnFamily columnFamily; + + private final ConcurrentMap<Integer, SortedIndexStorage> storages = new ConcurrentHashMap<>(); Review Comment: Why is it a map and not array, for example? ########## modules/storage-api/src/main/java/org/apache/ignite/internal/storage/index/SortedIndexStorage.java: ########## @@ -74,8 +74,13 @@ public interface SortedIndexStorage { * @throws IllegalArgumentException If backwards flag is passed and backwards iteration is not supported by the storage. */ Cursor<IndexRow> scan( - @Nullable BinaryTuple lowerBound, - @Nullable BinaryTuple upperBound, + @Nullable BinaryTuplePrefix lowerBound, + @Nullable BinaryTuplePrefix upperBound, @MagicConstant(flagsFromClass = SortedIndexStorage.class) int flags ); + + /** + * Removes all data from this index. + */ + void destroy(); Review Comment: Why "void" and not "Future<?>"? ########## modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/ColumnFamilyUtils.java: ########## @@ -58,9 +64,35 @@ static ColumnFamilyType fromCfName(String cfName) { return PARTITION; } else if (HASH_INDEX_CF_NAME.equals(cfName)) { return HASH_INDEX; + } else if (cfName.startsWith(SORTED_INDEX_CF_PREFIX)) { + return SORTED_INDEX; } else { return UNKNOWN; } } } + + /** + * Creates a column family name by index ID. + * + * @param indexId Index ID. + * @return Column family name. + * + * @see #sortedIndexId + */ + static String sortedIndexCfName(UUID indexId) { + return SORTED_INDEX_CF_PREFIX + indexId; + } + + /** + * Extracts a Sorted Index ID from the given Column Family name. + * + * @param cfName Column Family name. + * @return Sorted Index ID. + * + * @see #sortedIndexCfName + */ + static UUID sortedIndexId(String cfName) { Review Comment: Since CF name is byte[], can't we write UUID as a series of bytes? ########## modules/schema/src/main/java/org/apache/ignite/internal/schema/BinaryTuplePrefix.java: ########## @@ -0,0 +1,71 @@ +/* + * 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.schema; + +import java.math.BigDecimal; +import java.nio.ByteBuffer; +import org.apache.ignite.internal.binarytuple.BinaryTuplePrefixBuilder; +import org.apache.ignite.internal.binarytuple.BinaryTupleReader; +import org.apache.ignite.internal.schema.row.InternalTuple; + +/** + * Class that represents a Binary Tuple Prefix. + * + * @see BinaryTuplePrefixBuilder BinaryTuplePrefixBuilder for information about the Binary Tuple Prefix format. + */ +public class BinaryTuplePrefix extends BinaryTupleReader implements InternalTuple { + /** Tuple schema. */ + private final BinaryTupleSchema schema; + + /** + * Constructor. + * + * @param schema Full Tuple schema. + * @param bytes Serialized representation of a Binary Tuple Prefix. + */ + public BinaryTuplePrefix(BinaryTupleSchema schema, byte[] bytes) { + super(schema.elementCount() + 1, bytes); + this.schema = schema; + } + + /** + * Constructor. + * + * @param schema Full Tuple schema. + * @param buffer Serialized representation of a Binary Tuple Prefix. + */ + public BinaryTuplePrefix(BinaryTupleSchema schema, ByteBuffer buffer) { + super(schema.elementCount() + 1, buffer); + this.schema = schema; + } + + @Override + public int count() { + return elementCount(); + } + + @Override + public BigDecimal decimalValue(int index) { + return decimalValue(index, schema.element(index).decimalScale); + } + + @Override + public int elementCount() { + return intValue(super.elementCount() - 1); Review Comment: So it's a legit column? Pretty unexpected. I thought that you would just allocate 4 bytes at the end and that's it. Doesn't this approach complicate the comparison? -- 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]
