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 2021/04/10 01:55:26 UTC

[GitHub] [druid] suneet-s commented on a change in pull request #10949: Add support to first/last aggregators for numeric types during ingestion

suneet-s commented on a change in pull request #10949:
URL: https://github.com/apache/druid/pull/10949#discussion_r610969176



##########
File path: processing/src/main/java/org/apache/druid/query/aggregation/AbstractSerializablePairSerde.java
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.query.aggregation;
+
+import com.google.common.primitives.Longs;
+import org.apache.druid.collections.SerializablePair;
+import org.apache.druid.data.input.InputRow;
+import org.apache.druid.segment.GenericColumnSerializer;
+import org.apache.druid.segment.column.ColumnBuilder;
+import org.apache.druid.segment.data.GenericIndexed;
+import org.apache.druid.segment.data.ObjectStrategy;
+import org.apache.druid.segment.serde.ComplexColumnPartSupplier;
+import org.apache.druid.segment.serde.ComplexMetricExtractor;
+import org.apache.druid.segment.serde.ComplexMetricSerde;
+import org.apache.druid.segment.serde.LargeColumnSupportedComplexColumnSerializer;
+import org.apache.druid.segment.writeout.SegmentWriteOutMedium;
+
+import javax.annotation.Nullable;
+import java.nio.ByteBuffer;
+
+/**
+ * The class serializes a Pair<Long, ?> object for double/float/longFirst and double/float/longLast aggregators

Review comment:
       Can you describe why you chose not to also make `SerializablePairLongStringSerde` not extend this class?

##########
File path: processing/src/main/java/org/apache/druid/query/aggregation/AbstractSerializablePairSerde.java
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.query.aggregation;
+
+import com.google.common.primitives.Longs;
+import org.apache.druid.collections.SerializablePair;
+import org.apache.druid.data.input.InputRow;
+import org.apache.druid.segment.GenericColumnSerializer;
+import org.apache.druid.segment.column.ColumnBuilder;
+import org.apache.druid.segment.data.GenericIndexed;
+import org.apache.druid.segment.data.ObjectStrategy;
+import org.apache.druid.segment.serde.ComplexColumnPartSupplier;
+import org.apache.druid.segment.serde.ComplexMetricExtractor;
+import org.apache.druid.segment.serde.ComplexMetricSerde;
+import org.apache.druid.segment.serde.LargeColumnSupportedComplexColumnSerializer;
+import org.apache.druid.segment.writeout.SegmentWriteOutMedium;
+
+import javax.annotation.Nullable;
+import java.nio.ByteBuffer;
+
+/**
+ * The class serializes a Pair<Long, ?> object for double/float/longFirst and double/float/longLast aggregators
+ */
+public abstract class AbstractSerializablePairSerde<T extends SerializablePair<Long, ?>> extends ComplexMetricSerde

Review comment:
       nit: change class name to `AbstractSerializableLongObjectPairSerde`
   ```suggestion
   public abstract class AbstractSerializableLongObjectPairSerde<T extends SerializablePair<Long, ?>> extends ComplexMetricSerde
   ```

