lukasz-antoniak commented on code in PR #131:
URL: 
https://github.com/apache/cassandra-analytics/pull/131#discussion_r2270412500


##########
cassandra-five-zero-bridge/src/main/java/org/apache/cassandra/spark/reader/ReaderUtils.java:
##########
@@ -0,0 +1,623 @@
+/*
+ * 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.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.InvocationTargetException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.EnumMap;
+import java.util.EnumSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.zip.CRC32;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.primitives.Ints;
+
+import org.apache.cassandra.db.Clustering;
+import org.apache.cassandra.db.ClusteringPrefix;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.SerializationHeader;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.ByteBufferAccessor;
+import org.apache.cassandra.db.marshal.CompositeType;
+import org.apache.cassandra.db.marshal.TypeParser;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.db.rows.EncodingStats;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.io.sstable.CorruptSSTableException;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.format.SSTableFormat;
+import org.apache.cassandra.io.sstable.format.Version;
+import org.apache.cassandra.io.sstable.metadata.MetadataComponent;
+import org.apache.cassandra.io.sstable.metadata.MetadataType;
+import org.apache.cassandra.io.sstable.metadata.ValidationMetadata;
+import org.apache.cassandra.io.util.DataInputBuffer;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.RebufferingChannelInputStream;
+import org.apache.cassandra.io.util.File;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.spark.data.SSTable;
+import org.apache.cassandra.spark.sparksql.filters.PartitionKeyFilter;
+import org.apache.cassandra.spark.utils.ByteBufferUtils;
+import org.apache.cassandra.spark.utils.Pair;
+import org.apache.cassandra.spark.utils.streaming.BufferingInputStream;
+import org.apache.cassandra.utils.BloomFilter;
+import org.apache.cassandra.utils.BloomFilterSerializer;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.TokenUtils;
+import org.apache.cassandra.utils.vint.VIntCoding;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.cassandra.utils.FBUtilities.updateChecksumInt;
+
+@SuppressWarnings("WeakerAccess")
+public final class ReaderUtils extends TokenUtils
+{
+    private static final int CHECKSUM_LENGTH = 4;  // CRC32
+    private static final Constructor<?> SERIALIZATION_HEADER =
+    
Arrays.stream(SerializationHeader.Component.class.getDeclaredConstructors())
+          .filter(constructor -> constructor.getParameterCount() == 5)
+          .findFirst()
+          .orElseThrow(() -> new RuntimeException("Could not find 
SerializationHeader.Component constructor"));
+    public static final ByteBuffer SUPER_COLUMN_MAP_COLUMN = 
ByteBufferUtil.EMPTY_BYTE_BUFFER;
+
+    public static Descriptor constructDescriptor(@NotNull String keyspace, 
@NotNull String table, @NotNull SSTable ssTable)
+    {
+        File file = ReaderUtils.constructFilename(keyspace, table, 
ssTable.getDataFileName());
+        return Descriptor.fromFile(file);
+    }
+
+    /**
+     * Constructs full file path for a given combination of keyspace, table, 
and data file name,
+     * while adjusting for data files with non-standard names prefixed with 
keyspace and table
+     *
+     * @param keyspace Name of the keyspace
+     * @param table    Name of the table
+     * @param filename Name of the data file
+     * @return A full file path, adjusted for non-standard file names
+     */
+    @VisibleForTesting
+    @NotNull
+    public static File constructFilename(@NotNull String keyspace, @NotNull 
String table, @NotNull String filename)
+    {
+        String[] components = filename.split("-");
+        if (components.length == 6
+            && components[0].equals(keyspace)
+            && components[1].equals(table))
+        {
+            filename = filename.substring(keyspace.length() + table.length() + 
2);
+        }
+
+        return new File(String.format("./%s/%s", keyspace, table), filename);
+    }
+
+    static
+    {
+        SERIALIZATION_HEADER.setAccessible(true);
+    }
+
+    private ReaderUtils()
+    {
+        super();
+        throw new IllegalStateException(getClass() + " is static utility class 
and shall not be instantiated");
+    }
+
+    static ByteBuffer encodeCellName(TableMetadata metadata,
+                                     ClusteringPrefix clustering,
+                                     ByteBuffer columnName,
+                                     ByteBuffer collectionElement)
+    {
+        boolean isStatic = clustering == Clustering.STATIC_CLUSTERING;
+
+        if (!TableMetadata.Flag.isCompound(metadata.flags))
+        {
+            if (isStatic)
+            {
+                return columnName;
+            }
+
+            assert clustering.size() == 1 : "Expected clustering size to be 1, 
but was " + clustering.size();
+            return clustering.bufferAt(0);
+        }
+
+        // We use comparator.size() rather than clustering.size() because of 
static clusterings
+        int clusteringSize = metadata.comparator.size();
+        int size = clusteringSize + 
(TableMetadata.Flag.isDense(metadata.flags) ? 0 : 1)
+                   + (collectionElement == null ? 0 : 1);
+        if (TableMetadata.Flag.isSuper(metadata.flags))
+        {
+            size = clusteringSize + 1;
+        }
+
+        ByteBuffer[] values = new ByteBuffer[size];
+        for (int index = 0; index < clusteringSize; index++)
+        {
+            if (isStatic)
+            {
+                values[index] = ByteBufferUtil.EMPTY_BYTE_BUFFER;
+                continue;
+            }
+
+            ByteBuffer value = clustering.bufferAt(index);
+            // We can have null (only for dense compound tables for backward 
compatibility reasons),
+            // but that means we're done and should stop there as far as 
building the composite is concerned
+            if (value == null)
+            {
+                return CompositeType.build(ByteBufferAccessor.instance, 
Arrays.copyOfRange(values, 0, index));
+            }
+
+            values[index] = value;
+        }
+
+        if (TableMetadata.Flag.isSuper(metadata.flags))
+        {
+            // We need to set the "column" (in thrift terms) name, i.e. the 
value corresponding to the subcomparator.
+            // What it is depends on whether this is a cell for a declared 
"static" column
+            // or a "dynamic" column part of the super-column internal map.
+            assert columnName != null;  // This should never be null for 
supercolumns, see decodeForSuperColumn() above
+            values[clusteringSize] = columnName.equals(SUPER_COLUMN_MAP_COLUMN)
+                                     ? collectionElement
+                                     : columnName;
+        }
+        else
+        {
+            if (!TableMetadata.Flag.isDense(metadata.flags))
+            {
+                values[clusteringSize] = columnName;
+            }
+            if (collectionElement != null)
+            {
+                values[clusteringSize + 1] = collectionElement;
+            }
+        }
+
+        return CompositeType.build(ByteBufferAccessor.instance, isStatic, 
values);
+    }
+
+    @Nullable
+    public static Pair<DecoratedKey, DecoratedKey> keysFromIndex(@NotNull 
TableMetadata metadata,
+                                                                 @NotNull 
SSTable ssTable) throws IOException
+    {
+        return keysFromIndex(metadata.partitioner, ssTable);
+    }
+
+    @Nullable
+    public static Pair<DecoratedKey, DecoratedKey> keysFromIndex(@NotNull 
IPartitioner partitioner,
+                                                                 @NotNull 
SSTable ssTable) throws IOException
+    {
+        try (InputStream primaryIndex = ssTable.openPrimaryIndexStream())
+        {
+            if (primaryIndex != null)
+            {
+                Pair<ByteBuffer, ByteBuffer> keys = 
primaryIndexReadFirstAndLastKey(primaryIndex);
+                return Pair.of(partitioner.decorateKey(keys.left), 
partitioner.decorateKey(keys.right));
+            }
+        }
+        return null;
+    }
+
+    public static boolean anyFilterKeyInIndex(@NotNull SSTable ssTable,
+                                              @NotNull 
List<PartitionKeyFilter> filters) throws IOException
+    {
+        if (filters.isEmpty())
+        {
+            return false;
+        }
+
+        try (InputStream primaryIndex = ssTable.openPrimaryIndexStream())
+        {
+            if (primaryIndex != null)
+            {
+                return primaryIndexContainsAnyKey(primaryIndex, filters);
+            }
+        }
+
+        return true; // could not read primary index, so to be safe assume it 
contains the keys
+    }
+
+    public static Map<MetadataType, MetadataComponent> 
deserializeStatsMetadata(String keyspace,
+                                                                               
 String table,
+                                                                               
 SSTable ssTable,
+                                                                               
 EnumSet<MetadataType> selectedTypes) throws IOException
