ninja: fix precondition for unclustered tables

Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/e04efab3
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/e04efab3
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/e04efab3

Branch: refs/heads/trunk
Commit: e04efab3f9a60e5e8c34c845548b6ab6d0570376
Parents: d60c783
Author: Blake Eggleston <bdeggles...@gmail.com>
Authored: Tue Nov 6 11:57:45 2018 -0800
Committer: Blake Eggleston <bdeggles...@gmail.com>
Committed: Tue Nov 6 11:57:45 2018 -0800

----------------------------------------------------------------------
 .../io/sstable/metadata/MetadataCollector.java    | 18 +++++++++++-------
 1 file changed, 11 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/e04efab3/src/java/org/apache/cassandra/io/sstable/metadata/MetadataCollector.java
----------------------------------------------------------------------
diff --git 
a/src/java/org/apache/cassandra/io/sstable/metadata/MetadataCollector.java 
b/src/java/org/apache/cassandra/io/sstable/metadata/MetadataCollector.java
index f48d0a6..437d80f 100644
--- a/src/java/org/apache/cassandra/io/sstable/metadata/MetadataCollector.java
+++ b/src/java/org/apache/cassandra/io/sstable/metadata/MetadataCollector.java
@@ -45,6 +45,7 @@ import org.apache.cassandra.utils.StreamingHistogram;
 public class MetadataCollector implements PartitionStatisticsCollector
 {
     public static final double NO_COMPRESSION_RATIO = -1.0;
+    private static final ByteBuffer[] EMPTY_CLUSTERING = new ByteBuffer[0];
 
     static EstimatedHistogram defaultCellPerPartitionCountHistogram()
     {
@@ -95,8 +96,8 @@ public class MetadataCollector implements 
PartitionStatisticsCollector
     protected double compressionRatio = NO_COMPRESSION_RATIO;
     protected StreamingHistogram.StreamingHistogramBuilder 
estimatedTombstoneDropTime = defaultTombstoneDropTimeHistogramBuilder();
     protected int sstableLevel;
-    private ClusteringPrefix minClustering = Slice.Bound.TOP;
-    private ClusteringPrefix maxClustering = Slice.Bound.BOTTOM;
+    private ClusteringPrefix minClustering = null;
+    private ClusteringPrefix maxClustering = null;
     protected boolean hasLegacyCounterShards = false;
     protected long totalColumnsSet;
     protected long totalRows;
@@ -228,8 +229,8 @@ public class MetadataCollector implements 
PartitionStatisticsCollector
 
     public MetadataCollector updateClusteringValues(ClusteringPrefix 
clustering)
     {
-        minClustering = comparator.compare(clustering, minClustering) < 0 ? 
clustering : minClustering;
-        maxClustering = comparator.compare(clustering, maxClustering) > 0 ? 
clustering : maxClustering;
+        minClustering = minClustering == null || 
comparator.compare(clustering, minClustering) < 0 ? clustering : minClustering;
+        maxClustering = maxClustering == null || 
comparator.compare(clustering, maxClustering) > 0 ? clustering : maxClustering;
         return this;
     }
 
@@ -271,7 +272,10 @@ public class MetadataCollector implements 
PartitionStatisticsCollector
 
     public Map<MetadataType, MetadataComponent> finalizeMetadata(String 
partitioner, double bloomFilterFPChance, long repairedAt, SerializationHeader 
header)
     {
-        Preconditions.checkState(comparator.compare(maxClustering, 
minClustering) >= 0);
+        Preconditions.checkState((minClustering == null && maxClustering == 
null)
+                                 || comparator.compare(maxClustering, 
minClustering) >= 0);
+        ByteBuffer[] minValues = minClustering != null ? 
minClustering.getRawValues() : EMPTY_CLUSTERING;
+        ByteBuffer[] maxValues = maxClustering != null ? 
maxClustering.getRawValues() : EMPTY_CLUSTERING;
         Map<MetadataType, MetadataComponent> components = Maps.newHashMap();
         components.put(MetadataType.VALIDATION, new 
ValidationMetadata(partitioner, bloomFilterFPChance));
         components.put(MetadataType.STATS, new 
StatsMetadata(estimatedPartitionSize,
@@ -286,8 +290,8 @@ public class MetadataCollector implements 
PartitionStatisticsCollector
                                                              compressionRatio,
                                                              
estimatedTombstoneDropTime.build(),
                                                              sstableLevel,
-                                                             
makeList(minClustering.getRawValues()),
-                                                             
makeList(maxClustering.getRawValues()),
+                                                             
makeList(minValues),
+                                                             
makeList(maxValues),
                                                              
hasLegacyCounterShards,
                                                              repairedAt,
                                                              totalColumnsSet,


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org

Reply via email to