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/12/02 13:26:21 UTC

[GitHub] [druid] gianm commented on a change in pull request #11888: add 'TypeStrategy' to types

gianm commented on a change in pull request #11888:
URL: https://github.com/apache/druid/pull/11888#discussion_r747911389



##########
File path: core/src/main/java/org/apache/druid/segment/column/TypeStrategy.java
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.column;
+
+import org.apache.druid.common.config.NullHandling;
+
+import javax.annotation.Nullable;
+import java.nio.ByteBuffer;
+import java.util.Comparator;
+
+/**
+ * TypeStrategy provides value comparison and binary serialization for Druid types. This can be obtained for ANY Druid
+ * type via {@link TypeSignature#getStrategy()}.
+ *
+ * IMPORTANT!!! DO NOT USE THIS FOR WRITING COLUMNS, THERE ARE VERY LIKELY FAR BETTER WAYS TO DO THIS. However, if you
+ * need to store a single value or small number of values, continue reading.
+ *
+ * Implementations of this mechanism support reading and writing ONLY non-null values. To read and write nullable
+ * values and you have enough memory to burn a full byte for every value you want to store, consider using the
+ * {@link TypeStrategies#readNullableType} and {@link TypeStrategies#writeNullableType} family of
+ * methods, which will store values with a leading byte containing either {@link NullHandling#IS_NULL_BYTE} or
+ * {@link NullHandling#IS_NOT_NULL_BYTE} as appropriate. If you have a lot of values to write and a lot of nulls,
+ * consider alternative approaches to tracking your nulls.
+ *
+ * This mechanism allows using the natural {@link ByteBuffer#position()} and modify the underlying position as they
+ * operate, and also random access reads are specific offets, which do not modify the underlying position. If a method
+ * accepts an offset parameter, it does not modify the position, if not, it does.
+ *
+ * The only methods implementors are required to provide are {@link #read(ByteBuffer)},
+ * {@link #write(ByteBuffer, Object)} and {@link #estimateSizeBytes(Object)}, default implementations are provided
+ * to set and reset buffer positions as appropriate for the offset based methods, but may be overridden if a more
+ * optimized implementation is needed.
+ */
+public interface TypeStrategy<T> extends Comparator<T>
+{
+  /**
+   * The size in bytes that writing this value to memory would require, useful for constraining the values maximum size
+   *
+   * This does not include the null byte, use {@link #estimateSizeBytesNullable(Object)} instead.
+   */
+  int estimateSizeBytes(@Nullable T value);
+
+  /**
+   * The size in bytes that writing this value to memory would require, including the null byte, useful for constraining
+   * the values maximum size. If the value is null, the size will be {@link Byte#BYTES}, otherwise it will be
+   * {@link Byte#BYTES} + {@link #estimateSizeBytes(Object)}
+   */
+  default int estimateSizeBytesNullable(@Nullable T value)
+  {
+    if (value == null) {
+      return Byte.BYTES;
+    }
+    return Byte.BYTES + estimateSizeBytes(value);
+  }
+
+  /**
+   * Read a non-null value from the {@link ByteBuffer} at the current {@link ByteBuffer#position()}. This will move
+   * the underlying position by the size of the value read.
+   *
+   * The contract of this method is that any value returned from this method MUST be completely detached from the
+   * underlying {@link ByteBuffer}, since it might outlive the memory location being allocated to hold the object.
+   * In other words, if an object is memory mapped, it must be copied on heap, or relocated to another memory location
+   * that is owned by the caller with {@link #write}.
+   */
+  T read(ByteBuffer buffer);
+
+  /**
+   * Write a non-null value to the {@link ByteBuffer} at position {@link ByteBuffer#position()}. This will move the
+   * underlying position by the size of the value written.
+   *
+   * Callers should ensure the {@link ByteBuffer} has adequate capacity before writing values, use
+   * {@link #estimateSizeBytes(Object)} to determine the required size of a value before writing if the size
+   * is unknown.
+   */
+  void write(ByteBuffer buffer, T value);
+
+  /**
+   * Read a non-null value from the {@link ByteBuffer} at the requested position. This will not permanently move the
+   * underlying {@link ByteBuffer#position()}.
+   *
+   * The contract of this method is that any value returned from this method MUST be completely detached from the
+   * underlying {@link ByteBuffer}, since it might outlive the memory location being allocated to hold the object.
+   * In other words, if an object is memory mapped, it must be copied on heap, or relocated to another memory location
+   * that is owned by the caller with {@link #write}.
+   */
+  default T read(ByteBuffer buffer, int offset)
+  {
+    final int oldPosition = buffer.position();
+    buffer.position(offset);
+    T value = read(buffer);
+    buffer.position(oldPosition);

Review comment:
       Consider two changes:
   
   - Using try/finally so the buffer position is still valid if `read` throws an exception
   - Updating the javadoc to say that even though the buffer position is unchanged upon method exit, it may be changed temporarily while the method is running, so it isn't concurrency-safe.

##########
File path: core/src/main/java/org/apache/druid/segment/column/TypeStrategy.java
##########
@@ -0,0 +1,118 @@
+/*
+ * 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.column;
+
+import org.apache.druid.common.config.NullHandling;
+
+import java.nio.ByteBuffer;
+import java.util.Comparator;
+
+/**
+ * TypeStrategy provides value comparison and binary serialization for Druid types. This can be obtained for ANY Druid
+ * type via {@link TypeSignature#getStrategy()}.
+ *
+ * IMPORTANT!!! DO NOT USE THIS FOR WRITING COLUMNS, THERE ARE VERY LIKELY FAR BETTER WAYS TO DO THIS. However, if you
+ * need to store a single value or small number of values, continue reading.
+ *
+ * Most implementations of this mechanism, support reading and writing ONLY non-null values. The exception to this is
+ * {@link NullableTypeStrategy}, which might be acceptable to use if you need to read and write nullable values, AND,
+ * you have enough memory to burn a full byte for every value you want to store. It will store values with a leading
+ * byte containing either {@link NullHandling#IS_NULL_BYTE} or {@link NullHandling#IS_NOT_NULL_BYTE} as appropriate.
+ * If you have a lot of values to write and a lot of nulls, consider alternative approaches to tracking your nulls.
+ *
+ * This mechanism allows using the natural {@link ByteBuffer#position()} and modify the underlying position as they
+ * operate, and also random access reads are specific offets, which do not modify the underlying position. If a method
+ * accepts an offset parameter, it does not modify the position, if not, it does.
+ *
+ * The only methods implementors are required to provide are {@link #read(ByteBuffer)},
+ * {@link #write(ByteBuffer, Object, int)} and {@link #estimateSizeBytes(Object)}, default implementations are provided
+ * to set and reset buffer positions as appropriate for the offset based methods, but may be overridden if a more
+ * optimized implementation is needed.
+ */
+public interface TypeStrategy<T> extends Comparator<T>
+{
+  /**
+   * Estimate the size in bytes that writing this value to memory would require. This method is not required to be
+   * exactly correct, but many implementations might be. Implementations should err on the side of over-estimating if
+   * exact sizing is not efficient
+   */
+  int estimateSizeBytes(T value);
+
+
+  /**
+   * Read a non-null value from the {@link ByteBuffer} at the current {@link ByteBuffer#position()}. This will move
+   * the underlying position by the size of the value read.
+   *
+   * The contract of this method is that any value returned from this method MUST be completely detached from the
+   * underlying {@link ByteBuffer}, since it might outlive the memory location being allocated to hold the object.
+   * In other words, if an object is memory mapped, it must be copied on heap, or relocated to another memory location
+   * that is owned by the caller with {@link #write}.
+   */
+  T read(ByteBuffer buffer);
+
+  /**
+   * Write a non-null value to the {@link ByteBuffer} at position {@link ByteBuffer#position()}. This will move the
+   * underlying position by the size of the value written, and returns the number of bytes written.
+   *
+   * If writing the value would take more than 'maxSizeBytes' (or the buffer limit, whichever is smaller), this method
+   * will return a negative value indicating the number of additional bytes that would be required to fully write the
+   * value. Partial results may be written to the buffer when in this state, and the position may be left at whatever
+   * point the implementation ran out of space while writing the value.

Review comment:
       Does this mean the caller is expected to save the position? Or is the caller safe to assume it can roll back by `-retval`?
   
   i.e., is this code safe?
   
   ```
   int written = write(buf, value, maxBytes);
   if (written < 0) {
     buf.position(buf.position() + written); // roll back buf, undoing the write
   }
   ```
   
   or do callers need to do this?
   
   ```
   int oldPosition = buf.position();
   int written = write(buf, value, maxBytes);
   if (written < 0) {
     buf.position(oldPosition); // roll back buf, undoing the write
   }
   ```

##########
File path: core/src/main/java/org/apache/druid/segment/column/TypeStrategy.java
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.column;
+
+import org.apache.druid.common.config.NullHandling;
+
+import javax.annotation.Nullable;
+import java.nio.ByteBuffer;
+import java.util.Comparator;
+
+/**
+ * TypeStrategy provides value comparison and binary serialization for Druid types. This can be obtained for ANY Druid
+ * type via {@link TypeSignature#getStrategy()}.
+ *
+ * IMPORTANT!!! DO NOT USE THIS FOR WRITING COLUMNS, THERE ARE VERY LIKELY FAR BETTER WAYS TO DO THIS. However, if you
+ * need to store a single value or small number of values, continue reading.
+ *
+ * Implementations of this mechanism support reading and writing ONLY non-null values. To read and write nullable
+ * values and you have enough memory to burn a full byte for every value you want to store, consider using the
+ * {@link TypeStrategies#readNullableType} and {@link TypeStrategies#writeNullableType} family of
+ * methods, which will store values with a leading byte containing either {@link NullHandling#IS_NULL_BYTE} or
+ * {@link NullHandling#IS_NOT_NULL_BYTE} as appropriate. If you have a lot of values to write and a lot of nulls,
+ * consider alternative approaches to tracking your nulls.
+ *
+ * This mechanism allows using the natural {@link ByteBuffer#position()} and modify the underlying position as they
+ * operate, and also random access reads are specific offets, which do not modify the underlying position. If a method
+ * accepts an offset parameter, it does not modify the position, if not, it does.
+ *
+ * The only methods implementors are required to provide are {@link #read(ByteBuffer)},
+ * {@link #write(ByteBuffer, Object)} and {@link #estimateSizeBytes(Object)}, default implementations are provided
+ * to set and reset buffer positions as appropriate for the offset based methods, but may be overridden if a more
+ * optimized implementation is needed.
+ */
+public interface TypeStrategy<T> extends Comparator<T>
+{
+  /**
+   * The size in bytes that writing this value to memory would require, useful for constraining the values maximum size
+   *
+   * This does not include the null byte, use {@link #estimateSizeBytesNullable(Object)} instead.
+   */
+  int estimateSizeBytes(@Nullable T value);
+
+  /**
+   * The size in bytes that writing this value to memory would require, including the null byte, useful for constraining
+   * the values maximum size. If the value is null, the size will be {@link Byte#BYTES}, otherwise it will be
+   * {@link Byte#BYTES} + {@link #estimateSizeBytes(Object)}
+   */
+  default int estimateSizeBytesNullable(@Nullable T value)
+  {
+    if (value == null) {
+      return Byte.BYTES;
+    }
+    return Byte.BYTES + estimateSizeBytes(value);
+  }
+
+  /**
+   * Read a non-null value from the {@link ByteBuffer} at the current {@link ByteBuffer#position()}. This will move
+   * the underlying position by the size of the value read.
+   *
+   * The contract of this method is that any value returned from this method MUST be completely detached from the
+   * underlying {@link ByteBuffer}, since it might outlive the memory location being allocated to hold the object.
+   * In other words, if an object is memory mapped, it must be copied on heap, or relocated to another memory location
+   * that is owned by the caller with {@link #write}.
+   */
+  T read(ByteBuffer buffer);
+
+  /**
+   * Write a non-null value to the {@link ByteBuffer} at position {@link ByteBuffer#position()}. This will move the
+   * underlying position by the size of the value written.
+   *
+   * Callers should ensure the {@link ByteBuffer} has adequate capacity before writing values, use
+   * {@link #estimateSizeBytes(Object)} to determine the required size of a value before writing if the size
+   * is unknown.
+   */
+  void write(ByteBuffer buffer, T value);
+
+  /**
+   * Read a non-null value from the {@link ByteBuffer} at the requested position. This will not permanently move the
+   * underlying {@link ByteBuffer#position()}.
+   *
+   * The contract of this method is that any value returned from this method MUST be completely detached from the
+   * underlying {@link ByteBuffer}, since it might outlive the memory location being allocated to hold the object.
+   * In other words, if an object is memory mapped, it must be copied on heap, or relocated to another memory location
+   * that is owned by the caller with {@link #write}.
+   */
+  default T read(ByteBuffer buffer, int offset)
+  {
+    final int oldPosition = buffer.position();
+    buffer.position(offset);
+    T value = read(buffer);
+    buffer.position(oldPosition);
+    return value;
+  }
+
+  /**
+   * Write a non-null value to the {@link ByteBuffer} at the requested position. This will not permanently move the
+   * underlying {@link ByteBuffer#position()}, and returns the number of bytes written.
+   *
+   * Callers should ensure the {@link ByteBuffer} has adequate capacity before writing values, use
+   * {@link #estimateSizeBytes(Object)} to determine the required size of a value before writing if the size
+   * is unknown.
+   */
+  default int write(ByteBuffer buffer, int offset, T value)
+  {
+    final int oldPosition = buffer.position();
+    buffer.position(offset);
+    write(buffer, value);
+    final int size = buffer.position() - offset;
+    buffer.position(oldPosition);

Review comment:
       Consider two changes:
   
   - Using try/finally so the buffer position is still valid if `write` throws an exception
   - Updating the javadoc to say that even though the buffer position is unchanged upon method exit, it may be changed temporarily while the method is running, so it isn't concurrency-safe.

##########
File path: core/src/main/java/org/apache/druid/segment/column/TypeStrategy.java
##########
@@ -0,0 +1,118 @@
+/*
+ * 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.column;
+
+import org.apache.druid.common.config.NullHandling;
+
+import java.nio.ByteBuffer;
+import java.util.Comparator;
+
+/**
+ * TypeStrategy provides value comparison and binary serialization for Druid types. This can be obtained for ANY Druid
+ * type via {@link TypeSignature#getStrategy()}.
+ *
+ * IMPORTANT!!! DO NOT USE THIS FOR WRITING COLUMNS, THERE ARE VERY LIKELY FAR BETTER WAYS TO DO THIS. However, if you
+ * need to store a single value or small number of values, continue reading.
+ *
+ * Most implementations of this mechanism, support reading and writing ONLY non-null values. The exception to this is
+ * {@link NullableTypeStrategy}, which might be acceptable to use if you need to read and write nullable values, AND,
+ * you have enough memory to burn a full byte for every value you want to store. It will store values with a leading
+ * byte containing either {@link NullHandling#IS_NULL_BYTE} or {@link NullHandling#IS_NOT_NULL_BYTE} as appropriate.
+ * If you have a lot of values to write and a lot of nulls, consider alternative approaches to tracking your nulls.
+ *
+ * This mechanism allows using the natural {@link ByteBuffer#position()} and modify the underlying position as they
+ * operate, and also random access reads are specific offets, which do not modify the underlying position. If a method
+ * accepts an offset parameter, it does not modify the position, if not, it does.
+ *
+ * The only methods implementors are required to provide are {@link #read(ByteBuffer)},
+ * {@link #write(ByteBuffer, Object, int)} and {@link #estimateSizeBytes(Object)}, default implementations are provided
+ * to set and reset buffer positions as appropriate for the offset based methods, but may be overridden if a more
+ * optimized implementation is needed.
+ */
+public interface TypeStrategy<T> extends Comparator<T>
+{
+  /**
+   * Estimate the size in bytes that writing this value to memory would require. This method is not required to be
+   * exactly correct, but many implementations might be. Implementations should err on the side of over-estimating if
+   * exact sizing is not efficient
+   */
+  int estimateSizeBytes(T value);
+
+
+  /**
+   * Read a non-null value from the {@link ByteBuffer} at the current {@link ByteBuffer#position()}. This will move

Review comment:
       "non-null" isn't true for the NullableTypeStrategy impl, which is still a TypeStrategy, so this method contract should be adjusted.

##########
File path: core/src/main/java/org/apache/druid/segment/column/TypeStrategy.java
##########
@@ -0,0 +1,118 @@
+/*
+ * 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.column;
+
+import org.apache.druid.common.config.NullHandling;
+
+import java.nio.ByteBuffer;
+import java.util.Comparator;
+
+/**
+ * TypeStrategy provides value comparison and binary serialization for Druid types. This can be obtained for ANY Druid
+ * type via {@link TypeSignature#getStrategy()}.
+ *
+ * IMPORTANT!!! DO NOT USE THIS FOR WRITING COLUMNS, THERE ARE VERY LIKELY FAR BETTER WAYS TO DO THIS. However, if you
+ * need to store a single value or small number of values, continue reading.
+ *
+ * Most implementations of this mechanism, support reading and writing ONLY non-null values. The exception to this is
+ * {@link NullableTypeStrategy}, which might be acceptable to use if you need to read and write nullable values, AND,
+ * you have enough memory to burn a full byte for every value you want to store. It will store values with a leading
+ * byte containing either {@link NullHandling#IS_NULL_BYTE} or {@link NullHandling#IS_NOT_NULL_BYTE} as appropriate.
+ * If you have a lot of values to write and a lot of nulls, consider alternative approaches to tracking your nulls.
+ *
+ * This mechanism allows using the natural {@link ByteBuffer#position()} and modify the underlying position as they
+ * operate, and also random access reads are specific offets, which do not modify the underlying position. If a method
+ * accepts an offset parameter, it does not modify the position, if not, it does.
+ *
+ * The only methods implementors are required to provide are {@link #read(ByteBuffer)},
+ * {@link #write(ByteBuffer, Object, int)} and {@link #estimateSizeBytes(Object)}, default implementations are provided
+ * to set and reset buffer positions as appropriate for the offset based methods, but may be overridden if a more
+ * optimized implementation is needed.
+ */
+public interface TypeStrategy<T> extends Comparator<T>
+{
+  /**
+   * Estimate the size in bytes that writing this value to memory would require. This method is not required to be
+   * exactly correct, but many implementations might be. Implementations should err on the side of over-estimating if
+   * exact sizing is not efficient

Review comment:
       I'd include something here about how this estimate is expected to be used. (Like an example)
   
   That'll help people understand when they should use this method, and also understand how to implement it. I'm suggesting this because you gotta be careful with "estimate" methods: people have wildly differing ideas of how accurate they need to be, so it pays to be specific in the doc comments.

##########
File path: core/src/main/java/org/apache/druid/segment/column/TypeStrategy.java
##########
@@ -0,0 +1,118 @@
+/*
+ * 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.column;
+
+import org.apache.druid.common.config.NullHandling;
+
+import java.nio.ByteBuffer;
+import java.util.Comparator;
+
+/**
+ * TypeStrategy provides value comparison and binary serialization for Druid types. This can be obtained for ANY Druid
+ * type via {@link TypeSignature#getStrategy()}.
+ *
+ * IMPORTANT!!! DO NOT USE THIS FOR WRITING COLUMNS, THERE ARE VERY LIKELY FAR BETTER WAYS TO DO THIS. However, if you
+ * need to store a single value or small number of values, continue reading.
+ *
+ * Most implementations of this mechanism, support reading and writing ONLY non-null values. The exception to this is
+ * {@link NullableTypeStrategy}, which might be acceptable to use if you need to read and write nullable values, AND,
+ * you have enough memory to burn a full byte for every value you want to store. It will store values with a leading
+ * byte containing either {@link NullHandling#IS_NULL_BYTE} or {@link NullHandling#IS_NOT_NULL_BYTE} as appropriate.
+ * If you have a lot of values to write and a lot of nulls, consider alternative approaches to tracking your nulls.
+ *
+ * This mechanism allows using the natural {@link ByteBuffer#position()} and modify the underlying position as they
+ * operate, and also random access reads are specific offets, which do not modify the underlying position. If a method
+ * accepts an offset parameter, it does not modify the position, if not, it does.
+ *
+ * The only methods implementors are required to provide are {@link #read(ByteBuffer)},
+ * {@link #write(ByteBuffer, Object, int)} and {@link #estimateSizeBytes(Object)}, default implementations are provided
+ * to set and reset buffer positions as appropriate for the offset based methods, but may be overridden if a more
+ * optimized implementation is needed.
+ */
+public interface TypeStrategy<T> extends Comparator<T>

Review comment:
       Will stuff serialized with this interface be persisted to disk, sent between servers that might be running different versions of code, used in caches, etc? Or will it always be in-memory, single-server, & ephemeral?
   
   Implementers are going to want to know the answer, so they can design their formats accordingly. (If persistence is required then they might want to add a version byte, avoid using native endianness, etc.)

##########
File path: core/src/main/java/org/apache/druid/segment/column/TypeStrategy.java
##########
@@ -0,0 +1,118 @@
+/*
+ * 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.column;
+
+import org.apache.druid.common.config.NullHandling;
+
+import java.nio.ByteBuffer;
+import java.util.Comparator;
+
+/**
+ * TypeStrategy provides value comparison and binary serialization for Druid types. This can be obtained for ANY Druid
+ * type via {@link TypeSignature#getStrategy()}.
+ *
+ * IMPORTANT!!! DO NOT USE THIS FOR WRITING COLUMNS, THERE ARE VERY LIKELY FAR BETTER WAYS TO DO THIS. However, if you
+ * need to store a single value or small number of values, continue reading.
+ *
+ * Most implementations of this mechanism, support reading and writing ONLY non-null values. The exception to this is
+ * {@link NullableTypeStrategy}, which might be acceptable to use if you need to read and write nullable values, AND,
+ * you have enough memory to burn a full byte for every value you want to store. It will store values with a leading
+ * byte containing either {@link NullHandling#IS_NULL_BYTE} or {@link NullHandling#IS_NOT_NULL_BYTE} as appropriate.
+ * If you have a lot of values to write and a lot of nulls, consider alternative approaches to tracking your nulls.
+ *
+ * This mechanism allows using the natural {@link ByteBuffer#position()} and modify the underlying position as they
+ * operate, and also random access reads are specific offets, which do not modify the underlying position. If a method
+ * accepts an offset parameter, it does not modify the position, if not, it does.
+ *
+ * The only methods implementors are required to provide are {@link #read(ByteBuffer)},
+ * {@link #write(ByteBuffer, Object, int)} and {@link #estimateSizeBytes(Object)}, default implementations are provided
+ * to set and reset buffer positions as appropriate for the offset based methods, but may be overridden if a more
+ * optimized implementation is needed.
+ */
+public interface TypeStrategy<T> extends Comparator<T>
+{
+  /**
+   * Estimate the size in bytes that writing this value to memory would require. This method is not required to be
+   * exactly correct, but many implementations might be. Implementations should err on the side of over-estimating if
+   * exact sizing is not efficient
+   */
+  int estimateSizeBytes(T value);
+
+
+  /**
+   * Read a non-null value from the {@link ByteBuffer} at the current {@link ByteBuffer#position()}. This will move
+   * the underlying position by the size of the value read.
+   *
+   * The contract of this method is that any value returned from this method MUST be completely detached from the
+   * underlying {@link ByteBuffer}, since it might outlive the memory location being allocated to hold the object.
+   * In other words, if an object is memory mapped, it must be copied on heap, or relocated to another memory location
+   * that is owned by the caller with {@link #write}.
+   */
+  T read(ByteBuffer buffer);
+
+  /**
+   * Write a non-null value to the {@link ByteBuffer} at position {@link ByteBuffer#position()}. This will move the
+   * underlying position by the size of the value written, and returns the number of bytes written.
+   *
+   * If writing the value would take more than 'maxSizeBytes' (or the buffer limit, whichever is smaller), this method

Review comment:
       Should we spec this to be an error if `maxSizeBytes` is greater than `buffer.remaining()`? Seems like a weird thing for a caller to do. Or is there a reason a caller might do that?




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