##########
File path: processing/src/main/java/org/apache/druid/query/aggregation/SerializablePairLongFloatSerde.java
##########
@@ -0,0 +1,74 @@
+/*
+ * 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.query.aggregation;
+
+import org.apache.druid.common.config.NullHandling;
+
+import java.nio.ByteBuffer;
+
+/**
+ * The class serializes a Long-Float pair (SerializablePair<Long, Float>).
+ * The serialization structure is: Long:Float
+ * <p>
+ * The class is used on first/last Float aggregators to store the time and the first/last Float.
+ * Long:Float -> Timestamp:Float
+ */
+public class SerializablePairLongFloatSerde extends AbstractSerializablePairSerde<SerializablePairLongFloat>
+{
+  public static final String TYPE_NAME = "serializablePairLongFloat";
+
+  public SerializablePairLongFloatSerde()
+  {
+    super(SerializablePairLongFloat.class);
+  }
+
+  @Override
+  public String getTypeName()
+  {
+    return TYPE_NAME;
+  }
+
+  @Override
+  protected SerializablePairLongFloat toPairObject(ByteBuffer buffer)
+  {
+    final ByteBuffer readOnlyBuffer = buffer.asReadOnlyBuffer();
+    long lhs = readOnlyBuffer.getLong();
+    boolean isNotNull = readOnlyBuffer.get() == NullHandling.IS_NOT_NULL_BYTE;
+    if (isNotNull) {
+      return new SerializablePairLongFloat(lhs, readOnlyBuffer.getFloat());
+    } else {
+      return new SerializablePairLongFloat(lhs, null);
+    }
+  }
+
+  @Override
+  protected byte[] pairToBytes(SerializablePairLongFloat val)

Review comment:
       Since the super class says `val` can be null, all these implementations should be able to handle a null `val`.
   
   I haven't dug in yet to know what this means, but this same pattern exists in all 3 implementations.

##########
File path: processing/src/main/java/org/apache/druid/query/aggregation/AbstractSerializablePairSerde.java
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.query.aggregation;
+
+import com.google.common.primitives.Longs;
+import org.apache.druid.collections.SerializablePair;
+import org.apache.druid.data.input.InputRow;
+import org.apache.druid.segment.GenericColumnSerializer;
+import org.apache.druid.segment.column.ColumnBuilder;
+import org.apache.druid.segment.data.GenericIndexed;
+import org.apache.druid.segment.data.ObjectStrategy;
+import org.apache.druid.segment.serde.ComplexColumnPartSupplier;
+import org.apache.druid.segment.serde.ComplexMetricExtractor;
+import org.apache.druid.segment.serde.ComplexMetricSerde;
+import org.apache.druid.segment.serde.LargeColumnSupportedComplexColumnSerializer;
+import org.apache.druid.segment.writeout.SegmentWriteOutMedium;
+
+import javax.annotation.Nullable;
+import java.nio.ByteBuffer;
+
+/**
+ * The class serializes a Pair<Long, ?> object for double/float/longFirst and double/float/longLast aggregators
+ */
+public abstract class AbstractSerializablePairSerde<T extends SerializablePair<Long, ?>> extends ComplexMetricSerde
+{
+  private final Class<T> pairClassObject;
+
+  public AbstractSerializablePairSerde(Class<T> pairClassObject)
+  {
+    this.pairClassObject = pairClassObject;
+  }
+
+  @Override
+  public ComplexMetricExtractor getExtractor()
+  {
+    return new ComplexMetricExtractor()
+    {
+      @Override
+      public Class<T> extractedClass()
+      {
+        return pairClassObject;
+      }
+
+      @Override
+      public Object extractValue(InputRow inputRow, String metricName)
+      {
+        return inputRow.getRaw(metricName);
+      }
+    };
+  }
+
+  @Override
+  public void deserializeColumn(ByteBuffer buffer, ColumnBuilder columnBuilder)
+  {
+    final GenericIndexed column = GenericIndexed.read(buffer, getObjectStrategy(), columnBuilder.getFileMapper());
+    columnBuilder.setComplexColumnSupplier(new ComplexColumnPartSupplier(getTypeName(), column));
+  }
+
+  @Override
+  public ObjectStrategy<T> getObjectStrategy()
+  {
+    return new ObjectStrategy<T>()
+    {
+      @Override
+      public int compare(@Nullable T o1, @Nullable T o2)
+      {
+        return Longs.compare(o1.lhs, o2.lhs);
+      }
+
+      @Override
+      public Class<T> getClazz()
+      {
+        return pairClassObject;
+      }
+
+      @Override
+      public T fromByteBuffer(ByteBuffer buffer, int numBytes)
+      {
+        return toPairObject(buffer);
+      }
+
+      @Override
+      public byte[] toBytes(T val)
+      {
+        return pairToBytes(val);
+      }
+    };
+  }
+
+  @Override
+  public GenericColumnSerializer<T> getSerializer(SegmentWriteOutMedium segmentWriteOutMedium, String column)
+  {
+    return LargeColumnSupportedComplexColumnSerializer.create(segmentWriteOutMedium, column, this.getObjectStrategy());
+  }
+
+  protected abstract T toPairObject(ByteBuffer buffer);
+
+  protected abstract byte[] pairToBytes(T val);

Review comment:
       ```suggestion
     protected abstract byte[] pairToBytes(@Nullable T val);
   ```

##########
File path: processing/src/main/java/org/apache/druid/query/aggregation/AbstractSerializablePairSerde.java
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.query.aggregation;
+
+import com.google.common.primitives.Longs;
+import org.apache.druid.collections.SerializablePair;
+import org.apache.druid.data.input.InputRow;
+import org.apache.druid.segment.GenericColumnSerializer;
+import org.apache.druid.segment.column.ColumnBuilder;
+import org.apache.druid.segment.data.GenericIndexed;
+import org.apache.druid.segment.data.ObjectStrategy;
+import org.apache.druid.segment.serde.ComplexColumnPartSupplier;
+import org.apache.druid.segment.serde.ComplexMetricExtractor;
+import org.apache.druid.segment.serde.ComplexMetricSerde;
+import org.apache.druid.segment.serde.LargeColumnSupportedComplexColumnSerializer;
+import org.apache.druid.segment.writeout.SegmentWriteOutMedium;
+
+import javax.annotation.Nullable;
+import java.nio.ByteBuffer;
+
+/**
+ * The class serializes a Pair<Long, ?> object for double/float/longFirst and double/float/longLast aggregators
+ */
+public abstract class AbstractSerializablePairSerde<T extends SerializablePair<Long, ?>> extends ComplexMetricSerde
+{
+  private final Class<T> pairClassObject;
+
+  public AbstractSerializablePairSerde(Class<T> pairClassObject)
+  {
+    this.pairClassObject = pairClassObject;
+  }
+
+  @Override
+  public ComplexMetricExtractor getExtractor()
+  {
+    return new ComplexMetricExtractor()
+    {
+      @Override
+      public Class<T> extractedClass()
+      {
+        return pairClassObject;
+      }
+
+      @Override
+      public Object extractValue(InputRow inputRow, String metricName)
+      {
+        return inputRow.getRaw(metricName);
+      }
+    };
+  }
+
+  @Override
+  public void deserializeColumn(ByteBuffer buffer, ColumnBuilder columnBuilder)
+  {
+    final GenericIndexed column = GenericIndexed.read(buffer, getObjectStrategy(), columnBuilder.getFileMapper());
+    columnBuilder.setComplexColumnSupplier(new ComplexColumnPartSupplier(getTypeName(), column));
+  }
+
+  @Override
+  public ObjectStrategy<T> getObjectStrategy()
+  {
+    return new ObjectStrategy<T>()
+    {
+      @Override
+      public int compare(@Nullable T o1, @Nullable T o2)
+      {
+        return Longs.compare(o1.lhs, o2.lhs);
+      }
+
+      @Override
+      public Class<T> getClazz()
+      {
+        return pairClassObject;
+      }
+
+      @Override
+      public T fromByteBuffer(ByteBuffer buffer, int numBytes)
+      {
+        return toPairObject(buffer);
+      }
+
+      @Override
+      public byte[] toBytes(T val)

Review comment:
       ```suggestion
         public byte[] toBytes(@Nullable T val)
   ```

##########
File path: processing/src/main/java/org/apache/druid/query/aggregation/AbstractSerializablePairSerde.java
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.query.aggregation;
+
+import com.google.common.primitives.Longs;
+import org.apache.druid.collections.SerializablePair;
+import org.apache.druid.data.input.InputRow;
+import org.apache.druid.segment.GenericColumnSerializer;
+import org.apache.druid.segment.column.ColumnBuilder;
+import org.apache.druid.segment.data.GenericIndexed;
+import org.apache.druid.segment.data.ObjectStrategy;
+import org.apache.druid.segment.serde.ComplexColumnPartSupplier;
+import org.apache.druid.segment.serde.ComplexMetricExtractor;
+import org.apache.druid.segment.serde.ComplexMetricSerde;
+import org.apache.druid.segment.serde.LargeColumnSupportedComplexColumnSerializer;
+import org.apache.druid.segment.writeout.SegmentWriteOutMedium;
+
+import javax.annotation.Nullable;
+import java.nio.ByteBuffer;
+
+/**
+ * The class serializes a Pair<Long, ?> object for double/float/longFirst and double/float/longLast aggregators
+ */
+public abstract class AbstractSerializablePairSerde<T extends SerializablePair<Long, ?>> extends ComplexMetricSerde
+{
+  private final Class<T> pairClassObject;
+
+  public AbstractSerializablePairSerde(Class<T> pairClassObject)
+  {
+    this.pairClassObject = pairClassObject;
+  }
+
+  @Override
+  public ComplexMetricExtractor getExtractor()
+  {
+    return new ComplexMetricExtractor()
+    {
+      @Override
+      public Class<T> extractedClass()
+      {
+        return pairClassObject;
+      }
+
+      @Override
+      public Object extractValue(InputRow inputRow, String metricName)
+      {
+        return inputRow.getRaw(metricName);
+      }
+    };
+  }
+
+  @Override
+  public void deserializeColumn(ByteBuffer buffer, ColumnBuilder columnBuilder)
+  {
+    final GenericIndexed column = GenericIndexed.read(buffer, getObjectStrategy(), columnBuilder.getFileMapper());
+    columnBuilder.setComplexColumnSupplier(new ComplexColumnPartSupplier(getTypeName(), column));
+  }
+
+  @Override
+  public ObjectStrategy<T> getObjectStrategy()
+  {
+    return new ObjectStrategy<T>()
+    {
+      @Override
+      public int compare(@Nullable T o1, @Nullable T o2)
+      {
+        return Longs.compare(o1.lhs, o2.lhs);
+      }
+
+      @Override
+      public Class<T> getClazz()
+      {
+        return pairClassObject;
+      }
+
+      @Override
+      public T fromByteBuffer(ByteBuffer buffer, int numBytes)
+      {
+        return toPairObject(buffer);
+      }
+
+      @Override
+      public byte[] toBytes(T val)
+      {
+        return pairToBytes(val);
+      }
+    };
+  }
+
+  @Override
+  public GenericColumnSerializer<T> getSerializer(SegmentWriteOutMedium segmentWriteOutMedium, String column)
+  {
+    return LargeColumnSupportedComplexColumnSerializer.create(segmentWriteOutMedium, column, this.getObjectStrategy());
+  }
+
+  protected abstract T toPairObject(ByteBuffer buffer);
+
+  protected abstract byte[] pairToBytes(T val);

Review comment:
       javadocs please

##########
File path: processing/src/main/java/org/apache/druid/query/aggregation/AbstractSerializablePairSerde.java
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.query.aggregation;
+
+import com.google.common.primitives.Longs;
+import org.apache.druid.collections.SerializablePair;
+import org.apache.druid.data.input.InputRow;
+import org.apache.druid.segment.GenericColumnSerializer;
+import org.apache.druid.segment.column.ColumnBuilder;
+import org.apache.druid.segment.data.GenericIndexed;
+import org.apache.druid.segment.data.ObjectStrategy;
+import org.apache.druid.segment.serde.ComplexColumnPartSupplier;
+import org.apache.druid.segment.serde.ComplexMetricExtractor;
+import org.apache.druid.segment.serde.ComplexMetricSerde;
+import org.apache.druid.segment.serde.LargeColumnSupportedComplexColumnSerializer;
+import org.apache.druid.segment.writeout.SegmentWriteOutMedium;
+
+import javax.annotation.Nullable;
+import java.nio.ByteBuffer;
+
+/**
+ * The class serializes a Pair<Long, ?> object for double/float/longFirst and double/float/longLast aggregators
+ */
+public abstract class AbstractSerializablePairSerde<T extends SerializablePair<Long, ?>> extends ComplexMetricSerde
+{
+  private final Class<T> pairClassObject;
+
+  public AbstractSerializablePairSerde(Class<T> pairClassObject)
+  {
+    this.pairClassObject = pairClassObject;
+  }
+
+  @Override
+  public ComplexMetricExtractor getExtractor()
+  {
+    return new ComplexMetricExtractor()
+    {
+      @Override
+      public Class<T> extractedClass()
+      {
+        return pairClassObject;
+      }
+
+      @Override
+      public Object extractValue(InputRow inputRow, String metricName)
+      {
+        return inputRow.getRaw(metricName);
+      }
+    };
+  }
+
+  @Override
+  public void deserializeColumn(ByteBuffer buffer, ColumnBuilder columnBuilder)
+  {
+    final GenericIndexed column = GenericIndexed.read(buffer, getObjectStrategy(), columnBuilder.getFileMapper());
+    columnBuilder.setComplexColumnSupplier(new ComplexColumnPartSupplier(getTypeName(), column));
+  }
+
+  @Override
+  public ObjectStrategy<T> getObjectStrategy()
+  {
+    return new ObjectStrategy<T>()
+    {
+      @Override
+      public int compare(@Nullable T o1, @Nullable T o2)
+      {
+        return Longs.compare(o1.lhs, o2.lhs);

Review comment:
       This does not correctly handle if either o1 or o2 is null. See `StringFirstAggregatorFactory#VALUE_COMPARATOR`, we'll want a similar behavior here.
   
   Would it be possible to update the integration tests that were added to surface this error?




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

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