mike-tr-adamson commented on code in PR #2540: URL: https://github.com/apache/cassandra/pull/2540#discussion_r1291215803
########## src/java/org/apache/cassandra/index/sai/disk/v1/SkinnyPrimaryKeyMap.java: ########## @@ -0,0 +1,185 @@ +/* + * 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.cassandra.index.sai.disk.v1; + +import org.apache.cassandra.db.BufferDecoratedKey; +import org.apache.cassandra.db.Clustering; +import org.apache.cassandra.db.DecoratedKey; +import org.apache.cassandra.dht.IPartitioner; +import org.apache.cassandra.index.sai.disk.PrimaryKeyMap; +import org.apache.cassandra.index.sai.disk.format.IndexComponent; +import org.apache.cassandra.index.sai.disk.format.IndexDescriptor; +import org.apache.cassandra.index.sai.disk.v1.bitpack.BlockPackedReader; +import org.apache.cassandra.index.sai.disk.v1.bitpack.MonotonicBlockPackedReader; +import org.apache.cassandra.index.sai.disk.v1.bitpack.NumericValuesMeta; +import org.apache.cassandra.index.sai.disk.v1.sortedterms.SortedTermsMeta; +import org.apache.cassandra.index.sai.disk.v1.sortedterms.SortedTermsReader; +import org.apache.cassandra.index.sai.utils.PrimaryKey; +import org.apache.cassandra.io.sstable.format.SSTableReader; +import org.apache.cassandra.io.util.FileHandle; +import org.apache.cassandra.io.util.FileUtils; +import org.apache.cassandra.utils.Throwables; +import org.apache.cassandra.utils.bytecomparable.ByteComparable; +import org.apache.cassandra.utils.bytecomparable.ByteSource; +import org.apache.cassandra.utils.bytecomparable.ByteSourceInverse; + +import javax.annotation.concurrent.NotThreadSafe; +import javax.annotation.concurrent.ThreadSafe; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Arrays; + +/** + * A row-aware {@link PrimaryKeyMap} for skinny tables (those with no clustering columns). + * <p> + * This uses the following on-disk structures: + * <ul> + * <li>A block-packed structure for rowId to token lookups using {@link BlockPackedReader}. + * Uses the {@link IndexComponent#TOKEN_VALUES} component</li> + * <li>A monotonic block packed structure for rowId to partitionId lookups using {@link MonotonicBlockPackedReader}. + * Uses the {@link IndexComponent#PARTITION_SIZES} component</li> + * <li>A sorted terms structure for rowId to {@link PrimaryKey} and {@link PrimaryKey} to rowId lookups using + * {@link SortedTermsReader}. Uses the {@link IndexComponent#PARTITION_KEY_BLOCKS} and + * {@link IndexComponent#PARTITION_KEY_BLOCK_OFFSETS} components</li> + * </ul> + * + * While the {@link Factory} is threadsafe, individual instances of the {@link SkinnyPrimaryKeyMap} + * are not. + */ +@NotThreadSafe +public class SkinnyPrimaryKeyMap implements PrimaryKeyMap +{ + @ThreadSafe + public static class Factory implements PrimaryKeyMap.Factory + { + protected final MetadataSource metadataSource; + protected final LongArray.Factory tokenReaderFactory; + protected final LongArray.Factory partitionReaderFactory; + protected final SortedTermsReader partitionKeyReader; + protected final IPartitioner partitioner; + protected final PrimaryKey.Factory primaryKeyFactory; + + private FileHandle tokensFile; + private FileHandle partitionsFile; + private FileHandle partitionKeyBlockOffsetsFile; + private FileHandle partitionKeyBlocksFile; + + public Factory(IndexDescriptor indexDescriptor, SSTableReader sstable) + { + try + { + this.metadataSource = MetadataSource.loadGroupMetadata(indexDescriptor); + NumericValuesMeta tokensMeta = new NumericValuesMeta(metadataSource.get(indexDescriptor.componentName(IndexComponent.TOKEN_VALUES))); + this.tokensFile = indexDescriptor.createPerSSTableFileHandle(IndexComponent.TOKEN_VALUES); + this.tokenReaderFactory = new BlockPackedReader(tokensFile, tokensMeta); + NumericValuesMeta partitionsMeta = new NumericValuesMeta(metadataSource.get(indexDescriptor.componentName(IndexComponent.PARTITION_SIZES))); + this.partitionsFile = indexDescriptor.createPerSSTableFileHandle(IndexComponent.PARTITION_SIZES); + this.partitionReaderFactory = new MonotonicBlockPackedReader(partitionsFile, partitionsMeta); + NumericValuesMeta partitionKeyBlockOffsetsMeta = new NumericValuesMeta(metadataSource.get(indexDescriptor.componentName(IndexComponent.PARTITION_KEY_BLOCK_OFFSETS))); + SortedTermsMeta partitionKeysMeta = new SortedTermsMeta(metadataSource.get(indexDescriptor.componentName(IndexComponent.PARTITION_KEY_BLOCKS))); + this.partitionKeyBlockOffsetsFile = indexDescriptor.createPerSSTableFileHandle(IndexComponent.PARTITION_KEY_BLOCK_OFFSETS); + this.partitionKeyBlocksFile = indexDescriptor.createPerSSTableFileHandle(IndexComponent.PARTITION_KEY_BLOCKS); + this.partitionKeyReader = new SortedTermsReader(partitionKeyBlocksFile, partitionKeyBlockOffsetsFile, partitionKeysMeta, partitionKeyBlockOffsetsMeta); + this.partitioner = sstable.metadata().partitioner; + this.primaryKeyFactory = indexDescriptor.primaryKeyFactory; + } + catch (Throwable t) + { + throw Throwables.unchecked(Throwables.close(t, Arrays.asList(tokensFile, partitionsFile, partitionKeyBlocksFile, partitionKeyBlockOffsetsFile))); Review Comment: I think I have an elegant solution to this problem. I have added a `cleanup` mechanism to the `IndexDescriptor.createPer*FileHandle` methods that allow a cleanup to be done if the file handle creation fails. In this case, this allows us to pass `this::close` to the method, so any previously opened file handles that need closing will be closed. I was going to leave it there but realised we had a similar issue in `PerColumnIndexFiles`, so I made the change for per-SSTable and per-index versions of this method. -- 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] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]

