lukasz-antoniak commented on code in PR #131: URL: https://github.com/apache/cassandra-analytics/pull/131#discussion_r2270419294
########## cassandra-five-zero-bridge/src/main/java/org/apache/cassandra/spark/reader/SSTableReader.java: ########## @@ -0,0 +1,859 @@ +/* + * 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.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.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.DataInputPlus; +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.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.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 + { + now = System.nanoTime(); + summary = SSTableCache.INSTANCE.keysFromSummary(metadata, ssTable); + stats.readSummaryDb(ssTable, System.nanoTime() - now); + keys = Pair.of(summary.first(), summary.last()); + } + catch (IOException exception) + { + LOGGER.warn("Failed to read Summary.db file ssTable='{}'", ssTable, exception); + } + + if (keys == null) + { + LOGGER.warn("Could not load first and last key from Summary.db file, so attempting Index.db fileName={}", + ssTable.getDataFileName()); + 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, 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, Review Comment: Please check updated code. SSTable format and version are calculated based on data file name. -- 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: commits-unsubscr...@cassandra.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org