You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@druid.apache.org by GitBox <gi...@apache.org> on 2022/02/25 05:45:04 UTC

[GitHub] [druid] clintropolis commented on a change in pull request #12279: Store null columns in the segments

clintropolis commented on a change in pull request #12279:
URL: https://github.com/apache/druid/pull/12279#discussion_r814486368



##########
File path: processing/pom.xml
##########
@@ -99,6 +99,10 @@
             <groupId>commons-net</groupId>
             <artifactId>commons-net</artifactId>
         </dependency>
+        <dependency>
+            <groupId>org.apache.commons</groupId>
+            <artifactId>commons-compress</artifactId>

Review comment:
       what uses this?

##########
File path: processing/src/main/java/org/apache/druid/segment/serde/NullColumnPartSerde.java
##########
@@ -0,0 +1,262 @@
+/*
+ * 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.druid.segment.serde;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Suppliers;
+import org.apache.druid.java.util.common.io.smoosh.FileSmoosher;
+import org.apache.druid.query.extraction.ExtractionFn;
+import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
+import org.apache.druid.segment.ColumnValueSelector;
+import org.apache.druid.segment.DimensionSelector;
+import org.apache.druid.segment.NilColumnValueSelector;
+import org.apache.druid.segment.column.BitmapIndex;
+import org.apache.druid.segment.column.BitmapIndexes;
+import org.apache.druid.segment.column.DictionaryEncodedColumn;
+import org.apache.druid.segment.column.NumericColumn;
+import org.apache.druid.segment.data.BitmapSerdeFactory;
+import org.apache.druid.segment.data.IndexedInts;
+import org.apache.druid.segment.data.ReadableOffset;
+import org.apache.druid.segment.vector.MultiValueDimensionVectorSelector;
+import org.apache.druid.segment.vector.NilVectorSelector;
+import org.apache.druid.segment.vector.ReadableVectorOffset;
+import org.apache.druid.segment.vector.SingleValueDimensionVectorSelector;
+import org.apache.druid.segment.vector.VectorObjectSelector;
+import org.apache.druid.segment.vector.VectorValueSelector;
+
+import javax.annotation.Nullable;
+import java.nio.channels.WritableByteChannel;
+import java.util.Objects;
+
+/**
+ * A ColumnPartSerde to read and write null-only columns.
+ * Its serializer is no-op as nothing is stored for null-only columns.
+ * Its deserializer creates necessary column metadata and indexes when the column is read.
+ */
+public class NullColumnPartSerde implements ColumnPartSerde
+{
+  private static final Serializer NOOP_SERIALIZER = new Serializer()
+  {
+    @Override
+    public long getSerializedSize()
+    {
+      return 0;
+    }
+
+    @Override
+    public void writeTo(WritableByteChannel channel, FileSmoosher smoosher)
+    {
+    }
+  };
+
+  private final int numRows;
+  private final BitmapSerdeFactory bitmapSerdeFactory;
+  private final NullNumericColumn nullNumericColumn;
+  private final NullDictionaryEncodedColumn nullDictionaryEncodedColumn;
+  private final BitmapIndex bitmapIndex;
+
+  @JsonCreator
+  public NullColumnPartSerde(
+      @JsonProperty("numRows") int numRows,
+      @JsonProperty("bitmapSerdeFactory") BitmapSerdeFactory bitmapSerdeFactory
+  )
+  {
+    this.numRows = numRows;
+    this.bitmapSerdeFactory = bitmapSerdeFactory;
+    this.nullNumericColumn = new NullNumericColumn();
+    this.nullDictionaryEncodedColumn = new NullDictionaryEncodedColumn();
+    this.bitmapIndex = BitmapIndexes.forNilColumn(() -> numRows, bitmapSerdeFactory.getBitmapFactory());
+  }
+
+  @JsonProperty
+  public int getNumRows()
+  {
+    return numRows;
+  }
+
+  @JsonProperty
+  public BitmapSerdeFactory getBitmapSerdeFactory()
+  {
+    return bitmapSerdeFactory;
+  }
+
+  @Nullable
+  @Override
+  public Serializer getSerializer()
+  {
+    return NOOP_SERIALIZER;
+  }
+
+  @Override
+  public Deserializer getDeserializer()
+  {
+    return (buffer, builder, columnConfig) -> builder
+        .setHasMultipleValues(false)
+        .setHasNulls(true)
+        .setFilterable(true)
+        .setBitmapIndex(Suppliers.ofInstance(bitmapIndex))
+        .setNumericColumnSupplier(Suppliers.ofInstance(nullNumericColumn))
+        .setDictionaryEncodedColumnSupplier(Suppliers.ofInstance(nullDictionaryEncodedColumn));

Review comment:
       this isn't correct, both of these statements both set the column supplier in the builder, I think you probably only need the dictionary encoded column supplier, which i think should behave correctly, and then can drop the `NullNumericColumn`?
   
   Also, does this column have no type?

##########
File path: processing/src/main/java/org/apache/druid/segment/IndexMergerV9.java
##########
@@ -809,6 +933,63 @@ private void mergeCapabilities(
     }
   }
 
+  /**
+   * Creates a merged columnCapabilities to merge two queryableIndexes.
+   * This method first snapshots a pair of capabilities and then merges them.
+   */
+  @Nullable
+  private static ColumnCapabilitiesImpl mergeCapabilities(
+      @Nullable final ColumnCapabilities capabilities,
+      @Nullable final ColumnCapabilities other,
+      CoercionLogic coercionLogic
+  )
+  {
+    ColumnCapabilitiesImpl merged = ColumnCapabilitiesImpl.snapshot(capabilities, coercionLogic);
+    ColumnCapabilitiesImpl otherSnapshot = ColumnCapabilitiesImpl.snapshot(other, coercionLogic);
+    if (merged == null) {
+      return otherSnapshot;
+    } else if (otherSnapshot == null) {
+      return merged;
+    }
+
+    if (!Objects.equals(merged.getType(), otherSnapshot.getType())
+        || !Objects.equals(merged.getComplexTypeName(), otherSnapshot.getComplexTypeName())
+        || !Objects.equals(merged.getElementType(), otherSnapshot.getElementType())) {
+      throw new ISE(
+          "Cannot merge columns of type[%s] and [%s]",
+          merged.getType(),
+          otherSnapshot.getType()
+      );
+    }
+
+    merged.setDictionaryEncoded(merged.isDictionaryEncoded().or(otherSnapshot.isDictionaryEncoded()).isTrue());
+    merged.setHasMultipleValues(merged.hasMultipleValues().or(otherSnapshot.hasMultipleValues()).isTrue());
+    merged.setDictionaryValuesSorted(
+        merged.areDictionaryValuesSorted().and(otherSnapshot.areDictionaryValuesSorted()).isTrue()
+    );
+    merged.setDictionaryValuesUnique(
+        merged.areDictionaryValuesUnique().and(otherSnapshot.areDictionaryValuesUnique()).isTrue()
+    );
+    merged.setHasNulls(merged.hasNulls().or(other.hasNulls()).isTrue());
+    // When merging persisted queryableIndexes in the same ingestion job,
+    // all queryableIndexes should have the exact same hasBitmapIndexes flag set which is set in the ingestionSpec.
+    // One exception is null-only columns as they always do NOT have bitmap indexes no matter whether the flag is set
+    // in the ingestionSpec. As a result, the mismatch checked in the if clause below can happen
+    // when one of the columnCapability is from a real column and another is from a null-only column.
+    // See NullColumnPartSerde for how columnCapability is created for null-only columns.
+    // When the mismatch is found, we prefer the flag set in the ingestionSpec over
+    // the columnCapability of null-only columns.
+    if (merged.hasBitmapIndexes() != otherSnapshot.hasBitmapIndexes()) {
+      merged.setHasBitmapIndexes(false);
+    }

Review comment:
       is this line the reason that you can't use the merge function of `ColumnCapabilities`? if so that seems pretty sad, I wonder if there is a way to make it work for all uses...




-- 
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-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org