diff --git a/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/EnrichedRowData.java b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/EnrichedRowData.java index 10951c1c9d1e0..9733e487ccc77 100644 --- a/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/EnrichedRowData.java +++ b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/EnrichedRowData.java @@ -27,6 +27,7 @@ import org.apache.flink.table.data.StringData; import org.apache.flink.table.data.TimestampData; import org.apache.flink.types.RowKind; +import org.apache.flink.types.bitmap.Bitmap; import org.apache.flink.types.variant.Variant; import java.util.List; @@ -257,6 +258,16 @@ public Variant getVariant(int pos) { } } + @Override + public Bitmap getBitmap(int pos) { + int index = indexMapping[pos]; + if (index >= 0) { + return mutableRow.getBitmap(index); + } else { + return fixedRow.getBitmap(-(index + 1)); + } + } + @Override public boolean equals(Object o) { if (this == o) { diff --git a/flink-core/pom.xml b/flink-core/pom.xml index edfd27b046298..f4d48931de3df 100644 --- a/flink-core/pom.xml +++ b/flink-core/pom.xml @@ -145,6 +145,13 @@ under the License. flink-shaded-guava + + + org.roaringbitmap + RoaringBitmap + 1.3.0 + + diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/BitmapTypeInfo.java b/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/BitmapTypeInfo.java new file mode 100644 index 0000000000000..e90c0e650cdb9 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/BitmapTypeInfo.java @@ -0,0 +1,101 @@ +/* + * 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.flink.api.common.typeinfo; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.api.common.serialization.SerializerConfig; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.base.BitmapSerializer; +import org.apache.flink.types.bitmap.Bitmap; + +/** Type information for {@link Bitmap}. */ +@PublicEvolving +public class BitmapTypeInfo extends TypeInformation { + + private static final long serialVersionUID = 1L; + + public static final BitmapTypeInfo INSTANCE = new BitmapTypeInfo(); + + private BitmapTypeInfo() {} + + @Override + public boolean isBasicType() { + return false; + } + + @Override + public boolean isTupleType() { + return false; + } + + @Override + public int getArity() { + return 1; + } + + @Override + public int getTotalFields() { + return 1; + } + + @Override + public Class getTypeClass() { + return Bitmap.class; + } + + @Override + public boolean isKeyType() { + return true; + } + + @Override + public boolean isSortKeyType() { + return false; + } + + @Override + public TypeSerializer createSerializer(SerializerConfig config) { + return BitmapSerializer.INSTANCE; + } + + @Override + public String toString() { + return Bitmap.class.getSimpleName(); + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof BitmapTypeInfo) { + BitmapTypeInfo other = (BitmapTypeInfo) obj; + return other.canEqual(this); + } else { + return false; + } + } + + @Override + public int hashCode() { + return BitmapTypeInfo.class.hashCode(); + } + + @Override + public boolean canEqual(Object obj) { + return obj instanceof BitmapTypeInfo; + } +} diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/Types.java b/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/Types.java index 3f6d5936f29b5..d353cb1333915 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/Types.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/Types.java @@ -37,6 +37,7 @@ import org.apache.flink.types.Either; import org.apache.flink.types.Row; import org.apache.flink.types.Value; +import org.apache.flink.types.bitmap.Bitmap; import org.apache.flink.types.variant.Variant; import java.lang.reflect.Field; @@ -159,6 +160,12 @@ public class Types { public static final TypeInformation VARIANT = VariantTypeInfo.INSTANCE; + /** + * Returns type information for {@link org.apache.flink.types.bitmap.Bitmap}. Supports a null + * value. + */ + public static final TypeInformation BITMAP = BitmapTypeInfo.INSTANCE; + // CHECKSTYLE.OFF: MethodName /** diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/BitmapSerializer.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/BitmapSerializer.java new file mode 100644 index 0000000000000..ac512efe63d58 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/BitmapSerializer.java @@ -0,0 +1,101 @@ +/* + * 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.flink.api.common.typeutils.base; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.typeutils.SimpleTypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; +import org.apache.flink.types.bitmap.Bitmap; + +import java.io.IOException; + +/** Serializer for {@link Bitmap}. */ +@Internal +public class BitmapSerializer extends TypeSerializerSingleton { + + public static final BitmapSerializer INSTANCE = new BitmapSerializer(); + + @Override + public boolean isImmutableType() { + return false; + } + + @Override + public Bitmap createInstance() { + return Bitmap.empty(); + } + + @Override + public Bitmap copy(Bitmap from) { + return Bitmap.from(from); + } + + @Override + public Bitmap copy(Bitmap from, Bitmap reuse) { + return copy(from); + } + + @Override + public int getLength() { + return -1; + } + + @Override + public void serialize(Bitmap record, DataOutputView target) throws IOException { + byte[] bytes = record.toBytes(); + target.writeInt(bytes.length); + target.write(bytes); + } + + @Override + public Bitmap deserialize(DataInputView source) throws IOException { + int length = source.readInt(); + byte[] bytes = new byte[length]; + source.read(bytes); + return Bitmap.fromBytes(bytes); + } + + @Override + public Bitmap deserialize(Bitmap reuse, DataInputView source) throws IOException { + return this.deserialize(source); + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + int length = source.readInt(); + target.writeInt(length); + target.write(source, length); + } + + @Override + public TypeSerializerSnapshot snapshotConfiguration() { + return new BitmapSerializerSnapshot(); + } + + @Internal + public static final class BitmapSerializerSnapshot + extends SimpleTypeSerializerSnapshot { + /** Constructor to create snapshot from serializer (writing the snapshot). */ + public BitmapSerializerSnapshot() { + super(() -> INSTANCE); + } + } +} diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java index dd0fff1e22c28..76c7e6ce39253 100644 --- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java +++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java @@ -37,6 +37,7 @@ import org.apache.flink.api.common.io.InputFormat; import org.apache.flink.api.common.typeinfo.BasicArrayTypeInfo; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; +import org.apache.flink.api.common.typeinfo.BitmapTypeInfo; import org.apache.flink.api.common.typeinfo.PrimitiveArrayTypeInfo; import org.apache.flink.api.common.typeinfo.SqlTimeTypeInfo; import org.apache.flink.api.common.typeinfo.TypeInfo; @@ -50,6 +51,7 @@ import org.apache.flink.api.java.typeutils.TypeExtractionUtils.LambdaExecutable; import org.apache.flink.types.Row; import org.apache.flink.types.Value; +import org.apache.flink.types.bitmap.Bitmap; import org.apache.flink.types.variant.Variant; import org.apache.flink.util.InstantiationUtil; import org.apache.flink.util.Preconditions; @@ -1977,6 +1979,11 @@ private TypeInformation privateGetForClass( return (TypeInformation) VariantTypeInfo.INSTANCE; } + // check for Bitmap + if (Bitmap.class.isAssignableFrom(clazz)) { + return (TypeInformation) BitmapTypeInfo.INSTANCE; + } + // check for parameterized Collections, requirement: // 1. Interface types: the underlying implementation types are not preserved across // serialization diff --git a/flink-core/src/main/java/org/apache/flink/types/bitmap/Bitmap.java b/flink-core/src/main/java/org/apache/flink/types/bitmap/Bitmap.java new file mode 100644 index 0000000000000..40d21d1a5bd22 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/types/bitmap/Bitmap.java @@ -0,0 +1,164 @@ +/* + * 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.flink.types.bitmap; + +import org.apache.flink.annotation.PublicEvolving; + +import javax.annotation.Nullable; + +/** + * A compressed data structure for storing sets of 32-bit integers. + * + *

The modifying methods in this interface modify the bitmap in place by default. Consider using + * {@link Bitmap#from(Bitmap other)} to create a copied bitmap before modification if immutability + * is required. + */ +@PublicEvolving +public interface Bitmap { + + /** Adds the value to the bitmap. */ + void add(int value); + + /** + * Adds all integers in [rangeStart,rangeEnd) to the bitmap. + * + * @param rangeStart valid range [0L, 0xFFFFFFFFL] + * @param rangeEnd valid range [0L, 0xFFFFFFFFL + 1] + */ + void add(long rangeStart, long rangeEnd); + + /** Adds the first n elements of the specified array starting at the specified offset. */ + void addN(int[] values, int offset, int n); + + /** + * Performs an in-place logical AND (intersection) operation with another bitmap. + * + *

Does nothing if {@code other} is null. + */ + void and(@Nullable Bitmap other); + + /** + * Performs an in-place logical AND-NOT (difference) operation with another bitmap, which is + * equivalent to {@code this AND (NOT other)}. + * + *

Does nothing if {@code other} is null. + */ + void andNot(@Nullable Bitmap other); + + /** Resets to an empty bitmap. */ + void clear(); + + /** Checks whether the value appears in the bitmap. */ + boolean contains(int value); + + /** Gets the number of distinct values in the bitmap. */ + int getCardinality(); + + /** Gets the number of distinct values in the bitmap. This returns a full 64-bit result. */ + long getLongCardinality(); + + /** Checks whether the bitmap is empty. */ + boolean isEmpty(); + + /** + * Performs an in-place logical OR (union) operation with another bitmap. + * + *

Does nothing if {@code other} is null. + */ + void or(@Nullable Bitmap other); + + /** Removes the value from the bitmap. */ + void remove(int value); + + /** + * Converts the bitmap to an array of 32-bit integers, the values are sorted by {@link + * Integer#compareUnsigned}. Avoid calling this method if the bitmap is too large. + */ + int[] toArray(); + + /** + * Converts the bitmap to an array of bytes. + * + *

Following the format defined in 32-bit RoaringBitmap format + * specification. + */ + byte[] toBytes(); + + /** + * Converts the bitmap to a string, the values are sorted by {@link Integer#compareUnsigned}. + * The string will be truncated and end with "..." if it is too long. + * + *

For example: + * + *

    + *
  • {@code "{}"}, {@code "{1,2,3,4,5}"} + *
  • Negative values (converted to unsigned): {@code "{0,1,4294967294,4294967295}"} + *
  • String too long: {@code "{1,2,3,...}"} + *
+ */ + String toString(); + + /** + * Performs an in-place logical XOR (symmetric difference) operation with another bitmap. + * + *

Does nothing if {@code other} is null. + */ + void xor(@Nullable Bitmap other); + + // ~ Static Methods -------------------------------------------------------------- + + /** Gets an empty bitmap. */ + static Bitmap empty() { + return RoaringBitmapData.empty(); + } + + /** Gets a copied bitmap. Returns null if {@code other} is null. */ + static Bitmap from(Bitmap other) { + if (other == null) { + return null; + } + return RoaringBitmapData.from(other); + } + + /** + * Gets a bitmap from an array of bytes. Returns null if {@code bytes} is null. + * + *

Following the format defined in 32-bit RoaringBitmap format + * specification. + * + * @throws org.apache.flink.types.DeserializationException if failed to deserialize bitmap from + * bytes + */ + static Bitmap fromBytes(byte[] bytes) { + if (bytes == null) { + return null; + } + return RoaringBitmapData.fromBytes(bytes); + } + + /** Gets a bitmap from an array of values. Returns null if {@code values} is null. */ + static Bitmap fromArray(int[] values) { + if (values == null) { + return null; + } + return RoaringBitmapData.fromArray(values); + } +} diff --git a/flink-core/src/main/java/org/apache/flink/types/bitmap/RoaringBitmapData.java b/flink-core/src/main/java/org/apache/flink/types/bitmap/RoaringBitmapData.java new file mode 100644 index 0000000000000..d9ee1988cc671 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/types/bitmap/RoaringBitmapData.java @@ -0,0 +1,238 @@ +/* + * 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.flink.types.bitmap; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.types.DeserializationException; +import org.apache.flink.util.Preconditions; + +import org.roaringbitmap.IntConsumer; +import org.roaringbitmap.RoaringBitmap; + +import javax.annotation.Nonnull; +import javax.annotation.Nullable; + +import java.nio.ByteBuffer; +import java.util.Objects; + +/** + * An internal Bitmap implementation that wraps {@link RoaringBitmap} for Flink-specific + * modifications. + */ +@Internal +public final class RoaringBitmapData implements Bitmap { + + private final RoaringBitmap roaringBitmap; + + private RoaringBitmapData() { + this.roaringBitmap = new RoaringBitmap(); + } + + private RoaringBitmapData(RoaringBitmapData other) { + this.roaringBitmap = other.roaringBitmap.clone(); + } + + private RoaringBitmapData(RoaringBitmap roaringBitmap) { + this.roaringBitmap = roaringBitmap; + } + + // ~ Static Methods ---------------------------------------------------------------- + + public static RoaringBitmapData empty() { + return new RoaringBitmapData(); + } + + public static RoaringBitmapData from(@Nonnull Bitmap other) { + return new RoaringBitmapData(toRoaringBitmapData(other)); + } + + public static RoaringBitmapData fromBytes(@Nonnull byte[] bytes) + throws DeserializationException { + RoaringBitmapData rb32 = new RoaringBitmapData(); + try { + rb32.roaringBitmap.deserialize(ByteBuffer.wrap(bytes)); + } catch (Exception e) { + throw new DeserializationException("Failed to deserialize bitmap from bytes.", e); + } + return rb32; + } + + public static RoaringBitmapData fromArray(@Nonnull int[] values) { + RoaringBitmapData rb32 = new RoaringBitmapData(); + rb32.roaringBitmap.add(values); + return rb32; + } + + /** + * Wraps the given {@link RoaringBitmap} without copying. The returned {@link RoaringBitmapData} + * shares the same internal object as the input. + */ + public static RoaringBitmapData wrap(@Nonnull RoaringBitmap roaringBitmap) { + Preconditions.checkNotNull(roaringBitmap); + return new RoaringBitmapData(roaringBitmap); + } + + private static RoaringBitmapData toRoaringBitmapData(Bitmap bm) + throws IllegalArgumentException { + if (!(bm instanceof RoaringBitmapData)) { + throw new IllegalArgumentException("Unsupported bitmap type: " + bm.getClass() + "."); + } + return (RoaringBitmapData) bm; + } + + // ~ Bitmap Interface Implementations ------------------------------------------------ + + @Override + public void add(int value) { + roaringBitmap.add(value); + } + + /** + * @throws IllegalArgumentException if rangeStart or rangeEnd is out of range. + */ + @Override + public void add(long rangeStart, long rangeEnd) throws IllegalArgumentException { + roaringBitmap.add(rangeStart, rangeEnd); + } + + /** + * @throws NullPointerException if values is null. + * @throws IllegalArgumentException if offset or n is negative, or offset + n is greater than + * values.length. + */ + @Override + public void addN(int[] values, int offset, int n) + throws NullPointerException, IllegalArgumentException { + roaringBitmap.addN(values, offset, n); + } + + @Override + public void and(@Nullable Bitmap other) { + if (other == null) { + return; + } + roaringBitmap.and(toRoaringBitmapData(other).roaringBitmap); + } + + @Override + public void andNot(@Nullable Bitmap other) { + if (other == null) { + return; + } + roaringBitmap.andNot(toRoaringBitmapData(other).roaringBitmap); + } + + @Override + public void clear() { + roaringBitmap.clear(); + } + + @Override + public boolean contains(int value) { + return roaringBitmap.contains(value); + } + + @Override + public int getCardinality() { + return roaringBitmap.getCardinality(); + } + + @Override + public long getLongCardinality() { + return roaringBitmap.getLongCardinality(); + } + + @Override + public boolean isEmpty() { + return roaringBitmap.isEmpty(); + } + + @Override + public void or(@Nullable Bitmap other) { + if (other == null) { + return; + } + roaringBitmap.or(toRoaringBitmapData(other).roaringBitmap); + } + + @Override + public void remove(int value) { + roaringBitmap.remove(value); + } + + /** + * @throws RuntimeException if the bitmap is too large. + */ + @Override + public int[] toArray() throws RuntimeException { + return roaringBitmap.toArray(); + } + + @Override + public byte[] toBytes() { + roaringBitmap.runOptimize(); + ByteBuffer buffer = ByteBuffer.allocate(roaringBitmap.serializedSizeInBytes()); + roaringBitmap.serialize(buffer); + return buffer.array(); + } + + @Override + public String toString() { + return roaringBitmap.toString(); + } + + @Override + public void xor(@Nullable Bitmap other) { + if (other == null) { + return; + } + roaringBitmap.xor(toRoaringBitmapData(other).roaringBitmap); + } + + // ~ Non-interface Methods ---------------------------------------------------------- + + /** Calls the specified consumer for each value in the bitmap. */ + public void forEach(IntConsumer consumer) { + roaringBitmap.forEach(consumer); + } + + // ~ Object Overrides --------------------------------------------------------------- + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (!(obj instanceof RoaringBitmapData)) { + return false; + } + RoaringBitmapData other = (RoaringBitmapData) obj; + return Objects.equals(roaringBitmap, other.roaringBitmap); + } + + @Override + public int hashCode() { + // RoaringBitmap#hashCode() requires both bitmaps to have the same + // runOptimize() state to produce consistent hash codes for equal bitmaps. + // Calling runOptimize() here is a semantically safe side effect that only + // changes the internal container encoding without altering the bitmap's content. + roaringBitmap.runOptimize(); + return roaringBitmap.hashCode(); + } +} diff --git a/flink-core/src/test/java/org/apache/flink/api/common/typeinfo/BitmapTypeInfoTest.java b/flink-core/src/test/java/org/apache/flink/api/common/typeinfo/BitmapTypeInfoTest.java new file mode 100644 index 0000000000000..9898e63ecce6c --- /dev/null +++ b/flink-core/src/test/java/org/apache/flink/api/common/typeinfo/BitmapTypeInfoTest.java @@ -0,0 +1,30 @@ +/* + * 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.flink.api.common.typeinfo; + +import org.apache.flink.api.common.typeutils.TypeInformationTestBase; + +/** Test for {@link BitmapTypeInfo}. */ +class BitmapTypeInfoTest extends TypeInformationTestBase { + + @Override + protected BitmapTypeInfo[] getTestData() { + return new BitmapTypeInfo[] {BitmapTypeInfo.INSTANCE}; + } +} diff --git a/flink-core/src/test/java/org/apache/flink/api/common/typeutils/base/BitmapSerializerTest.java b/flink-core/src/test/java/org/apache/flink/api/common/typeutils/base/BitmapSerializerTest.java new file mode 100644 index 0000000000000..9d8a388c5e8de --- /dev/null +++ b/flink-core/src/test/java/org/apache/flink/api/common/typeutils/base/BitmapSerializerTest.java @@ -0,0 +1,62 @@ +/* + * 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.flink.api.common.typeutils.base; + +import org.apache.flink.api.common.typeutils.SerializerTestBase; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.types.bitmap.Bitmap; + +/** A test for the {@link BitmapSerializer}. */ +class BitmapSerializerTest extends SerializerTestBase { + + @Override + protected TypeSerializer createSerializer() { + return BitmapSerializer.INSTANCE; + } + + @Override + protected int getLength() { + return -1; + } + + @Override + protected Class getTypeClass() { + return Bitmap.class; + } + + @Override + protected Bitmap[] getTestData() { + Bitmap bm1 = Bitmap.empty(); + + Bitmap bm2 = Bitmap.empty(); + bm2.add(1L, 100L); + + Bitmap bm3 = Bitmap.empty(); + bm3.add(10000); + bm3.add(0xFFFF + 1); + + Bitmap bm4 = Bitmap.empty(); + bm4.addN(new int[] {1, Integer.MAX_VALUE, Integer.MIN_VALUE}, 0, 3); + + Bitmap bm5 = Bitmap.empty(); + bm5.add(0L, Integer.MAX_VALUE); + + return new Bitmap[] {bm1, bm2, bm3, bm4, bm5}; + } +} diff --git a/flink-core/src/test/java/org/apache/flink/api/common/typeutils/base/BitmapSerializerUpgradeTest.java b/flink-core/src/test/java/org/apache/flink/api/common/typeutils/base/BitmapSerializerUpgradeTest.java new file mode 100644 index 0000000000000..605124776240e --- /dev/null +++ b/flink-core/src/test/java/org/apache/flink/api/common/typeutils/base/BitmapSerializerUpgradeTest.java @@ -0,0 +1,108 @@ +/* + * 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.flink.api.common.typeutils.base; + +import org.apache.flink.FlinkVersion; +import org.apache.flink.api.common.typeutils.ClassRelocator; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerConditions; +import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility; +import org.apache.flink.api.common.typeutils.TypeSerializerUpgradeTestBase; +import org.apache.flink.api.common.typeutils.base.BitmapSerializer.BitmapSerializerSnapshot; +import org.apache.flink.test.util.MigrationTest; +import org.apache.flink.types.bitmap.Bitmap; + +import org.assertj.core.api.Condition; +import org.junit.jupiter.api.Disabled; + +import java.util.ArrayList; +import java.util.Collection; + +/** + * A {@link TypeSerializerUpgradeTestBase} for {@link BitmapSerializerSnapshot}. The test is + * disabled because bitmap is introduced in Flink 2.3. We should restore the test when there is a + * Flink 2.4 which should test compatibility with Flink 2.3. + */ +@Disabled +class BitmapSerializerUpgradeTest extends TypeSerializerUpgradeTestBase { + + private static final String SPEC_NAME = "bitmap-serializer"; + + @Override + public Collection> createTestSpecifications(FlinkVersion currentVersion) + throws Exception { + ArrayList> testSpecifications = new ArrayList<>(); + testSpecifications.add( + new TestSpecification<>( + SPEC_NAME, + currentVersion, + BitmapSerializerSetup.class, + BitmapSerializerVerifier.class)); + + return testSpecifications; + } + + @Override + public Collection getMigrationVersions() { + return FlinkVersion.rangeOf( + FlinkVersion.v2_3, MigrationTest.getMostRecentlyPublishedVersion()); + } + + // ---------------------------------------------------------------------------------------------- + // Specification for "bitmap-serializer" + // ---------------------------------------------------------------------------------------------- + + /** This class is only public to work with {@link ClassRelocator}. */ + public static final class BitmapSerializerSetup implements PreUpgradeSetup { + @Override + public TypeSerializer createPriorSerializer() { + return BitmapSerializer.INSTANCE; + } + + @Override + public Bitmap createTestData() { + Bitmap bitmap = Bitmap.empty(); + bitmap.add(1); + bitmap.add(Integer.MAX_VALUE); + return bitmap; + } + } + + /** This class is only public to work with {@link ClassRelocator}. */ + public static final class BitmapSerializerVerifier implements UpgradeVerifier { + @Override + public TypeSerializer createUpgradedSerializer() { + return BitmapSerializer.INSTANCE; + } + + @Override + public Condition testDataCondition() { + Bitmap bitmap = Bitmap.empty(); + bitmap.add(1); + bitmap.add(Integer.MAX_VALUE); + return new Condition<>(bitmap::equals, "value is " + bitmap); + } + + @Override + public Condition> schemaCompatibilityCondition( + FlinkVersion version) { + return TypeSerializerConditions.isCompatibleAsIs(); + } + } +} diff --git a/flink-core/src/test/java/org/apache/flink/api/java/typeutils/TypeExtractorTest.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/TypeExtractorTest.java index 6293a69cd01c0..4f99c5f2fd38a 100644 --- a/flink-core/src/test/java/org/apache/flink/api/java/typeutils/TypeExtractorTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/TypeExtractorTest.java @@ -33,6 +33,7 @@ import org.apache.flink.api.common.serialization.SerializerConfig; import org.apache.flink.api.common.typeinfo.BasicArrayTypeInfo; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; +import org.apache.flink.api.common.typeinfo.BitmapTypeInfo; import org.apache.flink.api.common.typeinfo.PrimitiveArrayTypeInfo; import org.apache.flink.api.common.typeinfo.SqlTimeTypeInfo; import org.apache.flink.api.common.typeinfo.TypeHint; @@ -54,6 +55,8 @@ import org.apache.flink.types.Row; import org.apache.flink.types.StringValue; import org.apache.flink.types.Value; +import org.apache.flink.types.bitmap.Bitmap; +import org.apache.flink.types.bitmap.RoaringBitmapData; import org.apache.flink.types.variant.BinaryVariant; import org.apache.flink.types.variant.Variant; import org.apache.flink.util.Collector; @@ -2115,6 +2118,13 @@ void testVariantType() { .isEqualTo(VariantTypeInfo.INSTANCE); } + @Test + void testBitmapType() { + assertThat(TypeExtractor.createTypeInfo(Bitmap.class)).isEqualTo(BitmapTypeInfo.INSTANCE); + assertThat(TypeExtractor.createTypeInfo(RoaringBitmapData.class)) + .isEqualTo(BitmapTypeInfo.INSTANCE); + } + public static class MapperWithMultiDimGenericArray implements MapFunction[][][]> { private static final long serialVersionUID = 1L; diff --git a/flink-core/src/test/java/org/apache/flink/types/RoaringBitmapDataTest.java b/flink-core/src/test/java/org/apache/flink/types/RoaringBitmapDataTest.java new file mode 100644 index 0000000000000..cce27ddb26dcc --- /dev/null +++ b/flink-core/src/test/java/org/apache/flink/types/RoaringBitmapDataTest.java @@ -0,0 +1,231 @@ +/* + * 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.flink.types; + +import org.apache.flink.types.bitmap.Bitmap; +import org.apache.flink.types.bitmap.RoaringBitmapData; + +import org.junit.jupiter.api.Test; +import org.roaringbitmap.RoaringBitmap; + +import java.util.ArrayList; +import java.util.List; +import java.util.Objects; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +/** Test for {@link RoaringBitmapData}. */ +class RoaringBitmapDataTest { + + @Test + void testBitmapInterfaceStaticMethods() { + assertThat(Bitmap.from(null)).isNull(); + assertThat(Bitmap.fromBytes(null)).isNull(); + assertThat(Bitmap.fromArray(null)).isNull(); + } + + @Test + void testStaticConstructors() { + // empty + assertThat(RoaringBitmapData.empty().isEmpty()).isTrue(); + + // from + assertThat(RoaringBitmapData.from(RoaringBitmapData.fromArray(new int[] {1, 2})).toArray()) + .containsExactly(1, 2); + + // fromBytes + assertThat( + RoaringBitmapData.fromBytes( + RoaringBitmapData.fromArray(new int[] {1, 2}).toBytes()) + .toArray()) + .containsExactly(1, 2); + assertThatThrownBy(() -> RoaringBitmapData.fromBytes(new byte[0])) + .isInstanceOf(DeserializationException.class) + .hasMessage("Failed to deserialize bitmap from bytes."); + assertThatThrownBy(() -> RoaringBitmapData.fromBytes("invalid".getBytes())) + .isInstanceOf(DeserializationException.class) + .hasMessage("Failed to deserialize bitmap from bytes."); + + // fromArray + assertThat(RoaringBitmapData.fromArray(new int[0]).toArray()).containsExactly(); + assertThat(RoaringBitmapData.fromArray(new int[] {1, 2}).toArray()).containsExactly(1, 2); + + // wrap + assertThatThrownBy(() -> RoaringBitmapData.wrap(null)) + .isInstanceOf(NullPointerException.class); + assertThat(RoaringBitmapData.wrap(RoaringBitmap.bitmapOf(1, 2)).toArray()) + .containsExactly(1, 2); + } + + @Test + void testInstanceMethods() { + RoaringBitmapData rb32 = RoaringBitmapData.empty(); + assertThat(rb32.isEmpty()).isTrue(); + assertThat(Objects.equals(rb32, new RoaringBitmap())).isFalse(); + + rb32.add(1); + assertThat(rb32.toArray()).containsExactly(1); + + rb32.add(1, 3); + assertThat(rb32.toArray()).containsExactly(1, 2); + assertThat(rb32.getCardinality()).isEqualTo(2); + + rb32.addN(new int[] {1, 2, 3, 4, 5}, 2, 2); + assertThat(rb32.toArray()).containsExactly(1, 2, 3, 4); + + rb32.clear(); + assertThat(rb32.isEmpty()).isTrue(); + + rb32.add(10); + rb32.add(11); + rb32.add(12); + assertThat(rb32.contains(0)).isFalse(); + assertThat(rb32.toArray()).containsExactly(10, 11, 12); + + List list = new ArrayList<>(); + rb32.forEach(list::add); + assertThat(list).containsExactly(10, 11, 12); + + rb32.remove(12); + assertThat(rb32.toArray()).containsExactly(10, 11); + assertThat(rb32.getLongCardinality()).isEqualTo(2); + + rb32.remove(10); + rb32.remove(12); + assertThat(rb32.toArray()).containsExactly(11); + } + + @Test + void testIllegalMethodCalls() { + RoaringBitmapData rb32 = RoaringBitmapData.empty(); + + // add + assertThatThrownBy(() -> rb32.add(-1L, 5L)).isInstanceOf(IllegalArgumentException.class); + assertThatThrownBy(() -> rb32.add(5L, 0xFFFFFFFFL + 5)) + .isInstanceOf(IllegalArgumentException.class); + + // addN + assertThatThrownBy(() -> rb32.addN(null, 0, 1)).isInstanceOf(NullPointerException.class); + assertThatThrownBy(() -> rb32.addN(new int[] {1, 2}, -1, 1)) + .isInstanceOf(IllegalArgumentException.class); + assertThatThrownBy(() -> rb32.addN(new int[] {1, 2}, 0, -1)) + .isInstanceOf(IllegalArgumentException.class); + assertThatThrownBy(() -> rb32.addN(new int[] {1, 2}, 0, 3)) + .isInstanceOf(IllegalArgumentException.class); + + // toArray + rb32.add(0L, 0x80000000L); + // NegativeArraySizeException + assertThatThrownBy(rb32::toArray).isInstanceOf(RuntimeException.class); + rb32.clear(); + rb32.add(0L, 0xFFFFFFFFL + 1); + // ArrayIndexOutOfBoundsException + assertThatThrownBy(rb32::toArray).isInstanceOf(RuntimeException.class); + } + + @Test + void testLogicalOperations() { + RoaringBitmapData rb32 = RoaringBitmapData.empty(); + rb32.add(1, 5); + assertThat(rb32.toArray()).containsExactly(1, 2, 3, 4); + + // and + rb32.and(null); + assertThat(rb32.toArray()).containsExactly(1, 2, 3, 4); + rb32.and(RoaringBitmapData.fromArray(new int[] {1, 2, 6, 10})); + assertThat(rb32.toArray()).containsExactly(1, 2); + + // or + rb32.or(null); + assertThat(rb32.toArray()).containsExactly(1, 2); + rb32.or(RoaringBitmapData.fromArray(new int[] {1, 6, 10})); + assertThat(rb32.toArray()).containsExactly(1, 2, 6, 10); + + // xor + rb32.xor(null); + assertThat(rb32.toArray()).containsExactly(1, 2, 6, 10); + rb32.xor(RoaringBitmapData.fromArray(new int[] {1, 2, 3, 4, 5})); + assertThat(rb32.toArray()).containsExactly(3, 4, 5, 6, 10); + + // andNot + rb32.andNot(null); + assertThat(rb32.toArray()).containsExactly(3, 4, 5, 6, 10); + rb32.andNot(RoaringBitmapData.fromArray(new int[] {1, 2, 3, 4, 5})); + assertThat(rb32.toArray()).containsExactly(6, 10); + } + + @Test + void testLargeCardinality() { + RoaringBitmapData rb32 = RoaringBitmapData.empty(); + rb32.add(1, Integer.MAX_VALUE); + assertThat(rb32.getCardinality()).isEqualTo(Integer.MAX_VALUE - 1); + assertThat(rb32.getLongCardinality()).isEqualTo(Integer.MAX_VALUE - 1); + + rb32.add(Integer.MAX_VALUE); + assertThat(rb32.getCardinality()).isEqualTo(Integer.MAX_VALUE); + assertThat(rb32.getLongCardinality()).isEqualTo(Integer.MAX_VALUE); + + rb32.add(-1); + rb32.add(Integer.MIN_VALUE); + assertThat(rb32.getCardinality()).isEqualTo((int) ((long) Integer.MAX_VALUE + 2)); + assertThat(rb32.getLongCardinality()).isEqualTo((long) Integer.MAX_VALUE + 2); + } + + @Test + void testOutputFormat() { + RoaringBitmapData rb32 = RoaringBitmapData.empty(); + assertThat(rb32.toArray()).containsExactly(); + assertThat(rb32.toString()).isEqualTo("{}"); + + rb32.add(0L, 4L); + assertThat(rb32.toArray()).containsExactly(0, 1, 2, 3); + assertThat(rb32.toString()).isEqualTo("{0,1,2,3}"); + + rb32.add(-1); + assertThat(rb32.toArray()).containsExactly(0, 1, 2, 3, -1); + assertThat(rb32.toString()).isEqualTo(String.format("{0,1,2,3,%s}", 0xFFFFFFFFL)); + + rb32.add(Integer.MIN_VALUE); + assertThat(rb32.toArray()).containsExactly(0, 1, 2, 3, Integer.MIN_VALUE, -1); + assertThat(rb32.toString()) + .isEqualTo(String.format("{0,1,2,3,%s,%s}", 0x80000000L, 0xFFFFFFFFL)); + + rb32.add(0L, Integer.MAX_VALUE); + rb32.add(Integer.MAX_VALUE); + assertThatThrownBy(rb32::toArray).isInstanceOf(RuntimeException.class); + String str = rb32.toString(); + assertThat(str.substring(str.length() - 5)).isEqualTo(",...}"); + } + + @Test + void testHashCode() { + RoaringBitmap rbm1 = new RoaringBitmap(); + RoaringBitmap rbm2 = new RoaringBitmap(); + RoaringBitmapData rb32 = RoaringBitmapData.empty(); + for (int i = 0; i < 100; i++) { + rbm1.add(i); + rbm2.add(i); + rb32.add(i); + } + rbm2.runOptimize(); + assertThat(rbm1.hashCode()).isNotEqualTo(rbm2.hashCode()); + assertThat(rb32.hashCode()).isEqualTo(rbm2.hashCode()); + } +} diff --git a/flink-dist/src/main/resources/META-INF/NOTICE b/flink-dist/src/main/resources/META-INF/NOTICE index 4913adfe9e3a6..ffeb7c67ee3db 100644 --- a/flink-dist/src/main/resources/META-INF/NOTICE +++ b/flink-dist/src/main/resources/META-INF/NOTICE @@ -22,6 +22,7 @@ This project bundles the following dependencies under the Apache Software Licens - org.xerial.snappy:snappy-java:1.1.10.7 - tools.profiler:async-profiler:2.9 - org.snakeyaml:snakeyaml-engine:2.6 +- org.roaringbitmap:RoaringBitmap:1.3.0 This project bundles the following dependencies under the BSD license. See bundled license files for details. diff --git a/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/rest/serde/LogicalTypeJsonDeserializer.java b/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/rest/serde/LogicalTypeJsonDeserializer.java index 4a86f68309d1a..b154c81e815a2 100644 --- a/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/rest/serde/LogicalTypeJsonDeserializer.java +++ b/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/rest/serde/LogicalTypeJsonDeserializer.java @@ -24,6 +24,7 @@ import org.apache.flink.table.types.logical.ArrayType; import org.apache.flink.table.types.logical.BigIntType; import org.apache.flink.table.types.logical.BinaryType; +import org.apache.flink.table.types.logical.BitmapType; import org.apache.flink.table.types.logical.BooleanType; import org.apache.flink.table.types.logical.CharType; import org.apache.flink.table.types.logical.DateType; @@ -162,6 +163,8 @@ private LogicalType deserializeInternal(JsonNode logicalTypeNode) { return deserializeRaw(logicalTypeNode).copy(isNullable); case VARIANT: return new VariantType(isNullable); + case BITMAP: + return new BitmapType(isNullable); default: throw new UnsupportedOperationException( String.format( diff --git a/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/rest/serde/LogicalTypeJsonSerializer.java b/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/rest/serde/LogicalTypeJsonSerializer.java index 9a42c78734a86..c7e9b3b2eb958 100644 --- a/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/rest/serde/LogicalTypeJsonSerializer.java +++ b/flink-table/flink-sql-gateway/src/main/java/org/apache/flink/table/gateway/rest/serde/LogicalTypeJsonSerializer.java @@ -129,6 +129,7 @@ private void serializeInternal(LogicalType logicalType, JsonGenerator jsonGenera case DOUBLE: case DATE: case VARIANT: + case BITMAP: break; case CHAR: jsonGenerator.writeNumberField( diff --git a/flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/rest/serde/LogicalTypeJsonSerDeTest.java b/flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/rest/serde/LogicalTypeJsonSerDeTest.java index 457af18618ac0..b71276fc41cc5 100644 --- a/flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/rest/serde/LogicalTypeJsonSerDeTest.java +++ b/flink-table/flink-sql-gateway/src/test/java/org/apache/flink/table/gateway/rest/serde/LogicalTypeJsonSerDeTest.java @@ -26,6 +26,7 @@ import org.apache.flink.table.types.logical.ArrayType; import org.apache.flink.table.types.logical.BigIntType; import org.apache.flink.table.types.logical.BinaryType; +import org.apache.flink.table.types.logical.BitmapType; import org.apache.flink.table.types.logical.BooleanType; import org.apache.flink.table.types.logical.CharType; import org.apache.flink.table.types.logical.DateType; @@ -195,6 +196,7 @@ private static List generateTestData() { new MultisetType(VarCharType.ofEmptyLiteral()), new MultisetType(BinaryType.ofEmptyLiteral()), new MultisetType(VarBinaryType.ofEmptyLiteral()), + new BitmapType(), RowType.of(new BigIntType(), new IntType(false), new VarCharType(200)), RowType.of( new LogicalType[] { diff --git a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/table/calcite/ExtendedRelTypeFactory.java b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/table/calcite/ExtendedRelTypeFactory.java index dba4dd597891b..57b9bc615ec93 100644 --- a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/table/calcite/ExtendedRelTypeFactory.java +++ b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/table/calcite/ExtendedRelTypeFactory.java @@ -43,8 +43,5 @@ RelDataType createStructuredType( String className, List typeList, List fieldNameList); /** Creates a BITMAP type. */ - default RelDataType createBitmapType() { - // The actual implementation will be provided by FlinkTypeFactory as part of FLINK-39185. - throw new UnsupportedOperationException("BITMAP type is not supported yet."); - } + RelDataType createBitmapType(); } diff --git a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/FailingSqlTestStep.java b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/FailingSqlTestStep.java index 7a9e94a94f492..da04ac889d1a1 100644 --- a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/FailingSqlTestStep.java +++ b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/FailingSqlTestStep.java @@ -42,8 +42,10 @@ public final class FailingSqlTestStep implements TestStep { FailingSqlTestStep( String sql, Class expectedException, String expectedErrorMessage) { Preconditions.checkArgument( + // UnsupportedOperationException is a special case in GenerateUtils#generateCompare expectedException == ValidationException.class - || expectedException == TableRuntimeException.class, + || expectedException == TableRuntimeException.class + || expectedException == UnsupportedOperationException.class, "Usually a SQL query should fail with either validation or runtime exception. " + "Otherwise this might require an update to the exception design."); this.sql = sql; diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/api/DataTypes.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/api/DataTypes.java index d9b514cd158b0..af1e718d90e04 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/api/DataTypes.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/api/DataTypes.java @@ -34,6 +34,7 @@ import org.apache.flink.table.types.logical.ArrayType; import org.apache.flink.table.types.logical.BigIntType; import org.apache.flink.table.types.logical.BinaryType; +import org.apache.flink.table.types.logical.BitmapType; import org.apache.flink.table.types.logical.BooleanType; import org.apache.flink.table.types.logical.CharType; import org.apache.flink.table.types.logical.DateType; @@ -1063,6 +1064,17 @@ public static DataType VARIANT() { return new AtomicDataType(new VariantType()); } + /** + * Data type of bitmap data. + * + *

The type supports storing 32-bit integers in a compressed form. + * + * @see BitmapType + */ + public static DataType BITMAP() { + return new AtomicDataType(new BitmapType()); + } + // -------------------------------------------------------------------------------------------- // Helper functions // -------------------------------------------------------------------------------------------- diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/ArrayData.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/ArrayData.java index ce39d0ef9294e..755bb300e5353 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/ArrayData.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/ArrayData.java @@ -22,6 +22,7 @@ import org.apache.flink.table.types.logical.ArrayType; import org.apache.flink.table.types.logical.DistinctType; import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.types.bitmap.Bitmap; import org.apache.flink.types.variant.Variant; import javax.annotation.Nullable; @@ -116,6 +117,11 @@ public interface ArrayData { */ RowData getRow(int pos, int numFields); + /** Returns the bitmap value at the given position. */ + default Bitmap getBitmap(int pos) { + throw new UnsupportedOperationException("Bitmap is not supported yet."); + } + // ------------------------------------------------------------------------------------------ // Conversion Utilities // ------------------------------------------------------------------------------------------ @@ -215,6 +221,9 @@ static ElementGetter createElementGetter(LogicalType elementType) { case VARIANT: elementGetter = ArrayData::getVariant; break; + case BITMAP: + elementGetter = ArrayData::getBitmap; + break; case NULL: case SYMBOL: case UNRESOLVED: diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/GenericArrayData.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/GenericArrayData.java index 90f62963c4a1c..f609c2f22f131 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/GenericArrayData.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/GenericArrayData.java @@ -20,6 +20,7 @@ import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.table.types.logical.ArrayType; +import org.apache.flink.types.bitmap.Bitmap; import org.apache.flink.types.variant.Variant; import org.apache.commons.lang3.ArrayUtils; @@ -259,6 +260,11 @@ public MapData getMap(int pos) { return (MapData) getObject(pos); } + @Override + public Bitmap getBitmap(int pos) { + return (Bitmap) getObject(pos); + } + private Object getObject(int pos) { return ((Object[]) array)[pos]; } diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/GenericRowData.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/GenericRowData.java index cea9c0a5e148e..b234edef62d05 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/GenericRowData.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/GenericRowData.java @@ -22,6 +22,7 @@ import org.apache.flink.table.types.logical.RowType; import org.apache.flink.table.types.logical.StructuredType; import org.apache.flink.types.RowKind; +import org.apache.flink.types.bitmap.Bitmap; import org.apache.flink.types.variant.Variant; import org.apache.flink.util.StringUtils; @@ -211,6 +212,11 @@ public Variant getVariant(int pos) { return (Variant) this.fields[pos]; } + @Override + public Bitmap getBitmap(int pos) { + return (Bitmap) this.fields[pos]; + } + @Override public boolean equals(Object o) { if (this == o) { diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/RowData.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/RowData.java index a1e16e69169cb..8a54ba52cc6fd 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/RowData.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/RowData.java @@ -25,6 +25,7 @@ import org.apache.flink.table.types.logical.RowType; import org.apache.flink.table.types.logical.StructuredType; import org.apache.flink.types.RowKind; +import org.apache.flink.types.bitmap.Bitmap; import org.apache.flink.types.variant.Variant; import javax.annotation.Nullable; @@ -205,6 +206,11 @@ public interface RowData { /** Returns the variant value at the given position. */ Variant getVariant(int pos); + /** Returns the bitmap value at the given position. */ + default Bitmap getBitmap(int pos) { + throw new UnsupportedOperationException("Bitmap is not supported yet."); + } + // ------------------------------------------------------------------------------------------ // Access Utilities // ------------------------------------------------------------------------------------------ @@ -287,6 +293,9 @@ static FieldGetter createFieldGetter(LogicalType fieldType, int fieldPos) { case VARIANT: fieldGetter = row -> row.getVariant(fieldPos); break; + case BITMAP: + fieldGetter = row -> row.getBitmap(fieldPos); + break; case NULL: case SYMBOL: case UNRESOLVED: diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/binary/BinaryArrayData.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/binary/BinaryArrayData.java index 4d238593b5696..10a8b3e6ef71c 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/binary/BinaryArrayData.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/binary/BinaryArrayData.java @@ -31,6 +31,7 @@ import org.apache.flink.table.types.logical.DistinctType; import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.table.types.logical.utils.LogicalTypeUtils; +import org.apache.flink.types.bitmap.Bitmap; import org.apache.flink.types.variant.Variant; import java.lang.reflect.Array; @@ -93,6 +94,7 @@ public static int calculateFixLengthPartSize(LogicalType type) { case STRUCTURED_TYPE: case RAW: case VARIANT: + case BITMAP: // long and double are 8 bytes; // otherwise it stores the length and offset of the variable-length part for types // such as is string, map, etc. @@ -287,6 +289,14 @@ public RowData getRow(int pos, int numFields) { return BinarySegmentUtils.readRowData(segments, numFields, offset, offsetAndSize); } + @Override + public Bitmap getBitmap(int pos) { + assertIndexIsValid(pos); + int fieldOffset = getElementOffset(pos, 8); + final long offsetAndSize = BinarySegmentUtils.getLong(segments, fieldOffset); + return BinarySegmentUtils.readBitmap(segments, offset, offsetAndSize); + } + @Override public boolean getBoolean(int pos) { assertIndexIsValid(pos); diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/binary/BinaryRowData.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/binary/BinaryRowData.java index 50c7b6a5f78a3..45bcf7cd3f112 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/binary/BinaryRowData.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/binary/BinaryRowData.java @@ -33,6 +33,7 @@ import org.apache.flink.table.types.logical.LogicalTypeRoot; import org.apache.flink.table.types.logical.TimestampType; import org.apache.flink.types.RowKind; +import org.apache.flink.types.bitmap.Bitmap; import org.apache.flink.types.variant.BinaryVariant; import java.nio.ByteOrder; @@ -395,6 +396,12 @@ public BinaryVariant getVariant(int pos) { return BinarySegmentUtils.readVariant(segments, offset, getLong(pos)); } + @Override + public Bitmap getBitmap(int pos) { + assertIndexIsValid(pos); + return BinarySegmentUtils.readBitmap(segments, offset, getLong(pos)); + } + /** The bit is 1 when the field is null. Default is 0. */ @Override public boolean anyNull() { diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/binary/BinarySegmentUtils.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/binary/BinarySegmentUtils.java index 59c29a7234d53..653fc559df9bc 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/binary/BinarySegmentUtils.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/binary/BinarySegmentUtils.java @@ -29,6 +29,7 @@ import org.apache.flink.table.data.RowData; import org.apache.flink.table.data.StringData; import org.apache.flink.table.data.TimestampData; +import org.apache.flink.types.bitmap.Bitmap; import org.apache.flink.types.variant.BinaryVariant; import java.io.IOException; @@ -1190,4 +1191,12 @@ public static BinaryVariant readVariant( return new BinaryVariant(value, meta); } + + /** Gets an instance of {@link Bitmap} from underlying {@link MemorySegment}. */ + public static Bitmap readBitmap(MemorySegment[] segments, int baseOffset, long offsetAndSize) { + final int size = ((int) offsetAndSize); + int offset = (int) (offsetAndSize >> 32); + byte[] bytes = copyToBytes(segments, offset + baseOffset, size); + return Bitmap.fromBytes(bytes); + } } diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/binary/NestedRowData.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/binary/NestedRowData.java index f2d6ed8e4068f..27085b487ae50 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/binary/NestedRowData.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/binary/NestedRowData.java @@ -27,6 +27,7 @@ import org.apache.flink.table.data.StringData; import org.apache.flink.table.data.TimestampData; import org.apache.flink.types.RowKind; +import org.apache.flink.types.bitmap.Bitmap; import org.apache.flink.types.variant.Variant; import static org.apache.flink.table.data.binary.BinaryRowData.calculateBitSetWidthInBytes; @@ -305,6 +306,12 @@ public Variant getVariant(int pos) { return BinarySegmentUtils.readVariant(segments, offset, getLong(pos)); } + @Override + public Bitmap getBitmap(int pos) { + assertIndexIsValid(pos); + return BinarySegmentUtils.readBitmap(segments, offset, getLong(pos)); + } + @Override public ArrayData getArray(int pos) { assertIndexIsValid(pos); diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/ColumnarArrayData.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/ColumnarArrayData.java index 7a546531703b0..a3b8e7dd56daf 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/ColumnarArrayData.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/ColumnarArrayData.java @@ -41,6 +41,7 @@ import org.apache.flink.table.data.columnar.vector.RowColumnVector; import org.apache.flink.table.data.columnar.vector.ShortColumnVector; import org.apache.flink.table.data.columnar.vector.TimestampColumnVector; +import org.apache.flink.types.bitmap.Bitmap; import org.apache.flink.types.variant.Variant; import java.util.Arrays; @@ -161,6 +162,11 @@ public RowData getRow(int pos, int numFields) { return ((RowColumnVector) data).getRow(offset + pos); } + @Override + public Bitmap getBitmap(int pos) { + throw new UnsupportedOperationException("Bitmap is not supported yet."); + } + @Override public void setBoolean(int pos, boolean value) { throw new UnsupportedOperationException("Not support the operation!"); diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/ColumnarRowData.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/ColumnarRowData.java index 8a4d1e86d9869..bd3dce1edf6ba 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/ColumnarRowData.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/columnar/ColumnarRowData.java @@ -30,6 +30,7 @@ import org.apache.flink.table.data.columnar.vector.BytesColumnVector.Bytes; import org.apache.flink.table.data.columnar.vector.VectorizedColumnBatch; import org.apache.flink.types.RowKind; +import org.apache.flink.types.bitmap.Bitmap; import org.apache.flink.types.variant.Variant; /** @@ -161,6 +162,11 @@ public Variant getVariant(int pos) { return vectorizedColumnBatch.getVariant(rowId, pos); } + @Override + public Bitmap getBitmap(int pos) { + throw new UnsupportedOperationException("Bitmap is not supported yet."); + } + @Override public ArrayData getArray(int pos) { return vectorizedColumnBatch.getArray(rowId, pos); diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/utils/JoinedRowData.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/utils/JoinedRowData.java index 258bde2386175..b9eff05a6c388 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/utils/JoinedRowData.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/utils/JoinedRowData.java @@ -27,6 +27,7 @@ import org.apache.flink.table.data.StringData; import org.apache.flink.table.data.TimestampData; import org.apache.flink.types.RowKind; +import org.apache.flink.types.bitmap.Bitmap; import org.apache.flink.types.variant.Variant; import javax.annotation.Nullable; @@ -259,6 +260,15 @@ public Variant getVariant(int pos) { } } + @Override + public Bitmap getBitmap(int pos) { + if (pos < row1.getArity()) { + return row1.getBitmap(pos); + } else { + return row2.getBitmap(pos - row1.getArity()); + } + } + @Override public boolean equals(Object o) { if (this == o) { diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/utils/ProjectedRowData.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/utils/ProjectedRowData.java index c09da8e49ac39..7f29ca5e61323 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/utils/ProjectedRowData.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/utils/ProjectedRowData.java @@ -29,6 +29,7 @@ import org.apache.flink.table.data.TimestampData; import org.apache.flink.table.types.DataType; import org.apache.flink.types.RowKind; +import org.apache.flink.types.bitmap.Bitmap; import org.apache.flink.types.variant.Variant; import java.util.Arrays; @@ -172,6 +173,11 @@ public Variant getVariant(int pos) { return row.getVariant(indexMapping[pos]); } + @Override + public Bitmap getBitmap(int pos) { + return row.getBitmap(indexMapping[pos]); + } + @Override public boolean equals(Object o) { throw new UnsupportedOperationException("Projected row data cannot be compared"); diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/BitmapType.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/BitmapType.java new file mode 100644 index 0000000000000..834e595ed0c80 --- /dev/null +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/BitmapType.java @@ -0,0 +1,85 @@ +/* + * 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.flink.table.types.logical; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.types.bitmap.Bitmap; +import org.apache.flink.types.bitmap.RoaringBitmapData; + +import java.util.Collections; +import java.util.List; +import java.util.Set; + +/** + * Data type of bitmap data. + * + *

This type supports storing 32-bit integers in a compressed form. This can be useful for + * efficiently representing and querying large sets of integers. + * + *

The serializable string representation of this type is {@code BITMAP}. + */ +@PublicEvolving +public final class BitmapType extends LogicalType { + + private static final Set INPUT_OUTPUT_CONVERSION = + conversionSet(Bitmap.class.getName(), RoaringBitmapData.class.getName()); + + public BitmapType(boolean isNullable) { + super(isNullable, LogicalTypeRoot.BITMAP); + } + + public BitmapType() { + this(true); + } + + @Override + public LogicalType copy(boolean isNullable) { + return new BitmapType(isNullable); + } + + @Override + public String asSerializableString() { + return withNullability("BITMAP"); + } + + @Override + public boolean supportsInputConversion(Class clazz) { + return INPUT_OUTPUT_CONVERSION.contains(clazz.getName()); + } + + @Override + public boolean supportsOutputConversion(Class clazz) { + return INPUT_OUTPUT_CONVERSION.contains(clazz.getName()); + } + + @Override + public Class getDefaultConversion() { + return Bitmap.class; + } + + @Override + public List getChildren() { + return Collections.emptyList(); + } + + @Override + public R accept(LogicalTypeVisitor visitor) { + return visitor.visit(this); + } +} diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/LogicalTypeRoot.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/LogicalTypeRoot.java index a4f57ce0f099d..6c823add433bf 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/LogicalTypeRoot.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/LogicalTypeRoot.java @@ -143,7 +143,9 @@ public enum LogicalTypeRoot { DESCRIPTOR(LogicalTypeFamily.DESCRIPTOR), - VARIANT(LogicalTypeFamily.EXTENSION); + VARIANT(LogicalTypeFamily.EXTENSION), + + BITMAP(LogicalTypeFamily.EXTENSION); private final Set families; diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/LogicalTypeVisitor.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/LogicalTypeVisitor.java index 4be1a986ba0fb..6a0e5614466d1 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/LogicalTypeVisitor.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/LogicalTypeVisitor.java @@ -97,5 +97,9 @@ default R visit(VariantType variantType) { return visit((LogicalType) variantType); } + default R visit(BitmapType bitmapType) { + return visit((LogicalType) bitmapType); + } + R visit(LogicalType other); } diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/utils/LogicalTypeCasts.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/utils/LogicalTypeCasts.java index 97cd40b8e6c87..469801013a5a0 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/utils/LogicalTypeCasts.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/utils/LogicalTypeCasts.java @@ -55,6 +55,7 @@ import static org.apache.flink.table.types.logical.LogicalTypeFamily.TIMESTAMP; import static org.apache.flink.table.types.logical.LogicalTypeRoot.BIGINT; import static org.apache.flink.table.types.logical.LogicalTypeRoot.BINARY; +import static org.apache.flink.table.types.logical.LogicalTypeRoot.BITMAP; import static org.apache.flink.table.types.logical.LogicalTypeRoot.BOOLEAN; import static org.apache.flink.table.types.logical.LogicalTypeRoot.CHAR; import static org.apache.flink.table.types.logical.LogicalTypeRoot.DATE; @@ -184,7 +185,7 @@ public final class LogicalTypeCasts { castTo(CHAR) .implicitFrom(CHAR) .explicitFromFamily(PREDEFINED, CONSTRUCTED) - .explicitFrom(RAW, NULL, STRUCTURED_TYPE) + .explicitFrom(RAW, NULL, STRUCTURED_TYPE, BITMAP) .injectiveFrom(WHEN_LENGTH_FITS, CHAR) .injectiveFrom(WHEN_MAX_CHAR_LENGTH_FITS, STRING_INJECTIVE_SOURCES) .build(); @@ -192,7 +193,7 @@ public final class LogicalTypeCasts { castTo(VARCHAR) .implicitFromFamily(CHARACTER_STRING) .explicitFromFamily(PREDEFINED, CONSTRUCTED) - .explicitFrom(RAW, NULL, STRUCTURED_TYPE) + .explicitFrom(RAW, NULL, STRUCTURED_TYPE, BITMAP) .injectiveFrom(WHEN_LENGTH_FITS, CHAR, VARCHAR) .injectiveFrom(WHEN_MAX_CHAR_LENGTH_FITS, STRING_INJECTIVE_SOURCES) .build(); @@ -204,7 +205,7 @@ public final class LogicalTypeCasts { castTo(BINARY) .implicitFrom(BINARY) .explicitFromFamily(CHARACTER_STRING) - .explicitFrom(VARBINARY, RAW) + .explicitFrom(VARBINARY, RAW, BITMAP) .injectiveFrom(WHEN_LENGTH_FITS, BINARY) .injectiveFrom(WHEN_BINARY_LENGTH_FITS_UTF8, CHAR, VARCHAR) .build(); @@ -212,7 +213,7 @@ public final class LogicalTypeCasts { castTo(VARBINARY) .implicitFromFamily(BINARY_STRING) .explicitFromFamily(CHARACTER_STRING) - .explicitFrom(BINARY, RAW) + .explicitFrom(BINARY, RAW, BITMAP) .injectiveFrom(WHEN_LENGTH_FITS, BINARY, VARBINARY) .injectiveFrom(WHEN_BINARY_LENGTH_FITS_UTF8, CHAR, VARCHAR) .build(); diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/utils/LogicalTypeDefaultVisitor.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/utils/LogicalTypeDefaultVisitor.java index 20879b14e1d2d..056ec5953ccdc 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/utils/LogicalTypeDefaultVisitor.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/utils/LogicalTypeDefaultVisitor.java @@ -22,6 +22,7 @@ import org.apache.flink.table.types.logical.ArrayType; import org.apache.flink.table.types.logical.BigIntType; import org.apache.flink.table.types.logical.BinaryType; +import org.apache.flink.table.types.logical.BitmapType; import org.apache.flink.table.types.logical.BooleanType; import org.apache.flink.table.types.logical.CharType; import org.apache.flink.table.types.logical.DateType; @@ -197,6 +198,11 @@ public R visit(SymbolType symbolType) { return defaultMethod(symbolType); } + @Override + public R visit(BitmapType bitmapType) { + return defaultMethod(bitmapType); + } + @Override public R visit(LogicalType other) { return defaultMethod(other); diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/utils/LogicalTypeParser.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/utils/LogicalTypeParser.java index 205697107dbd3..3711ac76f4fd2 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/utils/LogicalTypeParser.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/utils/LogicalTypeParser.java @@ -26,6 +26,7 @@ import org.apache.flink.table.types.logical.ArrayType; import org.apache.flink.table.types.logical.BigIntType; import org.apache.flink.table.types.logical.BinaryType; +import org.apache.flink.table.types.logical.BitmapType; import org.apache.flink.table.types.logical.BooleanType; import org.apache.flink.table.types.logical.CharType; import org.apache.flink.table.types.logical.DateType; @@ -334,7 +335,8 @@ private enum Keyword { NOT, DESCRIPTOR, STRUCTURED, - VARIANT + VARIANT, + BITMAP } private static final Set KEYWORDS = @@ -582,6 +584,8 @@ private LogicalType parseTypeByKeyword() { return new DescriptorType(); case VARIANT: return new VariantType(); + case BITMAP: + return new BitmapType(); default: throw parsingError("Unsupported type: " + token().value); } diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/utils/LogicalTypeUtils.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/utils/LogicalTypeUtils.java index 5c5e53ed70331..98629b15fc177 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/utils/LogicalTypeUtils.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/utils/LogicalTypeUtils.java @@ -35,6 +35,7 @@ import org.apache.flink.table.types.logical.TimestampType; import org.apache.flink.table.types.logical.ZonedTimestampType; import org.apache.flink.types.ColumnList; +import org.apache.flink.types.bitmap.Bitmap; import org.apache.flink.types.variant.Variant; import org.apache.flink.util.Preconditions; @@ -112,6 +113,8 @@ public static Class toInternalConversionClass(LogicalType type) { return ColumnList.class; case VARIANT: return Variant.class; + case BITMAP: + return Bitmap.class; case SYMBOL: case UNRESOLVED: default: diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/utils/ClassDataTypeConverter.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/utils/ClassDataTypeConverter.java index 8c35f7d2076dc..efb21532d2e48 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/utils/ClassDataTypeConverter.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/utils/ClassDataTypeConverter.java @@ -28,6 +28,8 @@ import org.apache.flink.table.types.logical.SymbolType; import org.apache.flink.types.ColumnList; import org.apache.flink.types.Row; +import org.apache.flink.types.bitmap.Bitmap; +import org.apache.flink.types.bitmap.RoaringBitmapData; import org.apache.flink.types.variant.BinaryVariant; import org.apache.flink.types.variant.Variant; @@ -80,6 +82,8 @@ public final class ClassDataTypeConverter { addDefaultDataType(ColumnList.class, DataTypes.DESCRIPTOR()); addDefaultDataType(BinaryVariant.class, DataTypes.VARIANT()); addDefaultDataType(Variant.class, DataTypes.VARIANT()); + addDefaultDataType(Bitmap.class, DataTypes.BITMAP()); + addDefaultDataType(RoaringBitmapData.class, DataTypes.BITMAP()); } private static void addDefaultDataType(Class clazz, DataType rootType) { diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/utils/LogicalTypeDataTypeConverter.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/utils/LogicalTypeDataTypeConverter.java index 0eeb508eff8f2..9aa83f3cbee1d 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/utils/LogicalTypeDataTypeConverter.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/utils/LogicalTypeDataTypeConverter.java @@ -28,6 +28,7 @@ import org.apache.flink.table.types.logical.ArrayType; import org.apache.flink.table.types.logical.BigIntType; import org.apache.flink.table.types.logical.BinaryType; +import org.apache.flink.table.types.logical.BitmapType; import org.apache.flink.table.types.logical.BooleanType; import org.apache.flink.table.types.logical.CharType; import org.apache.flink.table.types.logical.DateType; @@ -259,6 +260,11 @@ public DataType visit(DescriptorType descriptorType) { return new AtomicDataType(descriptorType); } + @Override + public DataType visit(BitmapType bitmapType) { + return new AtomicDataType(bitmapType); + } + @Override public DataType visit(LogicalType other) { if (other.is(LogicalTypeRoot.UNRESOLVED)) { diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/utils/TypeInfoDataTypeConverter.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/utils/TypeInfoDataTypeConverter.java index 12369fcfba4f2..7d8163aba531e 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/utils/TypeInfoDataTypeConverter.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/utils/TypeInfoDataTypeConverter.java @@ -43,6 +43,7 @@ import org.apache.flink.table.types.logical.RowType; import org.apache.flink.table.types.logical.StructuredType; import org.apache.flink.types.Row; +import org.apache.flink.types.bitmap.Bitmap; import org.apache.flink.types.variant.Variant; import javax.annotation.Nullable; @@ -149,6 +150,7 @@ public final class TypeInfoDataTypeConverter { DataTypes.ARRAY(DataTypes.DOUBLE().notNull().bridgedTo(double.class)) .bridgedTo(double[].class)); conversionMap.put(Types.VARIANT, DataTypes.VARIANT().bridgedTo(Variant.class)); + conversionMap.put(Types.BITMAP, DataTypes.BITMAP().bridgedTo(Bitmap.class)); } /** Converts the given {@link TypeInformation} into {@link DataType}. */ diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/utils/ValueDataTypeConverter.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/utils/ValueDataTypeConverter.java index 55057c2da85e2..d27b87efae47d 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/utils/ValueDataTypeConverter.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/utils/ValueDataTypeConverter.java @@ -25,6 +25,7 @@ import org.apache.flink.table.types.logical.BinaryType; import org.apache.flink.table.types.logical.CharType; import org.apache.flink.table.types.logical.LogicalTypeFamily; +import org.apache.flink.types.bitmap.RoaringBitmapData; import org.apache.flink.types.variant.Variant; import java.math.BigDecimal; @@ -93,6 +94,8 @@ else if (value instanceof byte[]) { .map(dt -> dt.notNull().bridgedTo(value.getClass())); } else if (value instanceof Variant) { convertedDataType = DataTypes.VARIANT(); + } else if (value instanceof RoaringBitmapData) { + convertedDataType = DataTypes.BITMAP(); } final Optional resultType; diff --git a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/ClassDataTypeConverterTest.java b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/ClassDataTypeConverterTest.java index eba5424b5703b..0f52a927c0003 100644 --- a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/ClassDataTypeConverterTest.java +++ b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/ClassDataTypeConverterTest.java @@ -23,6 +23,8 @@ import org.apache.flink.table.types.logical.SymbolType; import org.apache.flink.table.types.utils.ClassDataTypeConverter; import org.apache.flink.types.Row; +import org.apache.flink.types.bitmap.Bitmap; +import org.apache.flink.types.bitmap.RoaringBitmapData; import org.apache.flink.types.variant.BinaryVariant; import org.apache.flink.types.variant.Variant; @@ -94,7 +96,9 @@ private static Stream testData() { new AtomicDataType(new SymbolType<>()).bridgedTo(TimeIntervalUnit.class)), of(Row.class, null), of(Variant.class, DataTypes.VARIANT()), - of(BinaryVariant.class, DataTypes.VARIANT().bridgedTo(BinaryVariant.class))); + of(BinaryVariant.class, DataTypes.VARIANT().bridgedTo(BinaryVariant.class)), + of(Bitmap.class, DataTypes.BITMAP().bridgedTo(Bitmap.class)), + of(RoaringBitmapData.class, DataTypes.BITMAP().bridgedTo(RoaringBitmapData.class))); } @ParameterizedTest(name = "[{index}] class: {0} type: {1}") diff --git a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/DataTypesTest.java b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/DataTypesTest.java index f6ff7214b059f..8c274ed719137 100644 --- a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/DataTypesTest.java +++ b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/DataTypesTest.java @@ -24,6 +24,7 @@ import org.apache.flink.table.types.logical.ArrayType; import org.apache.flink.table.types.logical.BigIntType; import org.apache.flink.table.types.logical.BinaryType; +import org.apache.flink.table.types.logical.BitmapType; import org.apache.flink.table.types.logical.BooleanType; import org.apache.flink.table.types.logical.CharType; import org.apache.flink.table.types.logical.DateType; @@ -52,6 +53,8 @@ import org.apache.flink.table.types.utils.DataTypeFactoryMock; import org.apache.flink.table.types.utils.LogicalTypeDataTypeConverter; import org.apache.flink.types.Row; +import org.apache.flink.types.bitmap.Bitmap; +import org.apache.flink.types.bitmap.RoaringBitmapData; import org.apache.flink.types.variant.Variant; import org.junit.jupiter.params.ParameterizedTest; @@ -70,6 +73,7 @@ import static org.apache.flink.table.api.DataTypes.ARRAY; import static org.apache.flink.table.api.DataTypes.BIGINT; import static org.apache.flink.table.api.DataTypes.BINARY; +import static org.apache.flink.table.api.DataTypes.BITMAP; import static org.apache.flink.table.api.DataTypes.BOOLEAN; import static org.apache.flink.table.api.DataTypes.BYTES; import static org.apache.flink.table.api.DataTypes.CHAR; @@ -224,6 +228,9 @@ private static Stream testData() { TestSpec.forDataType(VARIANT()) .expectLogicalType(new VariantType()) .expectConversionClass(Variant.class), + TestSpec.forDataType(BITMAP()) + .expectLogicalType(new BitmapType()) + .expectConversionClass(Bitmap.class), TestSpec.forUnresolvedDataType(RAW(Types.VOID)) .expectUnresolvedString("[RAW('java.lang.Void', '?')]") .lookupReturns(dummyRaw(Void.class)) @@ -296,7 +303,14 @@ private static Stream testData() { .expectResolvedDataType(dummyRaw(DayOfWeek.class)), TestSpec.forUnresolvedDataType(DataTypes.of(Variant.class)) .expectUnresolvedString("['org.apache.flink.types.variant.Variant']") - .expectResolvedDataType(VARIANT())); + .expectResolvedDataType(VARIANT()), + TestSpec.forUnresolvedDataType(DataTypes.of(Bitmap.class)) + .expectUnresolvedString("['org.apache.flink.types.bitmap.Bitmap']") + .expectResolvedDataType(BITMAP()), + TestSpec.forUnresolvedDataType(DataTypes.of(RoaringBitmapData.class)) + .expectUnresolvedString( + "['org.apache.flink.types.bitmap.RoaringBitmapData']") + .expectResolvedDataType(BITMAP().bridgedTo(RoaringBitmapData.class))); } @ParameterizedTest(name = "{index}: {0}") diff --git a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/LogicalTypeCastsTest.java b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/LogicalTypeCastsTest.java index e6ac53f79bdf7..b630f5859c14a 100644 --- a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/LogicalTypeCastsTest.java +++ b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/LogicalTypeCastsTest.java @@ -25,6 +25,7 @@ import org.apache.flink.table.types.logical.ArrayType; import org.apache.flink.table.types.logical.BigIntType; import org.apache.flink.table.types.logical.BinaryType; +import org.apache.flink.table.types.logical.BitmapType; import org.apache.flink.table.types.logical.BooleanType; import org.apache.flink.table.types.logical.CharType; import org.apache.flink.table.types.logical.DateType; @@ -262,7 +263,18 @@ private static Stream testData() { new RawType<>(Integer.class, IntSerializer.INSTANCE), VarCharType.STRING_TYPE, false, - true)); + true), + // bitmap, remove these cases when built-in functions are introduced to build bitmap + Arguments.of(new BitmapType(), new CharType(), false, true), + Arguments.of(new BitmapType(), VarCharType.STRING_TYPE, false, true), + Arguments.of(new BitmapType(), new BinaryType(), false, true), + Arguments.of(new BitmapType(), new VarBinaryType(), false, true), + Arguments.of(new BitmapType(), new ArrayType(new IntType()), false, false), + Arguments.of(new CharType(), new BitmapType(), false, false), + Arguments.of(VarCharType.STRING_TYPE, new BitmapType(), false, false), + Arguments.of(new BinaryType(), new BitmapType(), false, false), + Arguments.of(new VarBinaryType(), new BitmapType(), false, false), + Arguments.of(new ArrayType(new IntType()), new BitmapType(), false, false)); } @ParameterizedTest(name = "{index}: [From: {0}, To: {1}, Implicit: {2}, Explicit: {3}]") diff --git a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/LogicalTypeParserTest.java b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/LogicalTypeParserTest.java index 4a6bd79d06986..258b397c87187 100644 --- a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/LogicalTypeParserTest.java +++ b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/LogicalTypeParserTest.java @@ -29,6 +29,7 @@ import org.apache.flink.table.types.logical.ArrayType; import org.apache.flink.table.types.logical.BigIntType; import org.apache.flink.table.types.logical.BinaryType; +import org.apache.flink.table.types.logical.BitmapType; import org.apache.flink.table.types.logical.BooleanType; import org.apache.flink.table.types.logical.CharType; import org.apache.flink.table.types.logical.DateType; @@ -306,6 +307,8 @@ private static Stream testData() { .expectType(createGenericLegacyType()), TestSpec.forString("VARIANT").expectType(new VariantType()), TestSpec.forString("VARIANT NOT NULL").expectType(new VariantType(false)), + TestSpec.forString("BITMAP").expectType(new BitmapType()), + TestSpec.forString("BITMAP NOT NULL").expectType(new BitmapType(false)), // error message testing diff --git a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/LogicalTypesTest.java b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/LogicalTypesTest.java index 3d87af5156dda..ee40463cf0b92 100644 --- a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/LogicalTypesTest.java +++ b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/LogicalTypesTest.java @@ -31,6 +31,7 @@ import org.apache.flink.table.types.logical.ArrayType; import org.apache.flink.table.types.logical.BigIntType; import org.apache.flink.table.types.logical.BinaryType; +import org.apache.flink.table.types.logical.BitmapType; import org.apache.flink.table.types.logical.BooleanType; import org.apache.flink.table.types.logical.CharType; import org.apache.flink.table.types.logical.DateType; @@ -62,6 +63,8 @@ import org.apache.flink.table.types.logical.YearMonthIntervalType; import org.apache.flink.table.types.logical.ZonedTimestampType; import org.apache.flink.types.Row; +import org.apache.flink.types.bitmap.Bitmap; +import org.apache.flink.types.bitmap.RoaringBitmapData; import org.apache.flink.types.variant.BinaryVariant; import org.apache.flink.types.variant.Variant; @@ -612,6 +615,20 @@ void testVariantType() { .supportsInputConversion(BinaryVariant.class); } + @Test + void testBitmapType() { + assertThat(new BitmapType()) + .isJavaSerializable() + .satisfies( + baseAssertions( + "BITMAP", + "BITMAP", + new Class[] {Bitmap.class, RoaringBitmapData.class}, + new Class[] {Bitmap.class, RoaringBitmapData.class}, + new LogicalType[] {}, + new BitmapType(false))); + } + @Test void testTypeInformationRawType() { final TypeInformationRawType rawType = diff --git a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/TypeInfoDataTypeConverterTest.java b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/TypeInfoDataTypeConverterTest.java index 9d961f47571d3..c22ae68b3f173 100644 --- a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/TypeInfoDataTypeConverterTest.java +++ b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/TypeInfoDataTypeConverterTest.java @@ -110,7 +110,8 @@ private static Stream testData() { TestSpec.forType(Types.ENUM(DayOfWeek.class)) .lookupExpects(DayOfWeek.class) .expectDataType(dummyRaw(DayOfWeek.class)), - TestSpec.forType(Types.VARIANT).expectDataType(DataTypes.VARIANT())); + TestSpec.forType(Types.VARIANT).expectDataType(DataTypes.VARIANT()), + TestSpec.forType(Types.BITMAP).expectDataType(DataTypes.BITMAP())); } @ParameterizedTest(name = "{index}: {0}") diff --git a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/ValueDataTypeConverterTest.java b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/ValueDataTypeConverterTest.java index cb09f1bb04fdc..14c4a7466f5dd 100644 --- a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/ValueDataTypeConverterTest.java +++ b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/ValueDataTypeConverterTest.java @@ -24,6 +24,8 @@ import org.apache.flink.table.types.logical.CharType; import org.apache.flink.table.types.logical.SymbolType; import org.apache.flink.table.types.utils.ValueDataTypeConverter; +import org.apache.flink.types.bitmap.Bitmap; +import org.apache.flink.types.bitmap.RoaringBitmapData; import org.apache.flink.types.variant.BinaryVariant; import org.apache.flink.types.variant.Variant; @@ -117,7 +119,11 @@ private static Stream testData() { of(new BigDecimal[0], null), of( Variant.newBuilder().of("hello"), - DataTypes.VARIANT().bridgedTo(BinaryVariant.class))); + DataTypes.VARIANT().bridgedTo(BinaryVariant.class)), + of(Bitmap.empty(), DataTypes.BITMAP().bridgedTo(RoaringBitmapData.class)), + of( + Bitmap.fromArray(new int[] {1, 2}), + DataTypes.BITMAP().bridgedTo(RoaringBitmapData.class))); } @ParameterizedTest(name = "[{index}] value: {0} type: {1}") diff --git a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/extraction/TypeInferenceExtractorTest.java b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/extraction/TypeInferenceExtractorTest.java index 4d7334fd82266..47b4f823d56b9 100644 --- a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/extraction/TypeInferenceExtractorTest.java +++ b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/types/extraction/TypeInferenceExtractorTest.java @@ -49,6 +49,7 @@ import org.apache.flink.table.types.inference.TypeStrategy; import org.apache.flink.table.types.utils.DataTypeFactoryMock; import org.apache.flink.types.Row; +import org.apache.flink.types.bitmap.Bitmap; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.MethodSource; @@ -841,7 +842,43 @@ private static Stream functionSpecs() { // --- TestSpec.forProcessTableFunction(MultiEvalProcessTableFunction.class) .expectErrorMessage( - "Process table functions require a non-overloaded, non-vararg, and static signature.")); + "Process table functions require a non-overloaded, non-vararg, and static signature."), + TestSpec.forScalarFunction("Bitmap in scalar function", BitmapTypeFunction.class) + .expectStaticArgument( + StaticArgument.scalar("bm", DataTypes.BITMAP(), false)) + .expectStaticArgument( + StaticArgument.scalar( + "array", DataTypes.ARRAY(DataTypes.BITMAP()), false)) + .expectStaticArgument( + StaticArgument.scalar( + "map", + DataTypes.MAP(DataTypes.INT(), DataTypes.BITMAP()), + false)) + .expectStaticArgument( + StaticArgument.scalar( + "row", + DataTypes.ROW(DataTypes.FIELD("a", DataTypes.BITMAP())), + false)) + .expectOutput(TypeStrategies.explicit(DataTypes.BITMAP())), + TestSpec.forAsyncScalarFunction( + "Bitmap in async scalar function", BitmapTypeAsyncFunction.class) + .expectStaticArgument( + StaticArgument.scalar("bm", DataTypes.BITMAP(), false)) + .expectOutput(TypeStrategies.explicit(DataTypes.BITMAP())), + TestSpec.forAggregateFunction( + "Bitmap in aggregate function", BitmapTypeAggFunction.class) + .expectStaticArgument( + StaticArgument.scalar("bitmap", DataTypes.BITMAP(), false)) + .expectAccumulator(TypeStrategies.explicit(DataTypes.BITMAP())) + .expectOutput(TypeStrategies.explicit(DataTypes.BITMAP())), + TestSpec.forScalarFunction( + "Bitmap bridged to custom Bitmap", + InvalidCustomBitmapTypeFunction1.class) + .expectErrorMessage( + "Logical type 'BITMAP' does not support a conversion from or to class 'org.apache.flink.table.types.extraction.TypeInferenceExtractorTest$CustomBitmap'."), + TestSpec.forScalarFunction("Custom Bitmap", InvalidCustomBitmapTypeFunction2.class) + .expectErrorMessage( + "Could not extract a valid type inference for function class 'org.apache.flink.table.types.extraction.TypeInferenceExtractorTest$InvalidCustomBitmapTypeFunction2'.")); } private static Stream procedureSpecs() { @@ -1176,7 +1213,11 @@ private static Stream procedureSpecs() { ArgumentHintOptionalOnPrimitiveParameterConflictProcedure.class) .expectErrorMessage( "Considering all hints, the method should comply with the signature:\n" - + "int[] call(_, java.lang.String, java.lang.Integer)")); + + "int[] call(_, java.lang.String, java.lang.Integer)"), + TestSpec.forProcedure("Bitmap in procedure", BitmapTypeProcedure.class) + .expectStaticArgument( + StaticArgument.scalar("bitmap", DataTypes.BITMAP(), false)) + .expectOutput(TypeStrategies.explicit(DataTypes.BITMAP()))); } @ParameterizedTest(name = "{index}: {0}") @@ -1312,8 +1353,13 @@ static TestSpec forAsyncTableFunction( } static TestSpec forAggregateFunction(Class> function) { + return forAggregateFunction(null, function); + } + + static TestSpec forAggregateFunction( + String description, Class> function) { return new TestSpec( - function.getSimpleName(), + description == null ? function.getSimpleName() : description, () -> TypeInferenceExtractor.forAggregateFunction( new DataTypeFactoryMock(), function)); @@ -2623,4 +2669,112 @@ public int eval(@ArgumentHint(ArgumentTrait.ROW_SEMANTIC_TABLE) Row t) { return 0; } } + + @FunctionHint(output = @DataTypeHint("BITMAP")) + private static class BitmapTypeFunction extends ScalarFunction { + public Bitmap eval( + Bitmap bm, + Bitmap[] array, + Map map, + @DataTypeHint("ROW") Row row) { + return null; + } + } + + private static class BitmapTypeAsyncFunction extends AsyncScalarFunction { + public void eval(CompletableFuture f, Bitmap bm) {} + } + + private static class BitmapTypeAggFunction extends AggregateFunction { + public void accumulate(Bitmap accumulator, Bitmap bitmap) {} + + @Override + public Bitmap createAccumulator() { + return null; + } + + @Override + public Bitmap getValue(Bitmap accumulator) { + return null; + } + } + + private static class BitmapTypeProcedure implements Procedure { + public Bitmap[] call(Object procedureContext, Bitmap bitmap) { + return null; + } + } + + @FunctionHint(input = @DataTypeHint(value = "BITMAP", bridgedTo = CustomBitmap.class)) + private static class InvalidCustomBitmapTypeFunction1 extends ScalarFunction { + public Bitmap eval(Bitmap bitmap) { + return null; + } + } + + private static class InvalidCustomBitmapTypeFunction2 extends ScalarFunction { + public Bitmap eval(CustomBitmap bitmap) { + return null; + } + } + + public static class CustomBitmap implements Bitmap { + + @Override + public void add(int value) {} + + @Override + public void add(long rangeStart, long rangeEnd) {} + + @Override + public void addN(int[] values, int offset, int n) {} + + @Override + public void and(@org.jetbrains.annotations.Nullable Bitmap other) {} + + @Override + public void andNot(@org.jetbrains.annotations.Nullable Bitmap other) {} + + @Override + public void clear() {} + + @Override + public boolean contains(int value) { + return false; + } + + @Override + public int getCardinality() { + return 0; + } + + @Override + public long getLongCardinality() { + return 0; + } + + @Override + public boolean isEmpty() { + return false; + } + + @Override + public void or(@org.jetbrains.annotations.Nullable Bitmap other) {} + + @Override + public void remove(int value) {} + + @Override + public int[] toArray() { + return new int[0]; + } + + @Override + public byte[] toBytes() { + return new byte[0]; + } + + @Override + public void xor(@org.jetbrains.annotations.Nullable Bitmap other) {} + } } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql/fun/SqlCastFunction.java b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql/fun/SqlCastFunction.java index eb67a1cdbaf7a..feca7c52a8f9f 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql/fun/SqlCastFunction.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql/fun/SqlCastFunction.java @@ -17,6 +17,7 @@ package org.apache.calcite.sql.fun; import org.apache.flink.table.planner.calcite.FlinkTypeFactory; +import org.apache.flink.table.planner.plan.schema.BitmapRelDataType; import org.apache.flink.table.types.logical.utils.LogicalTypeCasts; import com.google.common.collect.ImmutableSetMultimap; @@ -260,6 +261,10 @@ private boolean canCastFrom(RelDataType toType, RelDataType fromType) { if (toType.getSqlTypeName() == SqlTypeName.VARIANT) { return false; } + // Cast to BITMAP is not supported at the moment. + if (toType instanceof BitmapRelDataType) { + return false; + } switch (fromTypeName) { case ARRAY: case MAP: diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/BitmapToBinaryCastRule.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/BitmapToBinaryCastRule.java new file mode 100644 index 0000000000000..f363f335bbae0 --- /dev/null +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/BitmapToBinaryCastRule.java @@ -0,0 +1,88 @@ +/* + * 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.flink.table.planner.functions.casting; + +import org.apache.flink.table.planner.codegen.CodeGenUtils; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.LogicalTypeFamily; +import org.apache.flink.table.types.logical.LogicalTypeRoot; +import org.apache.flink.table.types.logical.utils.LogicalTypeChecks; +import org.apache.flink.types.bitmap.Bitmap; + +import static org.apache.flink.table.planner.functions.casting.BinaryToBinaryCastRule.couldPad; +import static org.apache.flink.table.planner.functions.casting.BinaryToBinaryCastRule.couldTrim; +import static org.apache.flink.table.planner.functions.casting.BinaryToBinaryCastRule.trimOrPadByteArray; +import static org.apache.flink.table.planner.functions.casting.CastRuleUtils.arrayLength; +import static org.apache.flink.table.planner.functions.casting.CastRuleUtils.methodCall; + +/** {@link LogicalTypeRoot#BITMAP} to {@link LogicalTypeFamily#BINARY_STRING} cast rule. */ +class BitmapToBinaryCastRule extends AbstractNullAwareCodeGeneratorCastRule { + + static final BitmapToBinaryCastRule INSTANCE = new BitmapToBinaryCastRule(); + + private BitmapToBinaryCastRule() { + super( + CastRulePredicate.builder() + .input(LogicalTypeRoot.BITMAP) + .target(LogicalTypeFamily.BINARY_STRING) + .build()); + } + + @Override + protected String generateCodeBlockInternal( + CodeGeneratorCastRule.Context context, + String inputTerm, + String returnVariable, + LogicalType inputLogicalType, + LogicalType targetLogicalType) { + final int targetLength = LogicalTypeChecks.getLength(targetLogicalType); + final String byteArrayTerm = + CodeGenUtils.newName(context.getCodeGeneratorContext(), "bitmapBytes"); + + if (context.legacyBehaviour() + || !(couldTrim(targetLength) || couldPad(targetLogicalType, targetLength))) { + return new CastRuleUtils.CodeWriter() + .assignStmt(returnVariable, methodCall(inputTerm, "toBytes")) + .toString(); + } else { + return new CastRuleUtils.CodeWriter() + .declStmt(byte[].class, byteArrayTerm, methodCall(inputTerm, "toBytes")) + .ifStmt( + arrayLength(byteArrayTerm) + " <= " + targetLength, + thenWriter -> { + if (couldPad(targetLogicalType, targetLength)) { + trimOrPadByteArray( + returnVariable, + targetLength, + byteArrayTerm, + thenWriter); + } else { + thenWriter.assignStmt(returnVariable, byteArrayTerm); + } + }, + elseWriter -> + trimOrPadByteArray( + returnVariable, + targetLength, + byteArrayTerm, + elseWriter)) + .toString(); + } + } +} diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/BitmapToStringCastRule.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/BitmapToStringCastRule.java new file mode 100644 index 0000000000000..d47efbcd63b8f --- /dev/null +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/BitmapToStringCastRule.java @@ -0,0 +1,49 @@ +/* + * 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.flink.table.planner.functions.casting; + +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.LogicalTypeFamily; +import org.apache.flink.table.types.logical.LogicalTypeRoot; +import org.apache.flink.types.bitmap.Bitmap; + +import static org.apache.flink.table.planner.functions.casting.CastRuleUtils.methodCall; + +/** {@link LogicalTypeRoot#BITMAP} to {@link LogicalTypeFamily#CHARACTER_STRING} cast rule. */ +class BitmapToStringCastRule extends AbstractCharacterFamilyTargetRule { + + static final BitmapToStringCastRule INSTANCE = new BitmapToStringCastRule(); + + private BitmapToStringCastRule() { + super( + CastRulePredicate.builder() + .input(LogicalTypeRoot.BITMAP) + .target(LogicalTypeFamily.CHARACTER_STRING) + .build()); + } + + @Override + public String generateStringExpression( + CodeGeneratorCastRule.Context context, + String inputTerm, + LogicalType inputLogicalType, + LogicalType targetLogicalType) { + return methodCall(inputTerm, "toString"); + } +} diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/CastRuleProvider.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/CastRuleProvider.java index 7b94816696e59..b18b44e126e17 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/CastRuleProvider.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/CastRuleProvider.java @@ -97,6 +97,9 @@ public class CastRuleProvider { .addRule(RowToRowCastRule.INSTANCE) // Variant rules .addRule(VariantToStringCastRule.INSTANCE) + // Bitmap rules + .addRule(BitmapToStringCastRule.INSTANCE) + .addRule(BitmapToBinaryCastRule.INSTANCE) // Special rules .addRule(CharVarCharTrimPadCastRule.INSTANCE) .addRule(NullToStringCastRule.INSTANCE); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/LogicalTypeJsonSerializer.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/LogicalTypeJsonSerializer.java index 5494c04af8b88..2857c4dbaa3bc 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/LogicalTypeJsonSerializer.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/LogicalTypeJsonSerializer.java @@ -543,6 +543,7 @@ protected Boolean defaultMethod(LogicalType logicalType) { case STRUCTURED_TYPE: case NULL: case DESCRIPTOR: + case BITMAP: return true; default: // fall back to generic serialization diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/schema/BitmapRelDataType.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/schema/BitmapRelDataType.java new file mode 100644 index 0000000000000..7222f28d6c8db --- /dev/null +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/schema/BitmapRelDataType.java @@ -0,0 +1,62 @@ +/* + * 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.flink.table.planner.plan.schema; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.types.logical.BitmapType; + +import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.sql.type.AbstractSqlType; +import org.apache.calcite.sql.type.SqlTypeName; + +/** The {@link RelDataType} representation of a {@link BitmapType}. */ +@Internal +public final class BitmapRelDataType extends AbstractSqlType { + + private final BitmapType bitmapType; + + public BitmapRelDataType(BitmapType bitmapType) { + super(SqlTypeName.OTHER, bitmapType.isNullable(), null); + this.bitmapType = bitmapType; + computeDigest(); + } + + public BitmapType getBitmapType() { + return bitmapType; + } + + public BitmapRelDataType createWithNullability(boolean nullable) { + if (nullable == isNullable()) { + return this; + } + return new BitmapRelDataType((BitmapType) bitmapType.copy(nullable)); + } + + @Override + protected void generateTypeString(StringBuilder sb, boolean withDetail) { + sb.append(bitmapType.asSummaryString()); + } + + @Override + protected void computeDigest() { + final StringBuilder sb = new StringBuilder(); + generateTypeString(sb, true); + digest = sb.toString(); + } +} diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/typeutils/LogicalRelDataTypeConverter.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/typeutils/LogicalRelDataTypeConverter.java index 7d0970ffc5a38..12093cfe1e571 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/typeutils/LogicalRelDataTypeConverter.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/typeutils/LogicalRelDataTypeConverter.java @@ -22,12 +22,14 @@ import org.apache.flink.table.api.TableException; import org.apache.flink.table.catalog.DataTypeFactory; import org.apache.flink.table.planner.calcite.FlinkTypeFactory; +import org.apache.flink.table.planner.plan.schema.BitmapRelDataType; import org.apache.flink.table.planner.plan.schema.RawRelDataType; import org.apache.flink.table.planner.plan.schema.StructuredRelDataType; import org.apache.flink.table.planner.plan.schema.TimeIndicatorRelDataType; import org.apache.flink.table.types.logical.ArrayType; import org.apache.flink.table.types.logical.BigIntType; import org.apache.flink.table.types.logical.BinaryType; +import org.apache.flink.table.types.logical.BitmapType; import org.apache.flink.table.types.logical.BooleanType; import org.apache.flink.table.types.logical.CharType; import org.apache.flink.table.types.logical.DateType; @@ -459,6 +461,11 @@ public RelDataType visit(DescriptorType descriptorType) { return relDataTypeFactory.createSqlType(SqlTypeName.COLUMN_LIST); } + @Override + public RelDataType visit(BitmapType bitmapType) { + return new BitmapRelDataType(bitmapType); + } + @Override public RelDataType visit(LogicalType other) { throw new TableException( @@ -587,6 +594,8 @@ private static LogicalType toLogicalTypeNotNull( return ((StructuredRelDataType) relDataType).getStructuredType(); } else if (relDataType instanceof RawRelDataType) { return ((RawRelDataType) relDataType).getRawType(); + } else if (relDataType instanceof BitmapRelDataType) { + return ((BitmapRelDataType) relDataType).getBitmapType(); } // fall through case REAL: diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/calcite/FlinkTypeFactory.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/calcite/FlinkTypeFactory.scala index 44de1c7b1dc34..e8e8883329459 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/calcite/FlinkTypeFactory.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/calcite/FlinkTypeFactory.scala @@ -159,6 +159,9 @@ class FlinkTypeFactory( case LogicalTypeRoot.VARIANT => createSqlType(SqlTypeName.VARIANT) + case LogicalTypeRoot.BITMAP => + new BitmapRelDataType(t.asInstanceOf[BitmapType]) + case _ @t => throw new TableException(s"Type is not supported: $t") } @@ -419,6 +422,10 @@ class FlinkTypeFactory( canonize(relDataType) } + override def createBitmapType(): RelDataType = { + canonize(new BitmapRelDataType(new BitmapType())) + } + override def createSqlType(typeName: SqlTypeName): RelDataType = { if (typeName == DECIMAL) { // if we got here, the precision and scale are not specified, here we @@ -447,6 +454,9 @@ class FlinkTypeFactory( case structured: StructuredRelDataType => structured.createWithNullability(isNullable) + case bitmap: BitmapRelDataType => + bitmap.createWithNullability(isNullable) + case generic: GenericRelDataType => new GenericRelDataType(generic.genericType, isNullable, typeSystem) @@ -682,6 +692,9 @@ object FlinkTypeFactory { case OTHER if relDataType.isInstanceOf[RawRelDataType] => relDataType.asInstanceOf[RawRelDataType].getRawType + case OTHER if relDataType.isInstanceOf[BitmapRelDataType] => + relDataType.asInstanceOf[BitmapRelDataType].getBitmapType + case _ @t => throw new TableException(s"Type is not supported: $t") } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/CodeGenUtils.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/CodeGenUtils.scala index c4fac01871059..811fd1a842087 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/CodeGenUtils.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/CodeGenUtils.scala @@ -43,6 +43,7 @@ import org.apache.flink.table.types.logical.utils.LogicalTypeUtils.toInternalCon import org.apache.flink.table.types.utils.DataTypeUtils.isInternal import org.apache.flink.table.utils.EncodingUtils import org.apache.flink.types.{ColumnList, Row, RowKind} +import org.apache.flink.types.bitmap.Bitmap import org.apache.flink.types.variant.Variant import java.lang.{Boolean => JBoolean, Byte => JByte, Double => JDouble, Float => JFloat, Integer => JInt, Long => JLong, Object => JObject, Short => JShort} @@ -275,6 +276,7 @@ object CodeGenUtils { case RAW => className[BinaryRawValueData[_]] case DESCRIPTOR => className[ColumnList] case VARIANT => className[Variant] + case BITMAP => className[Bitmap] case SYMBOL | UNRESOLVED => throw new IllegalArgumentException("Illegal type: " + t) } @@ -380,6 +382,8 @@ object CodeGenUtils { } val serTerm = ctx.addReusableObject(serializer, "serializer") s"$term.toObject($serTerm).hashCode()" + case BITMAP => + s"$term.hashCode()" case NULL | SYMBOL | UNRESOLVED => throw new IllegalArgumentException("Illegal type: " + t) } @@ -530,6 +534,8 @@ object CodeGenUtils { s"(($BINARY_RAW_VALUE) $rowTerm.getRawValue($indexTerm))" case VARIANT => s"$rowTerm.getVariant($indexTerm)" + case BITMAP => + s"$rowTerm.getBitmap($indexTerm)" case NULL | SYMBOL | UNRESOLVED => throw new IllegalArgumentException("Illegal type: " + t) } @@ -825,6 +831,8 @@ object CodeGenUtils { s"$writerTerm.writeRawValue($indexTerm, $fieldValTerm, $ser)" case VARIANT => s"$writerTerm.writeVariant($indexTerm, $fieldValTerm)" + case BITMAP => + s"$writerTerm.writeBitmap($indexTerm, $fieldValTerm)" case NULL | SYMBOL | UNRESOLVED => throw new IllegalArgumentException("Illegal type: " + t); } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/GenerateUtils.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/GenerateUtils.scala index 9e3f9976f833c..865404ac7e827 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/GenerateUtils.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/GenerateUtils.scala @@ -336,7 +336,7 @@ object GenerateUtils { INTERVAL_YEAR_MONTH | INTERVAL_DAY_TIME => generateNonNullLiteral(literalType, primitiveLiteralForType(literalValue), literalValue) - case ARRAY | MULTISET | MAP | ROW | STRUCTURED_TYPE | NULL | UNRESOLVED => + case ARRAY | MULTISET | MAP | ROW | STRUCTURED_TYPE | NULL | UNRESOLVED | BITMAP => throw new CodeGenException(s"Type not supported: $literalType") } } @@ -633,7 +633,7 @@ object GenerateUtils { case TINYINT | SMALLINT | INTEGER | BIGINT | FLOAT | DOUBLE | DATE | TIME_WITHOUT_TIME_ZONE | INTERVAL_YEAR_MONTH | INTERVAL_DAY_TIME => s"($leftTerm > $rightTerm ? 1 : $leftTerm < $rightTerm ? -1 : 0)" - case TIMESTAMP_WITH_TIME_ZONE | MULTISET | MAP | VARIANT => + case TIMESTAMP_WITH_TIME_ZONE | MULTISET | MAP | VARIANT | BITMAP => throw new UnsupportedOperationException( s"Type($t) is not an orderable data type, " + s"it is not supported as a ORDER_BY/GROUP_BY/JOIN_EQUAL field.") diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/AggregateUtil.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/AggregateUtil.scala index 31215d0b5d29a..78f0243826e75 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/AggregateUtil.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/AggregateUtil.scala @@ -917,7 +917,7 @@ object AggregateUtil extends Enumeration { case CHAR | VARCHAR | BOOLEAN | DECIMAL | TINYINT | SMALLINT | INTEGER | BIGINT | FLOAT | DOUBLE | DATE | TIME_WITHOUT_TIME_ZONE | TIMESTAMP_WITHOUT_TIME_ZONE | TIMESTAMP_WITH_LOCAL_TIME_ZONE | INTERVAL_YEAR_MONTH | INTERVAL_DAY_TIME | ARRAY | - VARIANT => + VARIANT | BITMAP => argTypes(0) case t => throw new TableException( diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/DataTypeJsonSerdeTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/DataTypeJsonSerdeTest.java index a3c5afd8ec819..3c24b472bac93 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/DataTypeJsonSerdeTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/DataTypeJsonSerdeTest.java @@ -58,6 +58,7 @@ private static Stream testDataTypeSerde() { DataTypes.INT().notNull().bridgedTo(int.class), DataTypes.TIMESTAMP_LTZ(3).toInternal(), DataTypes.TIMESTAMP_LTZ(9).bridgedTo(long.class), + DataTypes.BITMAP(), DataTypes.ROW( DataTypes.TIMESTAMP_LTZ(3).toInternal(), DataTypes.TIMESTAMP_LTZ(9).bridgedTo(long.class), diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/LogicalTypeJsonSerdeTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/LogicalTypeJsonSerdeTest.java index 674dd940dccf0..0742134ebd64b 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/LogicalTypeJsonSerdeTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/LogicalTypeJsonSerdeTest.java @@ -34,6 +34,7 @@ import org.apache.flink.table.types.logical.ArrayType; import org.apache.flink.table.types.logical.BigIntType; import org.apache.flink.table.types.logical.BinaryType; +import org.apache.flink.table.types.logical.BitmapType; import org.apache.flink.table.types.logical.BooleanType; import org.apache.flink.table.types.logical.CharType; import org.apache.flink.table.types.logical.DateType; @@ -265,6 +266,7 @@ private static List testLogicalTypeSerde() { new MultisetType(VarCharType.ofEmptyLiteral()), new MultisetType(BinaryType.ofEmptyLiteral()), new MultisetType(VarBinaryType.ofEmptyLiteral()), + new BitmapType(), RowType.of(new BigIntType(), new IntType(false), new VarCharType(200)), RowType.of( new LogicalType[] { diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/RelDataTypeJsonSerdeTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/RelDataTypeJsonSerdeTest.java index 7982558c2e931..5dc11186c26e4 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/RelDataTypeJsonSerdeTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/RelDataTypeJsonSerdeTest.java @@ -22,6 +22,7 @@ import org.apache.flink.table.planner.calcite.FlinkTypeFactory; import org.apache.flink.table.planner.calcite.FlinkTypeSystem; import org.apache.flink.table.planner.typeutils.LogicalRelDataTypeConverterTest.PojoClass; +import org.apache.flink.table.types.logical.BitmapType; import org.apache.flink.table.types.logical.DayTimeIntervalType; import org.apache.flink.table.types.logical.RawType; @@ -243,6 +244,13 @@ public static List testRelDataTypeSerde() { new RawType<>(true, Void.class, VoidSerializer.INSTANCE)), false)); + mutableTypes.add( + FACTORY.createTypeWithNullability( + FACTORY.createFieldTypeFromLogicalType(new BitmapType()), false)); + mutableTypes.add( + FACTORY.createTypeWithNullability( + FACTORY.createFieldTypeFromLogicalType(new BitmapType()), true)); + return mutableTypes; } } diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/BitmapSemanticTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/BitmapSemanticTest.java new file mode 100644 index 0000000000000..a2fb4bb9982b1 --- /dev/null +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/BitmapSemanticTest.java @@ -0,0 +1,286 @@ +/* + * 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.flink.table.planner.plan.nodes.exec.stream; + +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.functions.AggregateFunction; +import org.apache.flink.table.functions.ScalarFunction; +import org.apache.flink.table.functions.TableFunction; +import org.apache.flink.table.planner.plan.nodes.exec.testutils.SemanticTestBase; +import org.apache.flink.table.test.program.SinkTestStep; +import org.apache.flink.table.test.program.SourceTestStep; +import org.apache.flink.table.test.program.TableTestProgram; +import org.apache.flink.types.Row; +import org.apache.flink.types.RowKind; +import org.apache.flink.types.bitmap.Bitmap; + +import java.time.LocalDateTime; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +/** Semantic tests for {@link DataTypes#BITMAP()} type. */ +public class BitmapSemanticTest extends SemanticTestBase { + + @Override + public List programs() { + return List.of( + BUILTIN_AGG, + BUILTIN_AGG_WITH_RETRACTION, + BITMAP_AS_UDF_ARG, + BITMAP_AS_UDAF_ARG, + BITMAP_AS_UDTF_ARG, + BITMAP_AS_GROUP_BY_KEY, + BITMAP_AS_ORDER_BY_KEY, + BITMAP_AS_DISTINCT_KEY); + } + + static final TableTestProgram BUILTIN_AGG_WITH_RETRACTION; + + static final TableTestProgram BUILTIN_AGG; + + static { + Bitmap bm1 = Bitmap.fromArray(new int[] {1, 2}); + Bitmap bm2 = Bitmap.fromArray(new int[] {2, 3}); + + BUILTIN_AGG = + TableTestProgram.of("builtin-agg", "validates builtin agg") + .setupTableSource( + SourceTestStep.newBuilder("t") + .addSchema("bm BITMAP") + .producedValues(Row.of(bm1), Row.of(bm2), Row.of(bm2)) + .build()) + .setupTableSink( + SinkTestStep.newBuilder("sink_t") + .addSchema("fbm BITMAP", "lbm BITMAP", "c BIGINT") + .consumedValues( + Row.ofKind(RowKind.INSERT, bm1, bm1, 1), + Row.ofKind(RowKind.UPDATE_BEFORE, bm1, bm1, 1), + Row.ofKind(RowKind.UPDATE_AFTER, bm1, bm2, 2), + Row.ofKind(RowKind.UPDATE_BEFORE, bm1, bm2, 2), + Row.ofKind(RowKind.UPDATE_AFTER, bm1, bm2, 3)) + .build()) + .runSql( + "INSERT INTO sink_t SELECT FIRST_VALUE(bm), LAST_VALUE(bm), COUNT(bm) FROM t") + .build(); + + BUILTIN_AGG_WITH_RETRACTION = + TableTestProgram.of( + "builtin-agg-with-retraction", + "validates builtin agg with retraction") + .setupTableSource( + SourceTestStep.newBuilder("t") + .addSchema("bm BITMAP") + .addOption("changelog-mode", "I,UB,UA,D") + .producedValues( + Row.ofKind(RowKind.INSERT, bm1), + Row.ofKind(RowKind.INSERT, bm2), + Row.ofKind(RowKind.INSERT, bm2), + Row.ofKind(RowKind.DELETE, bm1)) + .build()) + .setupTableSink( + SinkTestStep.newBuilder("sink_t") + .addSchema("fbm BITMAP", "lbm BITMAP", "c BIGINT") + .consumedValues( + Row.ofKind(RowKind.INSERT, bm1, bm1, 1), + Row.ofKind(RowKind.UPDATE_BEFORE, bm1, bm1, 1), + Row.ofKind(RowKind.UPDATE_AFTER, bm1, bm2, 2), + Row.ofKind(RowKind.UPDATE_BEFORE, bm1, bm2, 2), + Row.ofKind(RowKind.UPDATE_AFTER, bm1, bm2, 3), + Row.ofKind(RowKind.UPDATE_BEFORE, bm1, bm2, 3), + Row.ofKind(RowKind.UPDATE_AFTER, bm2, bm2, 2)) + .build()) + .runSql( + "INSERT INTO sink_t SELECT FIRST_VALUE(bm), LAST_VALUE(bm), COUNT(bm) FROM t") + .build(); + } + + static final TableTestProgram BITMAP_AS_UDF_ARG = + TableTestProgram.of("bitmap-as-udf-arg", "validates bitmap as udf argument") + .setupTemporarySystemFunction("udf", BitmapSemanticTest.MyUdf.class) + .setupTableSource( + SourceTestStep.newBuilder("t") + .addSchema("bm BITMAP") + .producedValues( + Row.of(Bitmap.fromArray(new int[] {1})), + Row.of(Bitmap.fromArray(new int[] {1, 2})), + Row.of(Bitmap.fromArray(new int[] {1, 2, 3})), + new Row(1)) + .build()) + .setupTableSink( + SinkTestStep.newBuilder("sink_t") + .addSchema("c INTEGER") + .consumedValues(Row.of(1), Row.of(2), Row.of(3), new Row(1)) + .build()) + .runSql("INSERT INTO sink_t SELECT udf(bm) FROM t") + .build(); + + static final TableTestProgram BITMAP_AS_UDAF_ARG = + TableTestProgram.of("bitmap-as-udaf-arg", "validates bitmap as udaf argument") + .setupTemporarySystemFunction("udaf", MyUdaf.class) + .setupTableSource( + SourceTestStep.newBuilder("t") + .addSchema("bm BITMAP") + .producedValues( + Row.of(Bitmap.fromArray(new int[] {1})), + Row.of(Bitmap.fromArray(new int[] {1, 2})), + Row.of(Bitmap.fromArray(new int[] {1, 2, 3})), + new Row(1)) + .build()) + .setupTableSink( + SinkTestStep.newBuilder("sink_t") + .addSchema("c BIGINT") + .consumedValues( + Row.ofKind(RowKind.INSERT, 1), + Row.ofKind(RowKind.UPDATE_BEFORE, 1), + Row.ofKind(RowKind.UPDATE_AFTER, 3), + Row.ofKind(RowKind.UPDATE_BEFORE, 3), + Row.ofKind(RowKind.UPDATE_AFTER, 6)) + .build()) + .runSql("INSERT INTO sink_t SELECT udaf(bm) FROM t") + .build(); + + static final TableTestProgram BITMAP_AS_UDTF_ARG = + TableTestProgram.of("bitmap-as-udtf-arg", "validates bitmap as udtf argument") + .setupTemporarySystemFunction("udtf", MyUdtf.class) + .setupTableSource( + SourceTestStep.newBuilder("t") + .addSchema("bm BITMAP") + .producedValues( + Row.of(Bitmap.fromArray(new int[] {0})), + Row.of(Bitmap.fromArray(new int[] {1, 2, 3}))) + .build()) + .setupTableSink( + SinkTestStep.newBuilder("sink_t") + .addSchema("v INT") + .consumedValues(Row.of(0), Row.of(1), Row.of(2), Row.of(3)) + .build()) + .runSql( + "INSERT INTO sink_t SELECT t2.v FROM t, LATERAL TABLE(udtf(bm)) AS t2(v)") + .build(); + + static final TableTestProgram BITMAP_AS_GROUP_BY_KEY = + TableTestProgram.of("bitmap-as-group-by-key", "validates bitmap as group by key") + .setupTableSource( + SourceTestStep.newBuilder("t") + .addSchema("bm BITMAP", "v INTEGER") + .producedValues( + Row.of(Bitmap.fromArray(new int[] {1}), 1), + Row.of(Bitmap.fromArray(new int[] {2}), 2), + Row.of(Bitmap.fromArray(new int[] {1}), 2)) + .build()) + .setupTableSink( + SinkTestStep.newBuilder("sink_t") + .addSchema("bm STRING", "total INTEGER") + .consumedValues( + Row.ofKind(RowKind.INSERT, "{1}", 1), + Row.ofKind(RowKind.INSERT, "{2}", 2), + Row.ofKind(RowKind.UPDATE_BEFORE, "{1}", 1), + Row.ofKind(RowKind.UPDATE_AFTER, "{1}", 3)) + .build()) + .runSql( + "INSERT INTO sink_t SELECT CAST(bm AS STRING), SUM(v) AS total FROM t GROUP BY bm") + .build(); + + static final TableTestProgram BITMAP_AS_ORDER_BY_KEY = + TableTestProgram.of("bitmap-as-order-by-key", "validates bitmap fails as order by key") + .setupTableSource( + SourceTestStep.newBuilder("t") + .addSchema( + "bm BITMAP", + "ts TIMESTAMP(3)", + "WATERMARK FOR ts AS ts - INTERVAL '1' SECOND") + .producedValues(Row.of(Bitmap.empty(), LocalDateTime.now())) + .build()) + .setupTableSink( + SinkTestStep.newBuilder("sink_t") + .addSchema("ts TIMESTAMP(3)") + .consumedValues(new Row[0]) + .build()) + .runFailingSql( + "INSERT INTO sink_t " + + "SELECT FIRST_VALUE(ts) OVER (ORDER BY bm) " + + "FROM TABLE(TUMBLE(TABLE t, DESCRIPTOR(ts), INTERVAL '1' SECOND))", + UnsupportedOperationException.class, + "Type(BITMAP) is not an orderable data type, " + + "it is not supported as a ORDER_BY/GROUP_BY/JOIN_EQUAL field.") + .build(); + + static final TableTestProgram BITMAP_AS_DISTINCT_KEY = + TableTestProgram.of("bitmap-as-distinct-key", "validates bitmap as distinct key") + .setupTableSource( + SourceTestStep.newBuilder("t") + .addSchema("bm BITMAP", "v INTEGER") + .producedValues( + Row.of(Bitmap.fromArray(new int[] {1}), 1), + Row.of(Bitmap.fromArray(new int[] {2}), 2), + Row.of(Bitmap.fromArray(new int[] {2}), 2), + Row.of(Bitmap.fromArray(new int[] {1}), 2)) + .build()) + .setupTableSink( + SinkTestStep.newBuilder("sink_t") + .addSchema("cnt BIGINT", "total INTEGER") + .consumedValues( + Row.ofKind(RowKind.INSERT, 1, 1), + Row.ofKind(RowKind.INSERT, 1, 2), + Row.ofKind(RowKind.UPDATE_BEFORE, 1, 2), + Row.ofKind(RowKind.UPDATE_AFTER, 2, 2)) + .build()) + .runSql( + "INSERT INTO sink_t SELECT COUNT(DISTINCT bm) AS cnt, v FROM t GROUP BY v") + .build(); + + public static class MyUdf extends ScalarFunction { + + public Integer eval(Bitmap bm) { + if (bm == null) { + return null; + } + return bm.getCardinality(); + } + } + + public static class MyUdaf extends AggregateFunction> { + + public Long getValue(List accumulator) { + return accumulator.get(0); + } + + public List createAccumulator() { + return new ArrayList<>(List.of(0L)); + } + + public void accumulate(List accumulator, Bitmap bm) { + if (bm == null) { + return; + } + accumulator.set(0, accumulator.get(0) + bm.getLongCardinality()); + } + } + + public static class MyUdtf extends TableFunction { + + public void eval(Bitmap bm) { + if (bm == null) { + return; + } + Arrays.stream(bm.toArray()).forEach(this::collect); + } + } +} diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/typeutils/LogicalRelDataTypeConverterTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/typeutils/LogicalRelDataTypeConverterTest.java index 983890cea79b2..92f563cc5dc8a 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/typeutils/LogicalRelDataTypeConverterTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/typeutils/LogicalRelDataTypeConverterTest.java @@ -25,6 +25,7 @@ import org.apache.flink.table.types.logical.ArrayType; import org.apache.flink.table.types.logical.BigIntType; import org.apache.flink.table.types.logical.BinaryType; +import org.apache.flink.table.types.logical.BitmapType; import org.apache.flink.table.types.logical.BooleanType; import org.apache.flink.table.types.logical.CharType; import org.apache.flink.table.types.logical.DateType; @@ -150,6 +151,7 @@ private static Stream testConversion() { new MultisetType(VarCharType.ofEmptyLiteral()), new MultisetType(BinaryType.ofEmptyLiteral()), new MultisetType(VarBinaryType.ofEmptyLiteral()), + new BitmapType(), RowType.of(new BigIntType(), new IntType(false), new VarCharType(200)), RowType.of( new LogicalType[] { diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/codegen/ProjectionCodeGeneratorTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/codegen/ProjectionCodeGeneratorTest.scala index 8f1c5235ded6e..b6439ffd23b41 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/codegen/ProjectionCodeGeneratorTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/codegen/ProjectionCodeGeneratorTest.scala @@ -22,7 +22,8 @@ import org.apache.flink.table.data.{DecimalData, GenericRowData, RowData, Timest import org.apache.flink.table.data.binary.BinaryRowData import org.apache.flink.table.data.writer.BinaryRowWriter import org.apache.flink.table.runtime.generated.Projection -import org.apache.flink.table.types.logical.{BigIntType, DecimalType, IntType, RowType, TimestampType} +import org.apache.flink.table.types.logical._ +import org.apache.flink.types.bitmap.Bitmap import org.junit.jupiter.api.{Assertions, Test} import org.junit.jupiter.api.Assertions.assertEquals @@ -40,15 +41,17 @@ class ProjectionCodeGeneratorTest { .generateProjection( new CodeGeneratorContext(new Configuration, classLoader), "name", - RowType.of(new IntType(), new BigIntType()), - RowType.of(new BigIntType(), new IntType()), - Array(1, 0) + RowType.of(new IntType(), new BigIntType(), new BitmapType()), + RowType.of(new BitmapType(), new IntType(), new BigIntType()), + Array(2, 0, 1) ) .newInstance(classLoader) .asInstanceOf[Projection[RowData, BinaryRowData]] - val row: BinaryRowData = projection.apply(GenericRowData.of(ji(5), jl(8))) + val row: BinaryRowData = + projection.apply(GenericRowData.of(ji(5), jl(8), Bitmap.fromArray(Array(1, 2)))) Assertions.assertEquals(5, row.getInt(1)) - Assertions.assertEquals(8, row.getLong(0)) + Assertions.assertEquals(8, row.getLong(2)) + Assertions.assertEquals(Bitmap.fromArray(Array(1, 2)), row.getBitmap(0)) } @Test diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/TableSourceITCase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/TableSourceITCase.scala index 8a30ba0eaf998..d69e659b7f010 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/TableSourceITCase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/TableSourceITCase.scala @@ -23,17 +23,22 @@ import org.apache.flink.table.api.{createTypeInformation, TableException} import org.apache.flink.table.api.bridge.scala._ import org.apache.flink.table.planner.factories.TestValuesTableFactory import org.apache.flink.table.planner.runtime.utils.{StreamingTestBase, TestData, TestingAppendSink, TestingRetractSink} +import org.apache.flink.table.planner.runtime.utils.BatchTestBase.row import org.apache.flink.table.planner.runtime.utils.TestData.data1 import org.apache.flink.table.runtime.functions.scalar.SourceWatermarkFunction import org.apache.flink.table.utils.LegacyRowExtension import org.apache.flink.types.Row +import org.apache.flink.types.bitmap.Bitmap import org.assertj.core.api.Assertions.{assertThat, assertThatThrownBy} +import org.assertj.core.util.Arrays.array import org.junit.jupiter.api.{BeforeEach, Test} import org.junit.jupiter.api.extension.RegisterExtension import java.util.concurrent.atomic.AtomicInteger +import scala.collection.mutable + class TableSourceITCase extends StreamingTestBase { @RegisterExtension private val _: EachCallbackWrapper[LegacyRowExtension] = @@ -276,6 +281,44 @@ class TableSourceITCase extends StreamingTestBase { .isEqualTo(expected.sorted.mkString("\n")) } + @Test + def testBitmapDataType(): Unit = { + val data = new mutable.MutableList[Row] + val empty = Bitmap.empty() + for (i <- 1 to 3) { + val bitmap = Bitmap.fromArray(Array[Int](i, i + 1)) + data += row(i, bitmap, array(empty, bitmap), row(i, bitmap)) + } + val dataId = TestValuesTableFactory.registerData(data) + + tEnv.executeSql( + s""" + |CREATE TABLE T ( + | `a` INT, + | `b` BITMAP, + | `c` ARRAY, + | `d` ROW + |) WITH ( + | 'connector' = 'values', + | 'data-id' = '$dataId' + |) + |""".stripMargin + ) + + val result = tEnv.sqlQuery("SELECT b, c, d.f2, a FROM T").toDataStream + val sink = new TestingAppendSink + result.addSink(sink) + env.execute() + + val expected = Seq( + "{1,2},[{}, {1,2}],{1,2},1", + "{2,3},[{}, {2,3}],{2,3},2", + "{3,4},[{}, {3,4}],{3,4},3" + ) + assertThat(sink.getAppendResults.sorted.mkString("\n")) + .isEqualTo(expected.sorted.mkString("\n")) + } + @Test def testSimpleMetadataAccess(): Unit = { val result = tEnv diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/BoxedWrapperRowData.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/BoxedWrapperRowData.java index 366ffa4dc7872..d5e7f361d7fa9 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/BoxedWrapperRowData.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/BoxedWrapperRowData.java @@ -27,6 +27,7 @@ import org.apache.flink.types.LongValue; import org.apache.flink.types.RowKind; import org.apache.flink.types.ShortValue; +import org.apache.flink.types.bitmap.Bitmap; import org.apache.flink.types.variant.Variant; import org.apache.flink.util.StringUtils; @@ -148,6 +149,11 @@ public Variant getVariant(int pos) { return (Variant) this.fields[pos]; } + @Override + public Bitmap getBitmap(int pos) { + return (Bitmap) this.fields[pos]; + } + @Override public void setNullAt(int pos) { this.fields[pos] = null; diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/UpdatableRowData.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/UpdatableRowData.java index b8a62c18d2b5d..bdc3d279ce394 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/UpdatableRowData.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/UpdatableRowData.java @@ -20,6 +20,7 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.table.data.binary.TypedSetters; import org.apache.flink.types.RowKind; +import org.apache.flink.types.bitmap.Bitmap; import org.apache.flink.types.variant.Variant; import java.util.BitSet; @@ -136,6 +137,11 @@ public Variant getVariant(int pos) { return updated.get(pos) ? (Variant) fields[pos] : row.getVariant(pos); } + @Override + public Bitmap getBitmap(int pos) { + return updated.get(pos) ? (Bitmap) fields[pos] : row.getBitmap(pos); + } + @Override public ArrayData getArray(int pos) { return updated.get(pos) ? (ArrayData) fields[pos] : row.getArray(pos); diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/conversion/BitmapBitmapConverter.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/conversion/BitmapBitmapConverter.java new file mode 100644 index 0000000000000..45ff51354f21b --- /dev/null +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/conversion/BitmapBitmapConverter.java @@ -0,0 +1,47 @@ +/* + * 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.flink.table.data.conversion; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.types.bitmap.Bitmap; +import org.apache.flink.types.bitmap.RoaringBitmapData; + +/** + * Converter for {@link org.apache.flink.table.types.logical.BitmapType BitmapType} of unknown + * {@link Bitmap} external type. + */ +@Internal +public class BitmapBitmapConverter implements DataStructureConverter { + + private static final long serialVersionUID = 1L; + + @Override + public Bitmap toInternal(Bitmap external) { + if (!(external instanceof RoaringBitmapData)) { + throw new UnsupportedOperationException( + "Unsupported bitmap type: " + external.getClass().getSimpleName() + "."); + } + return external; + } + + @Override + public Bitmap toExternal(Bitmap internal) { + return internal; + } +} diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/conversion/DataStructureConverters.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/conversion/DataStructureConverters.java index 1b2717059b2bc..f01a1aca779da 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/conversion/DataStructureConverters.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/conversion/DataStructureConverters.java @@ -31,6 +31,8 @@ import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.table.types.logical.LogicalTypeRoot; import org.apache.flink.types.Row; +import org.apache.flink.types.bitmap.Bitmap; +import org.apache.flink.types.bitmap.RoaringBitmapData; import org.apache.flink.types.variant.Variant; import java.math.BigDecimal; @@ -196,6 +198,8 @@ public final class DataStructureConverters { putConverter(LogicalTypeRoot.RAW, byte[].class, RawByteArrayConverter::create); putConverter(LogicalTypeRoot.RAW, RawValueData.class, identity()); putConverter(LogicalTypeRoot.VARIANT, Variant.class, identity()); + putConverter(LogicalTypeRoot.BITMAP, Bitmap.class, constructor(BitmapBitmapConverter::new)); + putConverter(LogicalTypeRoot.BITMAP, RoaringBitmapData.class, identity()); } /** Returns a converter for the given {@link DataType}. */ diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/util/DataFormatConverters.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/util/DataFormatConverters.java index 0ec0a3331b570..756ca6293c867 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/util/DataFormatConverters.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/util/DataFormatConverters.java @@ -67,6 +67,8 @@ import org.apache.flink.table.types.utils.TypeConversions; import org.apache.flink.table.utils.DateTimeUtils; import org.apache.flink.types.Row; +import org.apache.flink.types.bitmap.Bitmap; +import org.apache.flink.types.bitmap.RoaringBitmapData; import org.apache.flink.types.variant.Variant; import org.apache.commons.lang3.ArrayUtils; @@ -156,6 +158,9 @@ public class DataFormatConverters { DataTypes.INTERVAL(DataTypes.SECOND(3)).bridgedTo(long.class), LongConverter.INSTANCE); + t2C.put(DataTypes.BITMAP().bridgedTo(Bitmap.class), BitmapConverter.INSTANCE); + t2C.put(DataTypes.BITMAP().bridgedTo(RoaringBitmapData.class), BitmapConverter.INSTANCE); + TYPE_TO_CONVERTER = Collections.unmodifiableMap(t2C); } @@ -745,6 +750,29 @@ Variant toExternalImpl(RowData row, int column) { } } + public static final class BitmapConverter extends IdentityConverter { + + private static final long serialVersionUID = 1L; + + public static final BitmapConverter INSTANCE = new BitmapConverter(); + + private BitmapConverter() {} + + @Override + Bitmap toInternalImpl(Bitmap value) { + if (!(value instanceof RoaringBitmapData)) { + throw new UnsupportedOperationException( + "Unsupported bitmap type: " + value.getClass().getSimpleName() + "."); + } + return value; + } + + @Override + Bitmap toExternalImpl(RowData row, int column) { + return row.getBitmap(column); + } + } + /** Converter for LocalDate. */ public static final class LocalDateConverter extends DataFormatConverter { diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/writer/AbstractBinaryWriter.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/writer/AbstractBinaryWriter.java index 00adec9e3e2b9..73f817e64815e 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/writer/AbstractBinaryWriter.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/writer/AbstractBinaryWriter.java @@ -40,6 +40,7 @@ import org.apache.flink.table.runtime.typeutils.MapDataSerializer; import org.apache.flink.table.runtime.typeutils.RawValueDataSerializer; import org.apache.flink.table.runtime.typeutils.RowDataSerializer; +import org.apache.flink.types.bitmap.Bitmap; import org.apache.flink.types.variant.BinaryVariant; import org.apache.flink.types.variant.Variant; @@ -134,6 +135,12 @@ public void writeVariant(int pos, Variant variant) { writeBytesToVarLenPart(pos, buffer.array(), length); } + @Override + public void writeBitmap(int pos, Bitmap bitmap) { + byte[] bytes = bitmap.toBytes(); + writeBytesToVarLenPart(pos, bytes, bytes.length); + } + private DataOutputViewStreamWrapper getOutputView() { if (outputView == null) { outputView = new DataOutputViewStreamWrapper(new BinaryRowWriterOutputView()); diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/writer/BinaryArrayWriter.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/writer/BinaryArrayWriter.java index c6ac99c912d67..857c2ecd42466 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/writer/BinaryArrayWriter.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/writer/BinaryArrayWriter.java @@ -246,6 +246,7 @@ public static NullSetter createNullSetter(LogicalType elementType) { case STRUCTURED_TYPE: case RAW: case VARIANT: + case BITMAP: return BinaryArrayWriter::setNullLong; case BOOLEAN: return BinaryArrayWriter::setNullBoolean; diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/writer/BinaryWriter.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/writer/BinaryWriter.java index 49096458521fa..4db8977ccd2f0 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/writer/BinaryWriter.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/data/writer/BinaryWriter.java @@ -36,6 +36,7 @@ import org.apache.flink.table.types.logical.LocalZonedTimestampType; import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.table.types.logical.TimestampType; +import org.apache.flink.types.bitmap.Bitmap; import org.apache.flink.types.variant.Variant; import java.io.Serializable; @@ -88,6 +89,8 @@ public interface BinaryWriter { void writeVariant(int pos, Variant variant); + void writeBitmap(int pos, Bitmap bitmap); + /** Finally, complete write to set real size to binary. */ void complete(); @@ -168,6 +171,9 @@ static void write( case VARIANT: writer.writeVariant(pos, (Variant) o); break; + case BITMAP: + writer.writeBitmap(pos, (Bitmap) o); + break; default: throw new UnsupportedOperationException("Not support type: " + type); } @@ -245,6 +251,8 @@ static ValueSetter createValueSetter(LogicalType elementType) { (RawValueDataSerializer) rawSerializer); case VARIANT: return (writer, pos, value) -> writer.writeVariant(pos, (Variant) value); + case BITMAP: + return (writer, pos, value) -> writer.writeBitmap(pos, (Bitmap) value); case NULL: case SYMBOL: case UNRESOLVED: diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/process/RepeatedRowData.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/process/RepeatedRowData.java index fd7b2b737a6ea..111ddd6662b19 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/process/RepeatedRowData.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/process/RepeatedRowData.java @@ -26,6 +26,7 @@ import org.apache.flink.table.data.StringData; import org.apache.flink.table.data.TimestampData; import org.apache.flink.types.RowKind; +import org.apache.flink.types.bitmap.Bitmap; import org.apache.flink.types.variant.Variant; /** A row that repeats the columns of a given row by the given count. */ @@ -148,4 +149,9 @@ public RowData getRow(int pos, int numFields) { public Variant getVariant(int pos) { return row.getVariant(pos / count); } + + @Override + public Bitmap getBitmap(int pos) { + return row.getBitmap(pos / count); + } } diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/typeutils/InternalSerializers.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/typeutils/InternalSerializers.java index 8b725ed577d1f..371dcd136e1d5 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/typeutils/InternalSerializers.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/typeutils/InternalSerializers.java @@ -21,6 +21,7 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.serialization.SerializerConfigImpl; import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.base.BitmapSerializer; import org.apache.flink.api.common.typeutils.base.BooleanSerializer; import org.apache.flink.api.common.typeutils.base.ByteSerializer; import org.apache.flink.api.common.typeutils.base.DoubleSerializer; @@ -126,6 +127,8 @@ private static TypeSerializer createInternal(LogicalType type) { + "Any other use is unsupported."); case VARIANT: return VariantSerializer.INSTANCE; + case BITMAP: + return BitmapSerializer.INSTANCE; case NULL: case SYMBOL: case UNRESOLVED: diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/typeutils/TypeCheckUtils.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/typeutils/TypeCheckUtils.java index c203b283dc7f3..e8fe0c0ad4e5d 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/typeutils/TypeCheckUtils.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/typeutils/TypeCheckUtils.java @@ -27,6 +27,7 @@ import static org.apache.flink.table.types.logical.LogicalTypeRoot.ARRAY; import static org.apache.flink.table.types.logical.LogicalTypeRoot.BIGINT; +import static org.apache.flink.table.types.logical.LogicalTypeRoot.BITMAP; import static org.apache.flink.table.types.logical.LogicalTypeRoot.BOOLEAN; import static org.apache.flink.table.types.logical.LogicalTypeRoot.DECIMAL; import static org.apache.flink.table.types.logical.LogicalTypeRoot.INTEGER; @@ -141,6 +142,10 @@ private static boolean isVariantType(LogicalType type) { return type.getTypeRoot() == VARIANT; } + private static boolean isBitmapType(LogicalType type) { + return type.getTypeRoot() == BITMAP; + } + public static boolean isComparable(LogicalType type) { return !isRaw(type) && !isMap(type) @@ -148,7 +153,8 @@ public static boolean isComparable(LogicalType type) { && !isRow(type) && !isArray(type) && !isStructuredType(type) - && !isVariantType(type); + && !isVariantType(type) + && !isBitmapType(type); } public static boolean isMutable(LogicalType type) { @@ -162,6 +168,7 @@ public static boolean isMutable(LogicalType type) { case ROW: case STRUCTURED_TYPE: case RAW: + case BITMAP: return true; case TIMESTAMP_WITH_TIME_ZONE: throw new UnsupportedOperationException("Unsupported type: " + type); diff --git a/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/data/BinaryArrayDataTest.java b/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/data/BinaryArrayDataTest.java index 5895a52c99e2a..cb132cea5b51c 100644 --- a/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/data/BinaryArrayDataTest.java +++ b/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/data/BinaryArrayDataTest.java @@ -35,6 +35,7 @@ import org.apache.flink.table.types.logical.IntType; import org.apache.flink.table.types.logical.RowType; import org.apache.flink.table.types.logical.VarCharType; +import org.apache.flink.types.bitmap.Bitmap; import org.apache.flink.types.variant.Variant; import org.junit.jupiter.api.Test; @@ -330,6 +331,28 @@ void testArrayTypes() { assertThat(newArray.getVariant(1)).isEqualTo(variant); } + { + // test bitmap + Bitmap bm1 = Bitmap.empty(); + Bitmap bm2 = Bitmap.fromArray(new int[] {1}); + + BinaryArrayData array = new BinaryArrayData(); + BinaryArrayWriter writer = new BinaryArrayWriter(array, 3, 8); + writer.setNullAt(0); + writer.writeBitmap(1, bm1); + writer.writeBitmap(2, bm2); + writer.complete(); + + assertThat(array.isNullAt(0)).isTrue(); + assertThat(array.getBitmap(1)).isEqualTo(bm1); + assertThat(array.getBitmap(2)).isEqualTo(bm2); + + BinaryArrayData newArray = splitArray(array); + assertThat(newArray.isNullAt(0)).isTrue(); + assertThat(newArray.getBitmap(1)).isEqualTo(bm1); + assertThat(newArray.getBitmap(2)).isEqualTo(bm2); + } + BinaryArrayData subArray = new BinaryArrayData(); BinaryArrayWriter subWriter = new BinaryArrayWriter(subArray, 2, 8); subWriter.setNullAt(0); diff --git a/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/data/BinaryRowDataTest.java b/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/data/BinaryRowDataTest.java index 3123669edf270..3344997ea038b 100644 --- a/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/data/BinaryRowDataTest.java +++ b/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/data/BinaryRowDataTest.java @@ -52,6 +52,7 @@ import org.apache.flink.table.types.logical.RowType; import org.apache.flink.table.types.logical.VarCharType; import org.apache.flink.types.RowKind; +import org.apache.flink.types.bitmap.Bitmap; import org.apache.flink.types.variant.Variant; import org.apache.flink.types.variant.VariantBuilder; @@ -1148,4 +1149,23 @@ public void testVariant() { assertThat(row.getVariant(0)).isEqualTo(v1); assertThat(row.getVariant(1)).isEqualTo(v2); } + + @Test + public void testBitmap() { + BinaryRowData row = new BinaryRowData(3); + BinaryRowWriter writer = new BinaryRowWriter(row); + + Bitmap bm1 = Bitmap.empty(); + bm1.add(1L, 100L); + Bitmap bm2 = Bitmap.empty(); + bm2.add(0xFFFF + 1); + writer.setNullAt(0); + writer.writeBitmap(1, bm1); + writer.writeBitmap(2, bm2); + writer.complete(); + + assertThat(row.isNullAt(0)).isTrue(); + assertThat(row.getBitmap(1)).isEqualTo(bm1); + assertThat(row.getBitmap(2)).isEqualTo(bm2); + } } diff --git a/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/data/DataFormatConvertersTest.java b/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/data/DataFormatConvertersTest.java index d637e8b052eb1..45390c54c08f7 100644 --- a/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/data/DataFormatConvertersTest.java +++ b/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/data/DataFormatConvertersTest.java @@ -49,6 +49,7 @@ import org.apache.flink.table.utils.DateTimeUtils; import org.apache.flink.types.Row; import org.apache.flink.types.RowKind; +import org.apache.flink.types.bitmap.Bitmap; import org.junit.jupiter.api.Test; @@ -128,7 +129,8 @@ class DataFormatConvertersTest { new LegacyTypeInformationType<>( LogicalTypeRoot.TIMESTAMP_WITHOUT_TIME_ZONE, new LegacyTimestampTypeInfo(7))), - DataTypes.TIMESTAMP(3).bridgedTo(TimestampData.class) + DataTypes.TIMESTAMP(3).bridgedTo(TimestampData.class), + DataTypes.BITMAP() }; private Object[] dataValues = @@ -140,7 +142,8 @@ class DataFormatConvertersTest { buildTimestamp(4000L, 123456789), Timestamp.valueOf("1970-01-01 00:00:00.123"), Timestamp.valueOf("1970-01-01 00:00:00.1234567"), - TimestampData.fromEpochMillis(1000L) + TimestampData.fromEpochMillis(1000L), + Bitmap.fromArray(new int[] {1, 2, 3}) }; private static Timestamp buildTimestamp(long mills, int nanos) { diff --git a/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/data/DataStructureConvertersTest.java b/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/data/DataStructureConvertersTest.java index d371a9b88b331..2c19c579a4ec5 100644 --- a/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/data/DataStructureConvertersTest.java +++ b/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/data/DataStructureConvertersTest.java @@ -28,6 +28,7 @@ import org.apache.flink.table.types.utils.DataTypeFactoryMock; import org.apache.flink.types.Row; import org.apache.flink.types.RowKind; +import org.apache.flink.types.bitmap.Bitmap; import org.apache.flink.types.variant.Variant; import org.apache.flink.util.InstantiationUtil; @@ -359,6 +360,8 @@ static List testData() { GenericPojo.class, new GenericPojo<>(LocalDate.ofEpochDay(123))), TestSpec.forDataType(DataTypes.VARIANT()) .convertedTo(Variant.class, Variant.newBuilder().of("hello")), + TestSpec.forDataType(DataTypes.BITMAP()) + .convertedTo(Bitmap.class, Bitmap.fromArray(new int[] {1, 2, 3})), // partial delete messages TestSpec.forDataType( @@ -396,7 +399,8 @@ static List testData() { DataTypes.FIELD( "f19", DataTypes.MULTISET(DataTypes.INT().notNull()) - .notNull()))) + .notNull()), + DataTypes.FIELD("f20", DataTypes.BITMAP().notNull()))) .convertedTo( Row.class, Row.ofKind( @@ -420,6 +424,7 @@ static List testData() { null, null, new Integer[] {null, null}, + null, null))); } diff --git a/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/data/RowDataTest.java b/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/data/RowDataTest.java index e4fe0740e5009..787e54957c749 100644 --- a/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/data/RowDataTest.java +++ b/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/data/RowDataTest.java @@ -47,6 +47,7 @@ import org.apache.flink.table.types.logical.TinyIntType; import org.apache.flink.table.types.logical.VarCharType; import org.apache.flink.types.RowKind; +import org.apache.flink.types.bitmap.Bitmap; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -62,6 +63,8 @@ /** Test for {@link RowData}s. */ class RowDataTest { + private static final int NUM_FIELDS = 19; + private StringData str; private RawValueData generic; private DecimalData decimal1; @@ -73,6 +76,7 @@ class RowDataTest { private RawValueDataSerializer genericSerializer; private TimestampData timestamp1; private TimestampData timestamp2; + private Bitmap bitmap; @BeforeEach void before() { @@ -100,6 +104,7 @@ void before() { timestamp1 = TimestampData.fromEpochMillis(123L); timestamp2 = TimestampData.fromLocalDateTime(LocalDateTime.of(1969, 1, 1, 0, 0, 0, 123456789)); + bitmap = Bitmap.fromArray(new int[] {1, 2, 3}); } @Test @@ -116,13 +121,13 @@ void testNestedRow() { writer.writeRow(0, getBinaryRow(), null); writer.complete(); - RowData nestedRow = row.getRow(0, 18); + RowData nestedRow = row.getRow(0, NUM_FIELDS); testGetters(nestedRow); testSetters(nestedRow); } private BinaryRowData getBinaryRow() { - BinaryRowData row = new BinaryRowData(18); + BinaryRowData row = new BinaryRowData(NUM_FIELDS); BinaryRowWriter writer = new BinaryRowWriter(row); writer.writeBoolean(0, true); writer.writeByte(1, (byte) 1); @@ -146,12 +151,13 @@ private BinaryRowData getBinaryRow() { writer.writeBinary(15, bytes); writer.writeTimestamp(16, timestamp1, 3); writer.writeTimestamp(17, timestamp2, 9); + writer.writeBitmap(18, bitmap); return row; } @Test void testGenericRow() { - GenericRowData row = new GenericRowData(18); + GenericRowData row = new GenericRowData(NUM_FIELDS); row.setField(0, true); row.setField(1, (byte) 1); row.setField(2, (short) 2); @@ -170,12 +176,13 @@ void testGenericRow() { row.setField(15, bytes); row.setField(16, timestamp1); row.setField(17, timestamp2); + row.setField(18, bitmap); testGetters(row); } @Test - void testBoxedWrapperRow() { - BoxedWrapperRowData row = new BoxedWrapperRowData(18); + public void testBoxedWrapperRow() { + BoxedWrapperRowData row = new BoxedWrapperRowData(NUM_FIELDS); row.setBoolean(0, true); row.setByte(1, (byte) 1); row.setShort(2, (short) 2); @@ -193,20 +200,22 @@ void testBoxedWrapperRow() { row.setNonPrimitiveValue(15, bytes); row.setNonPrimitiveValue(16, timestamp1); row.setNonPrimitiveValue(17, timestamp2); + row.setNonPrimitiveValue(18, bitmap); testGetters(row); testSetters(row); } @Test - void testJoinedRow() { - GenericRowData row1 = new GenericRowData(5); + public void testJoinedRow() { + int row1FieldNum = 5; + GenericRowData row1 = new GenericRowData(row1FieldNum); row1.setField(0, true); row1.setField(1, (byte) 1); row1.setField(2, (short) 2); row1.setField(3, 3); row1.setField(4, (long) 4); - GenericRowData row2 = new GenericRowData(13); + GenericRowData row2 = new GenericRowData(NUM_FIELDS - row1FieldNum); row2.setField(0, (float) 5); row2.setField(1, (double) 6); row2.setField(2, (char) 7); @@ -220,6 +229,7 @@ void testJoinedRow() { row2.setField(10, bytes); row2.setField(11, timestamp1); row2.setField(12, timestamp2); + row2.setField(13, bitmap); testGetters(new JoinedRowData(row1, row2)); } @@ -348,7 +358,7 @@ private void testFieldGettersWithNull(boolean nullable) { } private void testGetters(RowData row) { - assertThat(row.getArity()).isEqualTo(18); + assertThat(row.getArity()).isEqualTo(NUM_FIELDS); // test header assertThat(row.getRowKind()).isEqualTo(RowKind.INSERT); @@ -374,6 +384,7 @@ private void testGetters(RowData row) { assertThat(row.getBinary(15)).isEqualTo(bytes); assertThat(row.getTimestamp(16, 3)).isEqualTo(timestamp1); assertThat(row.getTimestamp(17, 9)).isEqualTo(timestamp2); + assertThat(row.getBitmap(18)).isEqualTo(bitmap); } private void testSetters(RowData row) {