+    {
+        return deserializeStatsMetadata(ssTable, selectedTypes, 
constructDescriptor(keyspace, table, ssTable));
+    }
+
+    public static Map<MetadataType, MetadataComponent> 
deserializeStatsMetadata(SSTable ssTable,
+                                                                               
 Descriptor descriptor) throws IOException
+    {
+        return deserializeStatsMetadata(ssTable,
+                                        EnumSet.of(MetadataType.VALIDATION, 
MetadataType.STATS, MetadataType.HEADER),
+                                        descriptor);
+    }
+
+    public static Map<MetadataType, MetadataComponent> 
deserializeStatsMetadata(SSTable ssTable,
+                                                                               
 EnumSet<MetadataType> selectedTypes,
+                                                                               
 Descriptor descriptor) throws IOException
+    {
+        try (InputStream statsStream = ssTable.openStatsStream())
+        {
+            return deserializeStatsMetadata(statsStream,
+                                            selectedTypes,
+                                            descriptor);
+        }
+    }
+
+    /**
+     * Deserialize Statistics.db file to pull out metadata components needed 
for SSTable deserialization
+     *
+     * @param is            input stream for Statistics.db file
+     * @param selectedTypes enum of MetadataType to deserialize
+     * @param descriptor    SSTable file descriptor
+     * @return map of MetadataComponent for each requested MetadataType
+     * @throws IOException
+     */
+    static Map<MetadataType, MetadataComponent> 
deserializeStatsMetadata(InputStream is,
+                                                                         
EnumSet<MetadataType> selectedTypes,
+                                                                         
Descriptor descriptor) throws IOException
+    {
+        DataInputStream in = new DataInputStream(is); // Cassandra 4.x vs 5.x

Review Comment:
   Done.



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

Reply via email to