yifan-c commented on code in PR #110: URL: https://github.com/apache/cassandra-analytics/pull/110#discussion_r2141235828
########## cassandra-five-zero-types/src/main/java/org/apache/cassandra/spark/reader/ComplexTypeBuffer.java: ########## @@ -0,0 +1,118 @@ +/* + * 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.spark.reader; + +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.List; + +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.db.marshal.ByteBufferAccessor; +import org.apache.cassandra.db.marshal.ListType; +import org.apache.cassandra.db.marshal.MapType; +import org.apache.cassandra.db.marshal.SetType; +import org.apache.cassandra.db.marshal.UserType; +import org.apache.cassandra.db.rows.Cell; +import org.apache.cassandra.serializers.CollectionSerializer; + +/** + * ComplexTypeBuffer is a util class for reconstructing multi-cell data into complex types such as unfrozen lists, maps, sets, or UDTs. + * ComplexTypeBuffer buffers all the cell ByteBuffers then reconstructs as a single ByteBuffer. + */ +public abstract class ComplexTypeBuffer +{ + private final List<ByteBuffer> buffers; + private final int cellCount; + private int length = 0; + + public ComplexTypeBuffer(int cellCount, int bufferSize) + { + this.cellCount = cellCount; + this.buffers = new ArrayList<>(bufferSize); + } + + public static ComplexTypeBuffer newBuffer(AbstractType<?> type, int cellCount) + { + ComplexTypeBuffer buffer; + if (type instanceof SetType) + { + buffer = new SetBuffer(cellCount); + } + else if (type instanceof ListType) + { + buffer = new ListBuffer(cellCount); + } + else if (type instanceof MapType) + { + buffer = new MapBuffer(cellCount); + } + else if (type instanceof UserType) + { + buffer = new UdtBuffer(cellCount); + } + else + { + throw new IllegalStateException("Unexpected type deserializing CQL Collection: " + type); + } + return buffer; + } + + public void addCell(Cell cell) + { + add(cell.buffer()); // Copy over value + } + + void add(ByteBuffer buffer) + { + buffers.add(buffer); + length += buffer.remaining(); + } + + ByteBuffer build() + { + ByteBuffer result = ByteBuffer.allocate(Integer.BYTES + (buffers.size() * Integer.BYTES) + length); + result.putInt(cellCount); + for (ByteBuffer buffer : buffers) + { + result.putInt(buffer.remaining()); + result.put(buffer); + } + // Cast to ByteBuffer required when compiling with Java 8 + return (ByteBuffer) result.flip(); + } + + /** + * Pack the cell ByteBuffers into a single ByteBuffer using Cassandra's packing algorithm. + * It is similar to {@link #build()}, but encoding the data differently. + * + * @return a single ByteBuffer with all cell ByteBuffers encoded. + */ + public ByteBuffer pack() + { + // See CollectionSerializer.deserialize for why using the protocol v3 variant is the right thing to do. + // TODO(c4c5): Do we need to specify ProtocolVersion.V3 as last parameter? Review Comment: The ProtocolVersion can be removed safely. See https://issues.apache.org/jira/browse/CASSANDRA-18114 Can you remove the comment at line#109 (that comes from the four-zero subproject)? To avoid any confusion. ########## cassandra-five-zero-bridge/src/main/java/org/apache/cassandra/spark/reader/SSTableReader.java: ########## @@ -0,0 +1,898 @@ +/* + * 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.spark.reader; + +import java.io.DataInputStream; +import java.io.EOFException; +import java.io.IOError; +import java.io.IOException; +import java.io.InputStream; +import java.math.BigInteger; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Function; +import java.util.stream.Collectors; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Streams; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.bridge.TokenRange; +import org.apache.cassandra.cql3.ColumnIdentifier; +import org.apache.cassandra.db.DecoratedKey; +import org.apache.cassandra.db.DeletionTime; +import org.apache.cassandra.db.RegularAndStaticColumns; +import org.apache.cassandra.db.SerializationHeader; +import org.apache.cassandra.db.UnfilteredDeserializer; +import org.apache.cassandra.db.filter.ColumnFilter; +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.db.marshal.UTF8Type; +import org.apache.cassandra.db.rows.DeserializationHelper; +import org.apache.cassandra.db.rows.EncodingStats; +import org.apache.cassandra.db.rows.Row; +import org.apache.cassandra.db.rows.Unfiltered; +import org.apache.cassandra.db.rows.UnfilteredRowIterator; +import org.apache.cassandra.dht.Token; +import org.apache.cassandra.io.sstable.Descriptor; +import org.apache.cassandra.io.sstable.ISSTableScanner; +import org.apache.cassandra.io.sstable.SSTableSimpleIterator; +import org.apache.cassandra.io.sstable.format.Version; +import org.apache.cassandra.io.sstable.format.bti.PartitionIndex; +import org.apache.cassandra.io.sstable.indexsummary.IndexSummary; +import org.apache.cassandra.io.sstable.metadata.MetadataComponent; +import org.apache.cassandra.io.sstable.metadata.MetadataType; +import org.apache.cassandra.io.sstable.metadata.StatsMetadata; +import org.apache.cassandra.io.sstable.metadata.ValidationMetadata; +import org.apache.cassandra.io.util.ChannelProxy; +import org.apache.cassandra.io.util.DataInputPlus; +import org.apache.cassandra.io.util.DataInputStreamPlusImpl; +import org.apache.cassandra.io.util.File; +import org.apache.cassandra.io.util.FileHandle; +import org.apache.cassandra.io.util.ReadOnlyInputStreamFileChannel; +import org.apache.cassandra.net.MessagingService; +import org.apache.cassandra.schema.ColumnMetadata; +import org.apache.cassandra.schema.DroppedColumn; +import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.service.ActiveRepairService; +import org.apache.cassandra.spark.data.FileType; +import org.apache.cassandra.spark.data.SSTable; +import org.apache.cassandra.analytics.reader.common.RawInputStream; +import org.apache.cassandra.spark.reader.common.SSTableStreamException; +import org.apache.cassandra.spark.sparksql.filters.PartitionKeyFilter; +import org.apache.cassandra.spark.sparksql.filters.PruneColumnFilter; +import org.apache.cassandra.spark.sparksql.filters.SparkRangeFilter; +import org.apache.cassandra.analytics.stats.Stats; +import org.apache.cassandra.spark.utils.ByteBufferUtils; +import org.apache.cassandra.spark.utils.Pair; +import org.apache.cassandra.spark.utils.ThrowableUtils; +import org.apache.cassandra.spark.utils.streaming.BufferingInputStream; +import org.apache.cassandra.utils.ByteBufferUtil; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +@SuppressWarnings("unused") +public class SSTableReader implements SparkSSTableReader, Scannable +{ + private static final Logger LOGGER = LoggerFactory.getLogger(SSTableReader.class); + + private final TableMetadata metadata; + @NotNull + private final SSTable ssTable; + private final StatsMetadata statsMetadata; + @NotNull + private final Version version; + @NotNull + private final DecoratedKey first; + @NotNull + private final DecoratedKey last; + @NotNull + private final BigInteger firstToken; + @NotNull + private final BigInteger lastToken; + private final SerializationHeader header; + private final DeserializationHelper helper; + @NotNull + private final AtomicReference<SSTableStreamReader> reader = new AtomicReference<>(null); + @Nullable + private final SparkRangeFilter sparkRangeFilter; + @NotNull + private final List<PartitionKeyFilter> partitionKeyFilters; + @NotNull + private final Stats stats; + @Nullable + private Long startOffset = null; + private Long openedNanos = null; + @NotNull + private final Function<StatsMetadata, Boolean> isRepaired; + + public static class Builder + { + @NotNull + final TableMetadata metadata; + @NotNull + final SSTable ssTable; + @Nullable + PruneColumnFilter columnFilter = null; + boolean readIndexOffset = true; + @NotNull + Stats stats = Stats.DoNothingStats.INSTANCE; + boolean useIncrementalRepair = true; + boolean isRepairPrimary = false; + Function<StatsMetadata, Boolean> isRepaired = stats -> stats.repairedAt != ActiveRepairService.UNREPAIRED_SSTABLE; + @Nullable + SparkRangeFilter sparkRangeFilter = null; + @NotNull + final List<PartitionKeyFilter> partitionKeyFilters = new ArrayList<>(); + + Builder(@NotNull TableMetadata metadata, @NotNull SSTable ssTable) + { + this.metadata = metadata; + this.ssTable = ssTable; + } + + public Builder withSparkRangeFilter(@Nullable SparkRangeFilter sparkRangeFilter) + { + this.sparkRangeFilter = sparkRangeFilter; + return this; + } + + public Builder withPartitionKeyFilters(@Nullable Collection<PartitionKeyFilter> partitionKeyFilters) + { + if (partitionKeyFilters != null) + { + this.partitionKeyFilters.addAll(partitionKeyFilters); + } + return this; + } + + public Builder withPartitionKeyFilter(@NotNull PartitionKeyFilter partitionKeyFilter) + { + partitionKeyFilters.add(partitionKeyFilter); + return this; + } + + public Builder withColumnFilter(@Nullable PruneColumnFilter columnFilter) + { + this.columnFilter = columnFilter; + return this; + } + + public Builder withReadIndexOffset(boolean readIndexOffset) + { + this.readIndexOffset = readIndexOffset; + return this; + } + + public Builder withStats(@NotNull Stats stats) + { + this.stats = stats; + return this; + } + + public Builder useIncrementalRepair(boolean useIncrementalRepair) + { + this.useIncrementalRepair = useIncrementalRepair; + return this; + } + + public Builder isRepairPrimary(boolean isRepairPrimary) + { + this.isRepairPrimary = isRepairPrimary; + return this; + } + + public Builder withIsRepairedFunction(Function<StatsMetadata, Boolean> isRepaired) + { + this.isRepaired = isRepaired; + return this; + } + + public SSTableReader build() throws IOException + { + return new SSTableReader(metadata, + ssTable, + sparkRangeFilter, + partitionKeyFilters, + columnFilter, + readIndexOffset, + stats, + useIncrementalRepair, + isRepairPrimary, + isRepaired); + } + } + + public static Builder builder(@NotNull TableMetadata metadata, @NotNull SSTable ssTable) + { + return new Builder(metadata, ssTable); + } + + // CHECKSTYLE IGNORE: Constructor with many parameters + public SSTableReader(@NotNull TableMetadata metadata, + @NotNull SSTable ssTable, + @Nullable SparkRangeFilter sparkRangeFilter, + @NotNull List<PartitionKeyFilter> partitionKeyFilters, + @Nullable PruneColumnFilter columnFilter, + boolean readIndexOffset, + @NotNull Stats stats, + boolean useIncrementalRepair, + boolean isRepairPrimary, + @NotNull Function<StatsMetadata, Boolean> isRepaired) throws IOException + { + long startTimeNanos = System.nanoTime(); + long now; + this.ssTable = ssTable; + this.stats = stats; + this.isRepaired = isRepaired; + this.sparkRangeFilter = sparkRangeFilter; + + Descriptor descriptor = ReaderUtils.constructDescriptor(metadata.keyspace, metadata.name, ssTable); + this.version = descriptor.version; + + SummaryDbUtils.Summary summary = null; + Pair<DecoratedKey, DecoratedKey> keys = null; + try + { + if (ssTable.isBigFormat()) + { + now = System.nanoTime(); + summary = SSTableCache.INSTANCE.keysFromSummary(metadata, ssTable); + if (summary != null) + { + stats.readSummaryDb(ssTable, System.nanoTime() - now); + keys = Pair.of(summary.first(), summary.last()); + } + if (keys == null) + { + LOGGER.warn("Could not load first and last key from Summary.db file, so attempting Index.db fileName={}", + ssTable.getDataFileName()); + } + } + } + catch (IOException exception) + { + LOGGER.warn("Failed to read Summary.db file ssTable='{}'", ssTable, exception); + } + + if (keys == null) + { + now = System.nanoTime(); + keys = SSTableCache.INSTANCE.keysFromIndex(metadata, ssTable); + stats.readIndexDb(ssTable, System.nanoTime() - now); + } + + if (keys == null) + { + throw new IOException("Could not load SSTable first or last tokens"); + } + + this.first = keys.left; + this.last = keys.right; + this.firstToken = ReaderUtils.tokenToBigInteger(first.getToken()); + this.lastToken = ReaderUtils.tokenToBigInteger(last.getToken()); + TokenRange readerRange = range(); + + List<PartitionKeyFilter> matchingKeyFilters = partitionKeyFilters.stream() + .filter(filter -> readerRange.contains(filter.token())) + .collect(Collectors.toList()); + boolean overlapsSparkRange = sparkRangeFilter == null || SparkSSTableReader.overlaps(this, sparkRangeFilter.tokenRange()); + if (!overlapsSparkRange // SSTable doesn't overlap with Spark worker token range + || (matchingKeyFilters.isEmpty() && !partitionKeyFilters.isEmpty())) // No matching partition key filters overlap with SSTable + { + this.partitionKeyFilters = Collections.emptyList(); + stats.skippedSSTable(sparkRangeFilter, partitionKeyFilters, firstToken, lastToken); + LOGGER.info("Ignoring SSTableReader with firstToken={} lastToken={}, does not overlap with any filter", + firstToken, lastToken); + statsMetadata = null; + header = null; + helper = null; + this.metadata = null; + return; + } + + if (!matchingKeyFilters.isEmpty()) + { + List<PartitionKeyFilter> matchInBloomFilter = + ReaderUtils.filterKeyInBloomFilter(ssTable, metadata.partitioner, descriptor, matchingKeyFilters); + this.partitionKeyFilters = ImmutableList.copyOf(matchInBloomFilter); + + // Check if required keys are actually present + if (matchInBloomFilter.isEmpty() || !ReaderUtils.anyFilterKeyInIndex(ssTable, metadata, descriptor, matchInBloomFilter)) + { + if (matchInBloomFilter.isEmpty()) + { + stats.missingInBloomFilter(); + } + else + { + stats.missingInIndex(); + } + LOGGER.info("Ignoring SSTable {}, no match found in index file for key filters", + this.ssTable.getDataFileName()); + statsMetadata = null; + header = null; + helper = null; + this.metadata = null; + return; + } + } + else + { + this.partitionKeyFilters = ImmutableList.copyOf(partitionKeyFilters); + } + + Map<MetadataType, MetadataComponent> componentMap = SSTableCache.INSTANCE.componentMapFromStats(ssTable, descriptor); + + ValidationMetadata validation = (ValidationMetadata) componentMap.get(MetadataType.VALIDATION); + if (validation != null && !validation.partitioner.equals(metadata.partitioner.getClass().getName())) + { + throw new IllegalStateException("Partitioner in ValidationMetadata does not match TableMetaData: " + + validation.partitioner + " vs. " + metadata.partitioner.getClass().getName()); + } + + this.statsMetadata = (StatsMetadata) componentMap.get(MetadataType.STATS); + SerializationHeader.Component headerComp = (SerializationHeader.Component) componentMap.get(MetadataType.HEADER); + if (headerComp == null) + { + throw new IOException("Cannot read SSTable if cannot deserialize stats header info"); + } + + if (useIncrementalRepair && !isRepairPrimary && isRepaired()) + { + stats.skippedRepairedSSTable(ssTable, statsMetadata.repairedAt); + LOGGER.info("Ignoring repaired SSTable on non-primary repair replica ssTable='{}' repairedAt={}", + ssTable, statsMetadata.repairedAt); + header = null; + helper = null; + this.metadata = null; + return; + } + + Set<String> columnNames = Streams.concat(metadata.columns().stream(), + metadata.staticColumns().stream()) + .map(column -> column.name.toString()) + .collect(Collectors.toSet()); + Map<ByteBuffer, DroppedColumn> droppedColumns = new HashMap<>(); + droppedColumns.putAll(buildDroppedColumns(metadata.keyspace, + metadata.name, + ssTable, + headerComp.getRegularColumns(), + columnNames, + ColumnMetadata.Kind.REGULAR)); + droppedColumns.putAll(buildDroppedColumns(metadata.keyspace, + metadata.name, + ssTable, + headerComp.getStaticColumns(), + columnNames, + ColumnMetadata.Kind.STATIC)); + if (!droppedColumns.isEmpty()) + { + LOGGER.info("Rebuilding table metadata with dropped columns numDroppedColumns={} ssTable='{}'", + droppedColumns.size(), ssTable); + metadata = metadata.unbuild().droppedColumns(droppedColumns).build(); + } + + this.header = headerComp.toHeader(metadata); + this.helper = new DeserializationHelper(metadata, + MessagingService.VERSION_30, + DeserializationHelper.Flag.FROM_REMOTE, + buildColumnFilter(metadata, columnFilter)); + this.metadata = metadata; + + if (readIndexOffset) + { + IndexSummary finalSummary = summary != null ? summary.summary() : null; + extractRange(sparkRangeFilter, partitionKeyFilters) + .ifPresent(range -> readOffsets(finalSummary, range)); + } + else + { + LOGGER.warn("Reading SSTable without looking up start/end offset, performance will potentially be degraded"); + } + + // Open SSTableStreamReader so opened in parallel inside thread pool + // and buffered + ready to go when CompactionIterator starts reading + reader.set(new SSTableStreamReader()); + stats.openedSSTable(ssTable, System.nanoTime() - startTimeNanos); + this.openedNanos = System.nanoTime(); + } + + private static Map<ByteBuffer, DroppedColumn> buildDroppedColumns(String keyspace, + String table, + SSTable ssTable, + Map<ByteBuffer, AbstractType<?>> columns, + Set<String> columnNames, + ColumnMetadata.Kind kind) + { + Map<ByteBuffer, DroppedColumn> droppedColumns = new HashMap<>(); + for (Map.Entry<ByteBuffer, AbstractType<?>> entry : columns.entrySet()) + { + String colName = UTF8Type.instance.getString((entry.getKey())); + if (!columnNames.contains(colName)) + { + AbstractType<?> type = entry.getValue(); + LOGGER.warn("Dropped column found colName={} sstable='{}'", colName, ssTable); + ColumnMetadata column = new ColumnMetadata(keyspace, + table, + ColumnIdentifier.getInterned(colName, true), + type, + ColumnMetadata.NO_POSITION, + kind, + null); // Cassandra 4.x vs 5.x + long droppedTime = TimeUnit.MILLISECONDS.toMicros(System.currentTimeMillis()) + - TimeUnit.MINUTES.toMicros(60); + droppedColumns.put(entry.getKey(), new DroppedColumn(column, droppedTime)); + } + } + return droppedColumns; + } + + /** + * Merge all the partition key filters to give the token range we care about. + * If no partition key filters, then use the Spark worker token range. + * + * @param sparkRangeFilter optional spark range filter + * @param partitionKeyFilters list of partition key filters + * @return the token range we care about for this Spark worker + */ + public static Optional<TokenRange> extractRange(@Nullable SparkRangeFilter sparkRangeFilter, + @NotNull List<PartitionKeyFilter> partitionKeyFilters) + { + Optional<TokenRange> partitionKeyRange = partitionKeyFilters.stream() + .map(PartitionKeyFilter::tokenRange) + .reduce(TokenRange::merge); + return partitionKeyRange.isPresent() + ? partitionKeyRange + : Optional.ofNullable(sparkRangeFilter != null ? sparkRangeFilter.tokenRange() : null); + } + + /** + * Read Data.db offsets by binary searching Summary.db into Index.db, then reading offsets in Index.db + * + * @param indexSummary Summary.db index summary + * @param range token range we care about for this Spark worker + */ + private void readOffsets(IndexSummary indexSummary, TokenRange range) + { + try + { + if (indexSummary != null) + { + // BIG format + // If start is null we failed to find an overlapping token in the Index.db file, + // this is unlikely as we already pre-filter the SSTable based on the start-end token range. + // But in this situation we read the entire Data.db file to be safe, even if it hits performance. + startOffset = IndexDbUtils.findDataDbOffset(indexSummary, range, metadata.partitioner, ssTable, stats); + if (startOffset == null) + { + LOGGER.error("Failed to find Data.db start offset, performance will be degraded sstable='{}'", ssTable); + } + } + else + { + // BTI format + try (InputStream primaryIndex = ssTable.openPrimaryIndexStream()) + { + File file = new File(ssTable.getDataFileName()); + long size = ssTable.length(FileType.PARTITIONS_INDEX); + try (ReadOnlyInputStreamFileChannel fileChannel = new ReadOnlyInputStreamFileChannel((BufferingInputStream<?>) primaryIndex, size); + ChannelProxy proxy = new ChannelProxy(file, fileChannel); + FileHandle fileHandle = new FileHandle.Builder(file).complete(f -> proxy); + PartitionIndex partitionIndex = PartitionIndex.load(fileHandle, metadata.partitioner, false); Review Comment: Disabling preloading makes sense to avoid downloading data over the wire. Maybe add a comment for posterity. ########## cassandra-five-zero-bridge/src/main/java/org/apache/cassandra/spark/reader/SSTableReader.java: ########## @@ -0,0 +1,898 @@ +/* + * 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.spark.reader; + +import java.io.DataInputStream; +import java.io.EOFException; +import java.io.IOError; +import java.io.IOException; +import java.io.InputStream; +import java.math.BigInteger; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Function; +import java.util.stream.Collectors; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Streams; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.bridge.TokenRange; +import org.apache.cassandra.cql3.ColumnIdentifier; +import org.apache.cassandra.db.DecoratedKey; +import org.apache.cassandra.db.DeletionTime; +import org.apache.cassandra.db.RegularAndStaticColumns; +import org.apache.cassandra.db.SerializationHeader; +import org.apache.cassandra.db.UnfilteredDeserializer; +import org.apache.cassandra.db.filter.ColumnFilter; +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.db.marshal.UTF8Type; +import org.apache.cassandra.db.rows.DeserializationHelper; +import org.apache.cassandra.db.rows.EncodingStats; +import org.apache.cassandra.db.rows.Row; +import org.apache.cassandra.db.rows.Unfiltered; +import org.apache.cassandra.db.rows.UnfilteredRowIterator; +import org.apache.cassandra.dht.Token; +import org.apache.cassandra.io.sstable.Descriptor; +import org.apache.cassandra.io.sstable.ISSTableScanner; +import org.apache.cassandra.io.sstable.SSTableSimpleIterator; +import org.apache.cassandra.io.sstable.format.Version; +import org.apache.cassandra.io.sstable.format.bti.PartitionIndex; +import org.apache.cassandra.io.sstable.indexsummary.IndexSummary; +import org.apache.cassandra.io.sstable.metadata.MetadataComponent; +import org.apache.cassandra.io.sstable.metadata.MetadataType; +import org.apache.cassandra.io.sstable.metadata.StatsMetadata; +import org.apache.cassandra.io.sstable.metadata.ValidationMetadata; +import org.apache.cassandra.io.util.ChannelProxy; +import org.apache.cassandra.io.util.DataInputPlus; +import org.apache.cassandra.io.util.DataInputStreamPlusImpl; +import org.apache.cassandra.io.util.File; +import org.apache.cassandra.io.util.FileHandle; +import org.apache.cassandra.io.util.ReadOnlyInputStreamFileChannel; +import org.apache.cassandra.net.MessagingService; +import org.apache.cassandra.schema.ColumnMetadata; +import org.apache.cassandra.schema.DroppedColumn; +import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.service.ActiveRepairService; +import org.apache.cassandra.spark.data.FileType; +import org.apache.cassandra.spark.data.SSTable; +import org.apache.cassandra.analytics.reader.common.RawInputStream; +import org.apache.cassandra.spark.reader.common.SSTableStreamException; +import org.apache.cassandra.spark.sparksql.filters.PartitionKeyFilter; +import org.apache.cassandra.spark.sparksql.filters.PruneColumnFilter; +import org.apache.cassandra.spark.sparksql.filters.SparkRangeFilter; +import org.apache.cassandra.analytics.stats.Stats; +import org.apache.cassandra.spark.utils.ByteBufferUtils; +import org.apache.cassandra.spark.utils.Pair; +import org.apache.cassandra.spark.utils.ThrowableUtils; +import org.apache.cassandra.spark.utils.streaming.BufferingInputStream; +import org.apache.cassandra.utils.ByteBufferUtil; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +@SuppressWarnings("unused") +public class SSTableReader implements SparkSSTableReader, Scannable +{ + private static final Logger LOGGER = LoggerFactory.getLogger(SSTableReader.class); + + private final TableMetadata metadata; + @NotNull + private final SSTable ssTable; + private final StatsMetadata statsMetadata; + @NotNull + private final Version version; + @NotNull + private final DecoratedKey first; + @NotNull + private final DecoratedKey last; + @NotNull + private final BigInteger firstToken; + @NotNull + private final BigInteger lastToken; + private final SerializationHeader header; + private final DeserializationHelper helper; + @NotNull + private final AtomicReference<SSTableStreamReader> reader = new AtomicReference<>(null); + @Nullable + private final SparkRangeFilter sparkRangeFilter; + @NotNull + private final List<PartitionKeyFilter> partitionKeyFilters; + @NotNull + private final Stats stats; + @Nullable + private Long startOffset = null; + private Long openedNanos = null; + @NotNull + private final Function<StatsMetadata, Boolean> isRepaired; + + public static class Builder + { + @NotNull + final TableMetadata metadata; + @NotNull + final SSTable ssTable; + @Nullable + PruneColumnFilter columnFilter = null; + boolean readIndexOffset = true; + @NotNull + Stats stats = Stats.DoNothingStats.INSTANCE; + boolean useIncrementalRepair = true; + boolean isRepairPrimary = false; + Function<StatsMetadata, Boolean> isRepaired = stats -> stats.repairedAt != ActiveRepairService.UNREPAIRED_SSTABLE; + @Nullable + SparkRangeFilter sparkRangeFilter = null; + @NotNull + final List<PartitionKeyFilter> partitionKeyFilters = new ArrayList<>(); + + Builder(@NotNull TableMetadata metadata, @NotNull SSTable ssTable) + { + this.metadata = metadata; + this.ssTable = ssTable; + } + + public Builder withSparkRangeFilter(@Nullable SparkRangeFilter sparkRangeFilter) + { + this.sparkRangeFilter = sparkRangeFilter; + return this; + } + + public Builder withPartitionKeyFilters(@Nullable Collection<PartitionKeyFilter> partitionKeyFilters) + { + if (partitionKeyFilters != null) + { + this.partitionKeyFilters.addAll(partitionKeyFilters); + } + return this; + } + + public Builder withPartitionKeyFilter(@NotNull PartitionKeyFilter partitionKeyFilter) + { + partitionKeyFilters.add(partitionKeyFilter); + return this; + } + + public Builder withColumnFilter(@Nullable PruneColumnFilter columnFilter) + { + this.columnFilter = columnFilter; + return this; + } + + public Builder withReadIndexOffset(boolean readIndexOffset) + { + this.readIndexOffset = readIndexOffset; + return this; + } + + public Builder withStats(@NotNull Stats stats) + { + this.stats = stats; + return this; + } + + public Builder useIncrementalRepair(boolean useIncrementalRepair) + { + this.useIncrementalRepair = useIncrementalRepair; + return this; + } + + public Builder isRepairPrimary(boolean isRepairPrimary) + { + this.isRepairPrimary = isRepairPrimary; + return this; + } + + public Builder withIsRepairedFunction(Function<StatsMetadata, Boolean> isRepaired) + { + this.isRepaired = isRepaired; + return this; + } + + public SSTableReader build() throws IOException + { + return new SSTableReader(metadata, + ssTable, + sparkRangeFilter, + partitionKeyFilters, + columnFilter, + readIndexOffset, + stats, + useIncrementalRepair, + isRepairPrimary, + isRepaired); + } + } + + public static Builder builder(@NotNull TableMetadata metadata, @NotNull SSTable ssTable) + { + return new Builder(metadata, ssTable); + } + + // CHECKSTYLE IGNORE: Constructor with many parameters + public SSTableReader(@NotNull TableMetadata metadata, + @NotNull SSTable ssTable, + @Nullable SparkRangeFilter sparkRangeFilter, + @NotNull List<PartitionKeyFilter> partitionKeyFilters, + @Nullable PruneColumnFilter columnFilter, + boolean readIndexOffset, + @NotNull Stats stats, + boolean useIncrementalRepair, + boolean isRepairPrimary, + @NotNull Function<StatsMetadata, Boolean> isRepaired) throws IOException + { + long startTimeNanos = System.nanoTime(); + long now; + this.ssTable = ssTable; + this.stats = stats; + this.isRepaired = isRepaired; + this.sparkRangeFilter = sparkRangeFilter; + + Descriptor descriptor = ReaderUtils.constructDescriptor(metadata.keyspace, metadata.name, ssTable); + this.version = descriptor.version; + + SummaryDbUtils.Summary summary = null; + Pair<DecoratedKey, DecoratedKey> keys = null; + try + { + if (ssTable.isBigFormat()) + { + now = System.nanoTime(); + summary = SSTableCache.INSTANCE.keysFromSummary(metadata, ssTable); + if (summary != null) + { + stats.readSummaryDb(ssTable, System.nanoTime() - now); + keys = Pair.of(summary.first(), summary.last()); + } + if (keys == null) + { + LOGGER.warn("Could not load first and last key from Summary.db file, so attempting Index.db fileName={}", + ssTable.getDataFileName()); + } + } + } + catch (IOException exception) + { + LOGGER.warn("Failed to read Summary.db file ssTable='{}'", ssTable, exception); + } + + if (keys == null) + { + now = System.nanoTime(); + keys = SSTableCache.INSTANCE.keysFromIndex(metadata, ssTable); + stats.readIndexDb(ssTable, System.nanoTime() - now); + } + + if (keys == null) + { + throw new IOException("Could not load SSTable first or last tokens"); + } + + this.first = keys.left; + this.last = keys.right; + this.firstToken = ReaderUtils.tokenToBigInteger(first.getToken()); + this.lastToken = ReaderUtils.tokenToBigInteger(last.getToken()); + TokenRange readerRange = range(); + + List<PartitionKeyFilter> matchingKeyFilters = partitionKeyFilters.stream() + .filter(filter -> readerRange.contains(filter.token())) + .collect(Collectors.toList()); + boolean overlapsSparkRange = sparkRangeFilter == null || SparkSSTableReader.overlaps(this, sparkRangeFilter.tokenRange()); + if (!overlapsSparkRange // SSTable doesn't overlap with Spark worker token range + || (matchingKeyFilters.isEmpty() && !partitionKeyFilters.isEmpty())) // No matching partition key filters overlap with SSTable + { + this.partitionKeyFilters = Collections.emptyList(); + stats.skippedSSTable(sparkRangeFilter, partitionKeyFilters, firstToken, lastToken); + LOGGER.info("Ignoring SSTableReader with firstToken={} lastToken={}, does not overlap with any filter", + firstToken, lastToken); + statsMetadata = null; + header = null; + helper = null; + this.metadata = null; + return; + } + + if (!matchingKeyFilters.isEmpty()) + { + List<PartitionKeyFilter> matchInBloomFilter = + ReaderUtils.filterKeyInBloomFilter(ssTable, metadata.partitioner, descriptor, matchingKeyFilters); + this.partitionKeyFilters = ImmutableList.copyOf(matchInBloomFilter); + + // Check if required keys are actually present + if (matchInBloomFilter.isEmpty() || !ReaderUtils.anyFilterKeyInIndex(ssTable, metadata, descriptor, matchInBloomFilter)) + { + if (matchInBloomFilter.isEmpty()) + { + stats.missingInBloomFilter(); + } + else + { + stats.missingInIndex(); + } + LOGGER.info("Ignoring SSTable {}, no match found in index file for key filters", + this.ssTable.getDataFileName()); + statsMetadata = null; + header = null; + helper = null; + this.metadata = null; + return; + } + } + else + { + this.partitionKeyFilters = ImmutableList.copyOf(partitionKeyFilters); + } + + Map<MetadataType, MetadataComponent> componentMap = SSTableCache.INSTANCE.componentMapFromStats(ssTable, descriptor); + + ValidationMetadata validation = (ValidationMetadata) componentMap.get(MetadataType.VALIDATION); + if (validation != null && !validation.partitioner.equals(metadata.partitioner.getClass().getName())) + { + throw new IllegalStateException("Partitioner in ValidationMetadata does not match TableMetaData: " + + validation.partitioner + " vs. " + metadata.partitioner.getClass().getName()); + } + + this.statsMetadata = (StatsMetadata) componentMap.get(MetadataType.STATS); + SerializationHeader.Component headerComp = (SerializationHeader.Component) componentMap.get(MetadataType.HEADER); + if (headerComp == null) + { + throw new IOException("Cannot read SSTable if cannot deserialize stats header info"); + } + + if (useIncrementalRepair && !isRepairPrimary && isRepaired()) + { + stats.skippedRepairedSSTable(ssTable, statsMetadata.repairedAt); + LOGGER.info("Ignoring repaired SSTable on non-primary repair replica ssTable='{}' repairedAt={}", + ssTable, statsMetadata.repairedAt); + header = null; + helper = null; + this.metadata = null; + return; + } + + Set<String> columnNames = Streams.concat(metadata.columns().stream(), + metadata.staticColumns().stream()) + .map(column -> column.name.toString()) + .collect(Collectors.toSet()); + Map<ByteBuffer, DroppedColumn> droppedColumns = new HashMap<>(); + droppedColumns.putAll(buildDroppedColumns(metadata.keyspace, + metadata.name, + ssTable, + headerComp.getRegularColumns(), + columnNames, + ColumnMetadata.Kind.REGULAR)); + droppedColumns.putAll(buildDroppedColumns(metadata.keyspace, + metadata.name, + ssTable, + headerComp.getStaticColumns(), + columnNames, + ColumnMetadata.Kind.STATIC)); + if (!droppedColumns.isEmpty()) + { + LOGGER.info("Rebuilding table metadata with dropped columns numDroppedColumns={} ssTable='{}'", + droppedColumns.size(), ssTable); + metadata = metadata.unbuild().droppedColumns(droppedColumns).build(); + } + + this.header = headerComp.toHeader(metadata); + this.helper = new DeserializationHelper(metadata, + MessagingService.VERSION_30, + DeserializationHelper.Flag.FROM_REMOTE, + buildColumnFilter(metadata, columnFilter)); + this.metadata = metadata; + + if (readIndexOffset) + { + IndexSummary finalSummary = summary != null ? summary.summary() : null; + extractRange(sparkRangeFilter, partitionKeyFilters) + .ifPresent(range -> readOffsets(finalSummary, range)); + } + else + { + LOGGER.warn("Reading SSTable without looking up start/end offset, performance will potentially be degraded"); + } + + // Open SSTableStreamReader so opened in parallel inside thread pool + // and buffered + ready to go when CompactionIterator starts reading + reader.set(new SSTableStreamReader()); + stats.openedSSTable(ssTable, System.nanoTime() - startTimeNanos); + this.openedNanos = System.nanoTime(); + } + + private static Map<ByteBuffer, DroppedColumn> buildDroppedColumns(String keyspace, + String table, + SSTable ssTable, + Map<ByteBuffer, AbstractType<?>> columns, + Set<String> columnNames, + ColumnMetadata.Kind kind) + { + Map<ByteBuffer, DroppedColumn> droppedColumns = new HashMap<>(); + for (Map.Entry<ByteBuffer, AbstractType<?>> entry : columns.entrySet()) + { + String colName = UTF8Type.instance.getString((entry.getKey())); + if (!columnNames.contains(colName)) + { + AbstractType<?> type = entry.getValue(); + LOGGER.warn("Dropped column found colName={} sstable='{}'", colName, ssTable); + ColumnMetadata column = new ColumnMetadata(keyspace, + table, + ColumnIdentifier.getInterned(colName, true), + type, + ColumnMetadata.NO_POSITION, + kind, + null); // Cassandra 4.x vs 5.x + long droppedTime = TimeUnit.MILLISECONDS.toMicros(System.currentTimeMillis()) + - TimeUnit.MINUTES.toMicros(60); + droppedColumns.put(entry.getKey(), new DroppedColumn(column, droppedTime)); + } + } + return droppedColumns; + } + + /** + * Merge all the partition key filters to give the token range we care about. + * If no partition key filters, then use the Spark worker token range. + * + * @param sparkRangeFilter optional spark range filter + * @param partitionKeyFilters list of partition key filters + * @return the token range we care about for this Spark worker + */ + public static Optional<TokenRange> extractRange(@Nullable SparkRangeFilter sparkRangeFilter, + @NotNull List<PartitionKeyFilter> partitionKeyFilters) + { + Optional<TokenRange> partitionKeyRange = partitionKeyFilters.stream() + .map(PartitionKeyFilter::tokenRange) + .reduce(TokenRange::merge); + return partitionKeyRange.isPresent() + ? partitionKeyRange + : Optional.ofNullable(sparkRangeFilter != null ? sparkRangeFilter.tokenRange() : null); + } + + /** + * Read Data.db offsets by binary searching Summary.db into Index.db, then reading offsets in Index.db + * + * @param indexSummary Summary.db index summary + * @param range token range we care about for this Spark worker + */ + private void readOffsets(IndexSummary indexSummary, TokenRange range) + { + try + { + if (indexSummary != null) + { + // BIG format + // If start is null we failed to find an overlapping token in the Index.db file, + // this is unlikely as we already pre-filter the SSTable based on the start-end token range. + // But in this situation we read the entire Data.db file to be safe, even if it hits performance. + startOffset = IndexDbUtils.findDataDbOffset(indexSummary, range, metadata.partitioner, ssTable, stats); + if (startOffset == null) + { + LOGGER.error("Failed to find Data.db start offset, performance will be degraded sstable='{}'", ssTable); + } + } + else + { + // BTI format + try (InputStream primaryIndex = ssTable.openPrimaryIndexStream()) + { + File file = new File(ssTable.getDataFileName()); + long size = ssTable.length(FileType.PARTITIONS_INDEX); + try (ReadOnlyInputStreamFileChannel fileChannel = new ReadOnlyInputStreamFileChannel((BufferingInputStream<?>) primaryIndex, size); + ChannelProxy proxy = new ChannelProxy(file, fileChannel); + FileHandle fileHandle = new FileHandle.Builder(file).complete(f -> proxy); + PartitionIndex partitionIndex = PartitionIndex.load(fileHandle, metadata.partitioner, false); + PartitionIndex.Reader reader = partitionIndex.openReader()) + { + Token.KeyBound minKeyBound = metadata.partitioner.getTokenFactory().fromString(range.lowerEndpoint().toString()).minKeyBound(); + Long startOffsetCandidate = reader.floor(minKeyBound, (position, assumeNoMatch, partitionPosition) -> position); + if (startOffsetCandidate == null) + { + startOffset = 0L; + } + else if (startOffsetCandidate < 0) + { + // we got the offset from data file + startOffset = ~startOffsetCandidate; + } + } + } + } Review Comment: I am thinking of this change, for the following reasons: 1. Unify the handling when `startOffset == null` for both BIG and BTI. 2. In BTI branch -- since only Murmur3 and Random partitioners are supported, we can create the corresponding token directly, w/o ser/deser via string value. 3. In BTI branch -- the filtering token range is open-closed. Its start token is excluded. Therefore, `ceiling`/immediate greater branch of the max keybound of the excluded token should be the key included by the token range. 4. Still BTI branch -- added the error handling for the invalid start offset found. ```suggestion if (indexSummary != null) { // BIG format // If start is null we failed to find an overlapping token in the Index.db file, // this is unlikely as we already pre-filter the SSTable based on the start-end token range. // But in this situation we read the entire Data.db file to be safe, even if it hits performance. startOffset = IndexDbUtils.findDataDbOffset(indexSummary, range, metadata.partitioner, ssTable, stats); } else { // BTI format try (InputStream primaryIndex = ssTable.openPrimaryIndexStream()) { File file = new File(ssTable.getDataFileName()); long size = ssTable.length(FileType.PARTITIONS_INDEX); try (ReadOnlyInputStreamFileChannel fileChannel = new ReadOnlyInputStreamFileChannel((BufferingInputStream<?>) primaryIndex, size); ChannelProxy proxy = new ChannelProxy(file, fileChannel); FileHandle fileHandle = new FileHandle.Builder(file).complete(f -> proxy); PartitionIndex partitionIndex = PartitionIndex.load(fileHandle, metadata.partitioner, false); PartitionIndex.Reader reader = partitionIndex.openReader()) { Token startToken = null; if (metadata.partitioner instanceof Murmur3Partitioner) { startToken = new Murmur3Partitioner.LongToken(range.lowerEndpoint().longValue()); } else { startToken = new RandomPartitioner.BigIntegerToken(range.lowerEndpoint()); } Token.KeyBound maxKeyBound = startToken.maxKeyBound(); startOffset = reader.ceiling(maxKeyBound, (position, assumeNoMatch, partitionPosition) -> position); if (startOffset != null) { // we got the offset from data file startOffset = ~startOffset; if (startOffset < 0) { LOGGER.error("Found invalid start offset in Data.db. Read from the begining of the file."); startOffset = null; } } } } } if (startOffset == null) { LOGGER.error("Failed to find Data.db start offset, performance will be degraded sstable='{}'", ssTable); } ``` -- 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]
