From 53044f1667f9827aee0909d870cfb8195d316e92 Mon Sep 17 00:00:00 2001 From: Jark Wu Date: Thu, 8 Dec 2016 22:44:29 +0800 Subject: [PATCH 1/7] [FLINK-5187] [core] Create analog of Row and RowTypeInfo and RowComparator in core --- docs/dev/types_serialization.md | 2 + .../flink/api/java/typeutils/RowTypeInfo.java | 176 +++++ .../java/typeutils/runtime/NullMaskUtils.java | 104 +++ .../java/typeutils/runtime/RowComparator.java | 698 ++++++++++++++++++ .../java/typeutils/runtime/RowSerializer.java | 243 ++++++ .../main/java/org/apache/flink/types/Row.java | 118 +++ .../typeutils/base/RowComparatorTest.java | 156 ++++ .../RowComparatorWithManyFieldsTests.java | 103 +++ .../typeutils/base/RowSerializerTest.java | 197 +++++ 9 files changed, 1797 insertions(+) create mode 100644 flink-core/src/main/java/org/apache/flink/api/java/typeutils/RowTypeInfo.java create mode 100644 flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/NullMaskUtils.java create mode 100644 flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/RowComparator.java create mode 100644 flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/RowSerializer.java create mode 100644 flink-core/src/main/java/org/apache/flink/types/Row.java create mode 100644 flink-core/src/test/java/org/apache/flink/api/common/typeutils/base/RowComparatorTest.java create mode 100644 flink-core/src/test/java/org/apache/flink/api/common/typeutils/base/RowComparatorWithManyFieldsTests.java create mode 100644 flink-core/src/test/java/org/apache/flink/api/common/typeutils/base/RowSerializerTest.java diff --git a/docs/dev/types_serialization.md b/docs/dev/types_serialization.md index 4b8e25f6453d3..63bf7c264e258 100644 --- a/docs/dev/types_serialization.md +++ b/docs/dev/types_serialization.md @@ -92,6 +92,8 @@ Internally, Flink makes the following distinctions between types: * POJOs: classes that follow a certain bean-like pattern + * Row (unlimited length of fields) + * Auxiliary types (Option, Either, Lists, Maps, ...) * Generic types: These will not be serialized by Flink itself, but by Kryo. diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/RowTypeInfo.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/RowTypeInfo.java new file mode 100644 index 0000000000000..8315792a49432 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/RowTypeInfo.java @@ -0,0 +1,176 @@ +/* + * 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.java.typeutils; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeutils.TypeComparator; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.java.typeutils.runtime.RowComparator; +import org.apache.flink.api.java.typeutils.runtime.RowSerializer; +import org.apache.flink.types.Row; + +import java.util.ArrayList; +import java.util.Collections; + +import static org.apache.flink.util.Preconditions.checkState; + +/** + * TypeInformation for {@link Row} + */ +@PublicEvolving +public class RowTypeInfo extends TupleTypeInfoBase { + + private static final long serialVersionUID = 9158518989896601963L; + + protected final String[] fieldNames; + /** Temporary variable for directly passing orders to comparators. */ + private boolean[] comparatorOrders = null; + + public RowTypeInfo(TypeInformation... types) { + super(Row.class, types); + + this.fieldNames = new String[types.length]; + + for (int i = 0; i < types.length; i++) { + fieldNames[i] = "f" + i; + } + } + + @Override + public TypeComparator createComparator( + int[] logicalKeyFields, + boolean[] orders, + int logicalFieldOffset, + ExecutionConfig config) { + comparatorOrders = orders; + TypeComparator comparator = super.createComparator( + logicalKeyFields, + orders, + logicalFieldOffset, + config); + comparatorOrders = null; + return comparator; + } + + @Override + protected TypeComparatorBuilder createTypeComparatorBuilder() { + if (comparatorOrders == null) { + throw new IllegalStateException("Cannot create comparator builder without orders."); + } + return new RowTypeComparatorBuilder(comparatorOrders); + } + + @Override + public String[] getFieldNames() { + return fieldNames; + } + + @Override + public int getFieldIndex(String fieldName) { + for (int i = 0; i < fieldNames.length; i++) { + if (fieldNames[i].equals(fieldName)) { + return i; + } + } + return -1; + } + + @Override + public TypeSerializer createSerializer(ExecutionConfig config) { + int len = getArity(); + TypeSerializer[] fieldSerializers = new TypeSerializer[len]; + for (int i = 0; i < len; i++) { + fieldSerializers[i] = types[i].createSerializer(config); + } + return new RowSerializer(fieldSerializers); + } + + private class RowTypeComparatorBuilder implements TypeComparatorBuilder { + + private final ArrayList fieldComparators = new ArrayList(); + private final ArrayList logicalKeyFields = new ArrayList(); + private final boolean[] comparatorOrders; + + public RowTypeComparatorBuilder(boolean[] comparatorOrders) { + this.comparatorOrders = comparatorOrders; + } + + @Override + public void initializeTypeComparatorBuilder(int size) { + fieldComparators.ensureCapacity(size); + logicalKeyFields.ensureCapacity(size); + } + + @Override + public void addComparatorField(int fieldId, TypeComparator comparator) { + fieldComparators.add(comparator); + logicalKeyFields.add(fieldId); + } + + @Override + public TypeComparator createTypeComparator(ExecutionConfig config) { + checkState( + fieldComparators.size() > 0, + "No field comparators were defined for the TupleTypeComparatorBuilder." + ); + + checkState( + logicalKeyFields.size() > 0, + "No key fields were defined for the TupleTypeComparatorBuilder." + ); + + checkState( + fieldComparators.size() == logicalKeyFields.size(), + "The number of field comparators and key fields is not equal." + ); + + final int maxKey = Collections.max(logicalKeyFields); + + checkState( + maxKey >= 0, + "The maximum key field must be greater or equal than 0." + ); + + TypeSerializer[] fieldSerializers = new TypeSerializer[maxKey + 1]; + + for (int i = 0; i <= maxKey; i++) { + fieldSerializers[i] = types[i].createSerializer(config); + } + + int[] keyPositions = new int[logicalKeyFields.size()]; + for (int i = 0; i < keyPositions.length; i++) { + keyPositions[i] = logicalKeyFields.get(i); + } + + TypeComparator[] comparators = new TypeComparator[fieldComparators.size()]; + for (int i = 0; i < fieldComparators.size(); i++) { + comparators[i] = fieldComparators.get(i); + } + + //noinspection unchecked + return new RowComparator( + getArity(), + keyPositions, + comparators, + (TypeSerializer[]) fieldSerializers, + comparatorOrders); + } + } +} diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/NullMaskUtils.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/NullMaskUtils.java new file mode 100644 index 0000000000000..18820917ca628 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/NullMaskUtils.java @@ -0,0 +1,104 @@ +/* + * 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.java.typeutils.runtime; + +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; +import org.apache.flink.types.Row; + +import java.io.IOException; + +public class NullMaskUtils { + + public static void writeNullMask(int len, Row value, DataOutputView target) throws IOException { + int b = 0x00; + int bytePos = 0; + + int fieldPos = 0; + int numPos = 0; + while (fieldPos < len) { + b = 0x00; + // set bits in byte + bytePos = 0; + numPos = Math.min(8, len - fieldPos); + while (bytePos < numPos) { + b = b << 1; + // set bit if field is null + if (value.getField(fieldPos + bytePos) == null) { + b |= 0x01; + } + bytePos += 1; + } + fieldPos += numPos; + // shift bits if last byte is not completely filled + b <<= (8 - bytePos); + // write byte + target.writeByte(b); + } + } + + public static void readIntoNullMask( + int len, + DataInputView source, + boolean[] nullMask) throws IOException { + int b = 0x00; + int bytePos = 0; + + int fieldPos = 0; + int numPos = 0; + while (fieldPos < len) { + // read byte + b = source.readUnsignedByte(); + bytePos = 0; + numPos = Math.min(8, len - fieldPos); + while (bytePos < numPos) { + nullMask[fieldPos + bytePos] = (b & 0x80) > 0; + b = b << 1; + bytePos += 1; + } + fieldPos += numPos; + } + } + + public static void readIntoAndCopyNullMask( + int len, + DataInputView source, + DataOutputView target, + boolean[] nullMask) throws IOException { + + int b = 0x00; + int bytePos = 0; + + int fieldPos = 0; + int numPos = 0; + while (fieldPos < len) { + // read byte + b = source.readUnsignedByte(); + // copy byte + target.writeByte(b); + bytePos = 0; + numPos = Math.min(8, len - fieldPos); + while (bytePos < numPos) { + nullMask[fieldPos + bytePos] = (b & 0x80) > 0; + b = b << 1; + bytePos += 1; + } + fieldPos += numPos; + } + } +} diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/RowComparator.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/RowComparator.java new file mode 100644 index 0000000000000..aaf942e491143 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/RowComparator.java @@ -0,0 +1,698 @@ +/* + * 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.java.typeutils.runtime; + +import org.apache.flink.api.common.typeutils.CompositeTypeComparator; +import org.apache.flink.api.common.typeutils.TypeComparator; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.java.tuple.Tuple4; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; +import org.apache.flink.core.memory.MemorySegment; +import org.apache.flink.types.KeyFieldOutOfBoundsException; +import org.apache.flink.types.Row; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +import static org.apache.flink.api.java.typeutils.runtime.NullMaskUtils.readIntoNullMask; +import static org.apache.flink.util.Preconditions.checkArgument; + +/** + * Comparator for {@link Row} + */ +public class RowComparator extends CompositeTypeComparator { + + private static final long serialVersionUID = 1L; + /** The number of fields of the Row */ + private final int arity; + /** key positions describe which fields are keys in what order */ + private final int[] keyPositions; + /** null-aware comparators for the key fields, in the same order as the key fields */ + private final NullAwareComparator[] comparators; + /** serializers to deserialize the first n fields for comparison */ + private final TypeSerializer[] serializers; + /** auxiliary fields for normalized key support */ + private final int[] normalizedKeyLengths; + private final int numLeadingNormalizableKeys; + private final int normalizableKeyPrefixLen; + private final boolean invertNormKey; + + // null masks for serialized comparison + private final boolean[] nullMask1; + private final boolean[] nullMask2; + + // cache for the deserialized key field objects + transient private final Object[] deserializedKeyFields1; + transient private final Object[] deserializedKeyFields2; + + /** + * General constructor for RowComparator. + * + * @param arity the number of fields of the Row + * @param keyPositions key positions describe which fields are keys in what order + * @param comparators non-null-aware comparators for the key fields, in the same order as + * the key fields + * @param serializers serializers to deserialize the first n fields for comparison + * @param orders sorting orders for the fields + */ + public RowComparator( + int arity, + int[] keyPositions, + TypeComparator[] comparators, + TypeSerializer[] serializers, + boolean[] orders) { + this(arity, keyPositions, makeNullAware(comparators, orders), serializers); + } + + + /** + * Intermediate constructor for creating auxiliary fields. + */ + private RowComparator( + int arity, + int[] keyPositions, + NullAwareComparator[] comparators, + TypeSerializer[] serializers) { + this( + arity, + keyPositions, + comparators, + serializers, + createAuxiliaryFields(keyPositions, comparators)); + } + + /** + * Intermediate constructor for creating auxiliary fields. + */ + private RowComparator( + int arity, + int[] keyPositions, + NullAwareComparator[] comparators, + TypeSerializer[] serializers, + Tuple4 auxiliaryFields) { + this( + arity, + keyPositions, + comparators, + serializers, + auxiliaryFields.f0, + auxiliaryFields.f1, + auxiliaryFields.f2, + auxiliaryFields.f3); + } + + /** + * Intermediate constructor for creating auxiliary fields. + */ + private RowComparator( + int arity, + int[] keyPositions, + NullAwareComparator[] comparators, + TypeSerializer[] serializers, + int[] normalizedKeyLengths, + int numLeadingNormalizableKeys, int normalizableKeyPrefixLen, boolean invertNormKey) { + this.arity = arity; + this.keyPositions = keyPositions; + this.comparators = comparators; + this.serializers = serializers; + this.normalizedKeyLengths = normalizedKeyLengths; + this.numLeadingNormalizableKeys = numLeadingNormalizableKeys; + this.normalizableKeyPrefixLen = normalizableKeyPrefixLen; + this.invertNormKey = invertNormKey; + this.nullMask1 = new boolean[arity]; + this.nullMask2 = new boolean[arity]; + deserializedKeyFields1 = instantiateDeserializationFields(); + deserializedKeyFields2 = instantiateDeserializationFields(); + } + + // -------------------------------------------------------------------------------------------- + // Comparator Methods + // -------------------------------------------------------------------------------------------- + + @Override + public void getFlatComparator(List flatComparators) { + for (NullAwareComparator c : comparators) { + Collections.addAll(flatComparators, c.getFlatComparators()); + } + } + + @Override + public int hash(Row record) { + int code = 0; + int i = 0; + + try { + for (; i < keyPositions.length; i++) { + code *= TupleComparatorBase.HASH_SALT[i & 0x1F]; + Object element = record.getField(keyPositions[i]); // element can be null + code += comparators[i].hash(element); + } + } catch (IndexOutOfBoundsException e) { + throw new KeyFieldOutOfBoundsException(keyPositions[i]); + } + + return code; + } + + @Override + public void setReference(Row toCompare) { + int i = 0; + try { + for (; i < keyPositions.length; i++) { + TypeComparator comparator = comparators[i]; + Object element = toCompare.getField(keyPositions[i]); + comparator.setReference(element); // element can be null + } + } catch (IndexOutOfBoundsException e) { + throw new KeyFieldOutOfBoundsException(keyPositions[i]); + } + } + + @Override + public boolean equalToReference(Row candidate) { + int i = 0; + try { + for (; i < keyPositions.length; i++) { + TypeComparator comparator = comparators[i]; + Object element = candidate.getField(keyPositions[i]); // element can be null + // check if reference is not equal + if (!comparator.equalToReference(element)) { + return false; + } + } + } catch (IndexOutOfBoundsException e) { + throw new KeyFieldOutOfBoundsException(keyPositions[i]); + } + return true; + } + + @Override + public int compareToReference(TypeComparator referencedComparator) { + RowComparator other = (RowComparator) referencedComparator; + int i = 0; + try { + for (; i < keyPositions.length; i++) { + int cmp = comparators[i].compareToReference(other.comparators[i]); + if (cmp != 0) { + return cmp; + } + } + } catch (IndexOutOfBoundsException e) { + throw new KeyFieldOutOfBoundsException(keyPositions[i]); + } + return 0; + } + + @Override + public int compare(Row first, Row second) { + int i = 0; + try { + for (; i < keyPositions.length; i++) { + int keyPos = keyPositions[i]; + TypeComparator comparator = comparators[i]; + Object firstElement = first.getField(keyPos); // element can be null + Object secondElement = second.getField(keyPos); // element can be null + + int cmp = comparator.compare(firstElement, secondElement); + if (cmp != 0) { + return cmp; + } + } + } catch (IndexOutOfBoundsException e) { + throw new KeyFieldOutOfBoundsException(keyPositions[i]); + } + return 0; + } + + @Override + public int compareSerialized( + DataInputView firstSource, + DataInputView secondSource) throws IOException { + + int len = serializers.length; + int keyLen = keyPositions.length; + + readIntoNullMask(arity, firstSource, nullMask1); + readIntoNullMask(arity, secondSource, nullMask2); + + // deserialize + for (int i = 0; i < len; i++) { + TypeSerializer serializer = serializers[i]; + + // deserialize field 1 + if (!nullMask1[i]) { + deserializedKeyFields1[i] = serializer.deserialize( + deserializedKeyFields1[i], + firstSource); + } + + // deserialize field 2 + if (!nullMask2[i]) { + deserializedKeyFields2[i] = serializer.deserialize( + deserializedKeyFields2[i], + secondSource); + } + } + + // compare + for (int i = 0; i < keyLen; i++) { + int keyPos = keyPositions[i]; + TypeComparator comparator = comparators[i]; + + boolean isNull1 = nullMask1[keyPos]; + boolean isNull2 = nullMask2[keyPos]; + + int cmp = 0; + // both values are null -> equality + if (isNull1 && isNull2) { + cmp = 0; + } + // first value is null -> inequality + else if (isNull1) { + cmp = comparator.compare(null, deserializedKeyFields2[keyPos]); + } + // second value is null -> inequality + else if (isNull2) { + cmp = comparator.compare(deserializedKeyFields1[keyPos], null); + } + // no null values + else { + cmp = comparator.compare( + deserializedKeyFields1[keyPos], + deserializedKeyFields2[keyPos]); + } + + if (cmp != 0) { + return cmp; + } + } + + return 0; + } + + @Override + public boolean supportsNormalizedKey() { + return numLeadingNormalizableKeys > 0; + } + + @Override + public boolean supportsSerializationWithKeyNormalization() { + return false; + } + + @Override + public int getNormalizeKeyLen() { + return normalizableKeyPrefixLen; + } + + @Override + public boolean isNormalizedKeyPrefixOnly(int keyBytes) { + return numLeadingNormalizableKeys < keyPositions.length || + normalizableKeyPrefixLen == Integer.MAX_VALUE || + normalizableKeyPrefixLen > keyBytes; + } + + @Override + public void putNormalizedKey( + Row record, MemorySegment target, int offset, int numBytes) { + int bytesLeft = numBytes; + int currentOffset = offset; + + for (int i=0;i 0;i++){ + int len = normalizedKeyLengths[i]; + len = bytesLeft >= len ? len : bytesLeft; + + TypeComparator comparator = comparators[i]; + Object element = record.getField(keyPositions[i]); // element can be null + // write key + comparator.putNormalizedKey(element, target, currentOffset, len); + + bytesLeft -= len; + currentOffset += len; + } + + } + + @Override + public void writeWithKeyNormalization( + Row record, + DataOutputView target) throws IOException { + throw new UnsupportedOperationException( + "Record serialization with leading normalized keys not supported."); + + } + + @Override + public Row readWithKeyDenormalization(Row reuse, DataInputView source) throws IOException { + throw new UnsupportedOperationException( + "Record deserialization with leading normalized keys not supported."); + } + + @Override + public boolean invertNormalizedKey() { + return invertNormKey; + } + + @Override + public TypeComparator duplicate() { + NullAwareComparator[] comparatorsCopy = new NullAwareComparator[comparators.length]; + for (int i = 0; i < comparators.length; i++) { + comparatorsCopy[i] = (NullAwareComparator) comparators[i].duplicate(); + } + + TypeSerializer[] serializersCopy = new TypeSerializer[serializers.length]; + for (int i = 0; i < serializers.length; i++) { + serializersCopy[i] = serializers[i].duplicate(); + } + + return new RowComparator( + arity, + keyPositions, + (NullAwareComparator[]) comparatorsCopy, + (TypeSerializer[]) serializersCopy, + normalizedKeyLengths, + numLeadingNormalizableKeys, + normalizableKeyPrefixLen, + invertNormKey); + } + + @Override + public int extractKeys(Object record, Object[] target, int index) { + int len = comparators.length; + int localIndex = index; + for(int i=0;i + createAuxiliaryFields(int[] keyPositions, NullAwareComparator[] comparators) { + int[] normalizedKeyLengths = new int[keyPositions.length]; + int numLeadingNormalizableKeys = 0; + int normalizableKeyPrefixLen = 0; + boolean inverted = false; + + for (int i = 0; i < keyPositions.length; i++) { + NullAwareComparator k = comparators[i]; + // as long as the leading keys support normalized keys, we can build up the composite key + if (k.supportsNormalizedKey()) { + if (i == 0) { + // the first comparator decides whether we need to invert the key direction + inverted = k.invertNormalizedKey(); + } else if (k.invertNormalizedKey() != inverted) { + // if a successor does not agree on the inversion direction, it cannot be part of the + // normalized key + return new Tuple4<>( + normalizedKeyLengths, + numLeadingNormalizableKeys, + normalizableKeyPrefixLen, + inverted); + } + numLeadingNormalizableKeys++; + int len = k.getNormalizeKeyLen(); + if (len < 0) { + throw new RuntimeException( + "Comparator " + k.getClass().getName() + + " specifies an invalid length for the normalized key: " + len); + } + normalizedKeyLengths[i] = len; + normalizableKeyPrefixLen += len; + if (normalizableKeyPrefixLen < 0) { + // overflow, which means we are out of budget for normalized key space anyways + return new Tuple4<>( + normalizedKeyLengths, + numLeadingNormalizableKeys, + Integer.MAX_VALUE, + inverted); + } + } else { + return new Tuple4<>( + normalizedKeyLengths, + numLeadingNormalizableKeys, + normalizableKeyPrefixLen, + inverted); + } + } + return new Tuple4<>( + normalizedKeyLengths, + numLeadingNormalizableKeys, + normalizableKeyPrefixLen, + inverted); + } + + private static NullAwareComparator[] makeNullAware( + TypeComparator[] comparators, + boolean[] orders) { + checkArgument(comparators.length == orders.length); + NullAwareComparator[] result = new NullAwareComparator[comparators.length]; + for (int i = 0; i < comparators.length; i++) { + result[i] = new NullAwareComparator(comparators[i], orders[i]); + } + return (NullAwareComparator[]) result; + } + + + /** + * Null-aware comparator that wraps a comparator which does not support null references. + *

+ * NOTE: This class assumes to be used within a composite type comparator (such + * as {@link RowComparator}) that handles serialized comparison. + */ + public static class NullAwareComparator extends TypeComparator { + private static final long serialVersionUID = 1L; + + private final TypeComparator wrappedComparator; + private final boolean order; + + // number of flat fields + private final int flatFields; + + // stores the null for reference comparison + private boolean nullReference = false; + + public NullAwareComparator(TypeComparator wrappedComparator, boolean order) { + this.wrappedComparator = wrappedComparator; + this.order = order; + this.flatFields = wrappedComparator.getFlatComparators().length; + } + + @Override + public int hash(T record) { + if (record != null) { + return wrappedComparator.hash(record); + } else { + return 0; + } + } + + @Override + public void setReference(T toCompare) { + if (toCompare == null) { + nullReference = true; + } else { + nullReference = false; + wrappedComparator.setReference(toCompare); + } + } + + @Override + public boolean equalToReference(T candidate) { + // both values are null + if (candidate == null && nullReference) { + return true; + } + // one value is null + else if (candidate == null || nullReference) { + return false; + } + // no null value + else { + return wrappedComparator.equalToReference(candidate); + } + } + + @Override + public int compareToReference(TypeComparator referencedComparator) { + NullAwareComparator otherComparator = (NullAwareComparator) referencedComparator; + boolean otherNullReference = otherComparator.nullReference; + // both values are null -> equality + if (nullReference && otherNullReference) { + return 0; + } + // first value is null -> inequality + // but order is considered + else if (nullReference) { + return order ? 1 : -1; + } + // second value is null -> inequality + // but order is considered + else if (otherNullReference) { + return order ? -1 : 1; + } + // no null values + else { + return wrappedComparator.compareToReference(otherComparator.wrappedComparator); + } + } + + @Override + public int compare(T first, T second) { + // both values are null -> equality + if (first == null && second == null) { + return 0; + } + // first value is null -> inequality + // but order is considered + else if (first == null) { + return order ? -1 : 1; + } + // second value is null -> inequality + // but order is considered + else if (second == null) { + return order ? 1 : -1; + } + // no null values + else { + return wrappedComparator.compare(first, second); + } + } + + @Override + public int compareSerialized( + DataInputView firstSource, + DataInputView secondSource) throws IOException { + + throw new UnsupportedOperationException( + "Comparator does not support null-aware serialized comparision."); + } + + @Override + public boolean supportsNormalizedKey() { + return wrappedComparator.supportsNormalizedKey(); + } + + @Override + public boolean supportsSerializationWithKeyNormalization() { + return false; + } + + @Override + public int getNormalizeKeyLen() { + int len = wrappedComparator.getNormalizeKeyLen(); + if (len == Integer.MAX_VALUE) { + return Integer.MAX_VALUE; + } else { + // add one for a null byte + return len + 1; + } + } + + @Override + public boolean isNormalizedKeyPrefixOnly(int keyBytes) { + return wrappedComparator.isNormalizedKeyPrefixOnly(keyBytes - 1); + } + + @Override + public void putNormalizedKey(T record, MemorySegment target, int offset, int numBytes) { + if (numBytes > 0) { + // write a null byte with padding + if (record == null) { + target.putBoolean(offset, false); + // write padding + for (int j = 0; j < numBytes - 1; j++) { + target.put(offset + 1 + j, (byte) 0); + } + } + // write a non-null byte with key + else { + target.putBoolean(offset, true); + // write key + wrappedComparator.putNormalizedKey(record, target, offset + 1, numBytes - 1); + } + } + } + + @Override + public void writeWithKeyNormalization(T record, DataOutputView target) throws IOException { + throw new UnsupportedOperationException( + "Record serialization with leading normalized keys not supported."); + } + + @Override + public T readWithKeyDenormalization(T reuse, DataInputView source) throws IOException { + throw new UnsupportedOperationException( + "Record deserialization with leading normalized keys not supported."); + } + + @Override + public boolean invertNormalizedKey() { + return wrappedComparator.invertNormalizedKey(); + } + + @Override + public TypeComparator duplicate() { + return new NullAwareComparator(wrappedComparator.duplicate(), order); + } + + @Override + public int extractKeys(Object record, Object[] target, int index) { + if (record == null) { + for (int i = 0; i < flatFields; i++) { + target[index + i] = null; + } + return flatFields; + } else { + return wrappedComparator.extractKeys(record, target, index); + } + } + + @Override + public TypeComparator[] getFlatComparators() { + // determine the flat comparators and wrap them again in null-aware comparators + List> flatComparators = new ArrayList<>(); + if (wrappedComparator instanceof CompositeTypeComparator) { + ((CompositeTypeComparator) wrappedComparator).getFlatComparator(flatComparators); + } else { + flatComparators.add(wrappedComparator); + } + + TypeComparator[] result = new TypeComparator[flatComparators.size()]; + for (int i = 0; i < result.length; i++) { + result[i] = new NullAwareComparator<>(flatComparators.get(i), order); + } + return result; + } + } +} diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/RowSerializer.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/RowSerializer.java new file mode 100644 index 0000000000000..5457c0591ac33 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/RowSerializer.java @@ -0,0 +1,243 @@ +/* + * 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.java.typeutils.runtime; + +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; +import org.apache.flink.types.Row; + +import java.io.IOException; +import java.util.Arrays; + +import static org.apache.flink.api.java.typeutils.runtime.NullMaskUtils.readIntoAndCopyNullMask; +import static org.apache.flink.api.java.typeutils.runtime.NullMaskUtils.readIntoNullMask; +import static org.apache.flink.api.java.typeutils.runtime.NullMaskUtils.writeNullMask; +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Serializer for {@link Row}. + */ +public class RowSerializer extends TypeSerializer { + + private static final long serialVersionUID = 1L; + private final boolean[] nullMask; + private final TypeSerializer[] fieldSerializers; + + public RowSerializer(TypeSerializer[] fieldSerializers) { + this.fieldSerializers = (TypeSerializer[]) checkNotNull(fieldSerializers); + this.nullMask = new boolean[fieldSerializers.length]; + } + + @Override + public boolean isImmutableType() { + return false; + } + + @Override + public TypeSerializer duplicate() { + boolean stateful = false; + TypeSerializer[] duplicateFieldSerializers = new TypeSerializer[fieldSerializers.length]; + + for (int i = 0; i < fieldSerializers.length; i++) { + duplicateFieldSerializers[i] = fieldSerializers[i].duplicate(); + if (duplicateFieldSerializers[i] != fieldSerializers[i]) { + // at least one of them is stateful + stateful = true; + } + } + + if (stateful) { + return new RowSerializer(duplicateFieldSerializers); + } else { + return this; + } + } + + @Override + public Row createInstance() { + return new Row(fieldSerializers.length); + } + + @Override + public Row copy(Row from) { + int len = fieldSerializers.length; + + if (from.getArity() != len) { + throw new RuntimeException("Row arity of from does not match serializers."); + } + + Row result = new Row(len); + for (int i = 0; i < len; i++) { + Object fromField = from.getField(i); + if (fromField != null) { + Object copy = fieldSerializers[i].copy(fromField); + result.setField(i, copy); + } else { + result.setField(i, null); + } + } + return result; + } + + @Override + public Row copy(Row from, Row reuse) { + int len = fieldSerializers.length; + + // cannot reuse, do a non-reuse copy + if (reuse == null) { + return copy(from); + } + + if (from.getArity() != len || reuse.getArity() != len) { + throw new RuntimeException( + "Row arity of reuse or from is incompatible with this RowSerializer."); + } + + for (int i = 0; i < len; i++) { + Object fromField = from.getField(i); + if (fromField != null) { + Object reuseField = reuse.getField(i); + if (reuseField != null) { + Object copy = fieldSerializers[i].copy(fromField, reuseField); + reuse.setField(i, copy); + } else { + Object copy = fieldSerializers[i].copy(fromField); + reuse.setField(i, copy); + } + } else { + reuse.setField(i, null); + } + } + return reuse; + } + + @Override + public int getLength() { + return -1; + } + + @Override + public void serialize(Row record, DataOutputView target) throws IOException { + int len = fieldSerializers.length; + + if (record.getArity() != len) { + throw new RuntimeException("Row arity of from does not match serializers."); + } + + // write a null mask + writeNullMask(len, record, target); + + // serialize non-null fields + for (int i = 0; i < len; i++) { + Object o = record.getField(i); + if (o != null) { + fieldSerializers[i].serialize(o, target); + } + } + } + + + @Override + public Row deserialize(DataInputView source) throws IOException { + int len = fieldSerializers.length; + + Row result = new Row(len); + + // read null mask + readIntoNullMask(len, source, nullMask); + + for (int i = 0; i < len; i++) { + if (nullMask[i]) { + result.setField(i, null); + } else { + result.setField(i, fieldSerializers[i].deserialize(source)); + } + } + + return result; + } + + @Override + public Row deserialize(Row reuse, DataInputView source) throws IOException { + int len = fieldSerializers.length; + + if (reuse.getArity() != len) { + throw new RuntimeException("Row arity of from does not match serializers."); + } + + // read null mask + readIntoNullMask(len, source, nullMask); + + for (int i = 0; i < len; i++) { + if (nullMask[i]) { + reuse.setField(i, null); + } else { + Object reuseField = reuse.getField(i); + if (reuseField != null) { + reuse.setField(i, fieldSerializers[i].deserialize(reuseField, source)); + } else { + reuse.setField(i, fieldSerializers[i].deserialize(source)); + } + } + } + + return reuse; + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + int len = fieldSerializers.length; + + // copy null mask + readIntoAndCopyNullMask(len, source, target, nullMask); + + for (int i = 0; i < len; i++) { + if (!nullMask[i]) { + fieldSerializers[i].copy(source, target); + } + } + } + + @Override + public boolean equals(Object obj) { + if (canEqual(obj)) { + RowSerializer other = (RowSerializer) obj; + if (this.fieldSerializers.length == other.fieldSerializers.length) { + for (int i = 0; i < this.fieldSerializers.length; i++) { + if (!this.fieldSerializers[i].equals(other.fieldSerializers[i])) { + return false; + } + } + return true; + } + } + + return false; + } + + @Override + public boolean canEqual(Object obj) { + return obj instanceof RowSerializer; + } + + @Override + public int hashCode() { + return Arrays.hashCode(fieldSerializers); + } +} diff --git a/flink-core/src/main/java/org/apache/flink/types/Row.java b/flink-core/src/main/java/org/apache/flink/types/Row.java new file mode 100644 index 0000000000000..31b5c9d233fe7 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/types/Row.java @@ -0,0 +1,118 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.types; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.api.java.typeutils.RowTypeInfo; + +import java.io.Serializable; +import java.util.Arrays; + +/** + * A Row has no limited length and contain a set of fields, which may all be different types. + * Because Row is not strongly typed, Flink's type extraction mechanism can't extract correct field + * types. So that users should manually tell Flink the type information via creating a + * {@link RowTypeInfo}. + * + *

+ * The fields in the Row may be accessed by position (zero-based) {@link #getField(int)}. And can + * set fields by {@link #setField(int, Object)}. + *

+ * Row is in principle serializable. However, it may contain non-serializable fields, + * in which case serialization will fail. + * + */ +@PublicEvolving +public class Row implements Serializable{ + + private static final long serialVersionUID = 1L; + + /** Number of field. */ + private final int arity; + /** The array to store actual values. */ + private final Object[] fields; + + /** + * Create a new Row instance. + * @param arity The number of field in the Row + */ + public Row(int arity) { + this.arity = arity; + this.fields = new Object[arity]; + } + + /** + * Get the number of field in the Row. + * @return The number of field in the Row. + */ + public int getArity() { + return arity; + } + + /** + * Gets the field at the specified position. + * @param pos The position of the field, 0-based. + * @return The field at the specified position. + * @throws IndexOutOfBoundsException Thrown, if the position is negative, or equal to, or larger than the number of fields. + */ + public Object getField(int pos) { + return fields[pos]; + } + + /** + * Sets the field at the specified position. + * + * @param pos The position of the field, 0-based. + * @param value The value to be assigned to the field at the specified position. + * @throws IndexOutOfBoundsException Thrown, if the position is negative, or equal to, or larger than the number of fields. + */ + public void setField(int pos, Object value) { + fields[pos] = value; + } + + @Override + public String toString() { + return Arrays.deepToString(fields); + } + + /** + * Deep equality for Row by calling equals on each field. + * @param o The object checked for equality. + * @return true if this is equal to o. + */ + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + Row row = (Row) o; + + return arity == row.arity && Arrays.equals(fields, row.fields); + } + + @Override + public int hashCode() { + int result = arity; + result = 31 * result + Arrays.hashCode(fields); + return result; + } +} diff --git a/flink-core/src/test/java/org/apache/flink/api/common/typeutils/base/RowComparatorTest.java b/flink-core/src/test/java/org/apache/flink/api/common/typeutils/base/RowComparatorTest.java new file mode 100644 index 0000000000000..a56ff421083a8 --- /dev/null +++ b/flink-core/src/test/java/org/apache/flink/api/common/typeutils/base/RowComparatorTest.java @@ -0,0 +1,156 @@ +/* + * 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.ExecutionConfig; +import org.apache.flink.api.common.typeinfo.BasicTypeInfo; +import org.apache.flink.api.common.typeutils.ComparatorTestBase; +import org.apache.flink.api.common.typeutils.TypeComparator; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.api.java.typeutils.TupleTypeInfo; +import org.apache.flink.api.java.typeutils.TypeExtractor; +import org.apache.flink.types.Row; +import org.junit.Before; + +import java.io.Serializable; + +import static org.junit.Assert.assertEquals; + +public class RowComparatorTest extends ComparatorTestBase { + + private final RowTypeInfo typeInfo = new RowTypeInfo( + BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.DOUBLE_TYPE_INFO, + BasicTypeInfo.STRING_TYPE_INFO, + new TupleTypeInfo>( + BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.BOOLEAN_TYPE_INFO, + BasicTypeInfo.SHORT_TYPE_INFO + ), + TypeExtractor.createTypeInfo(MyPojo.class)); + + private MyPojo testPojo1 = new MyPojo(); + private MyPojo testPojo2 = new MyPojo(); + private MyPojo testPojo3 = new MyPojo(); + + private final Row[] data = new Row[]{ + createRow(null, null, null, null, null), + createRow(0, null, null, null, null), + createRow(0, 0.0, null, null, null), + createRow(0, 0.0, "a", null, null), + createRow(1, 0.0, "a", null, null), + createRow(1, 1.0, "a", null, null), + createRow(1, 1.0, "b", null, null), + createRow(1, 1.0, "b", new Tuple3<>(1, false, (short) 2), null), + createRow(1, 1.0, "b", new Tuple3<>(2, false, (short) 2), null), + createRow(1, 1.0, "b", new Tuple3<>(2, true, (short) 2), null), + createRow(1, 1.0, "b", new Tuple3<>(2, true, (short) 3), null), + createRow(1, 1.0, "b", new Tuple3<>(2, true, (short) 3), testPojo1), + createRow(1, 1.0, "b", new Tuple3<>(2, true, (short) 3), testPojo2), + createRow(1, 1.0, "b", new Tuple3<>(2, true, (short) 3), testPojo3) + }; + + @Before + public void init() { + // TODO we cannot test null here as PojoComparator has no support for null keys + testPojo1.name = ""; + testPojo2.name = "Test1"; + testPojo3.name = "Test2"; + } + + @Override + protected void deepEquals(String message, Row should, Row is) { + int arity = should.getArity(); + assertEquals(message, arity, is.getArity()); + for (int i = 0; i < arity; i++) { + Object copiedValue = should.getField(i); + Object element = is.getField(i); + assertEquals(message, element, copiedValue); + } + } + + @Override + protected TypeComparator createComparator(boolean ascending) { + return typeInfo.createComparator( + new int[] {0, 1, 2, 3, 4, 5, 6}, + new boolean[] {ascending, ascending, ascending, ascending, ascending, ascending, ascending}, + 0, + new ExecutionConfig()); + } + + @Override + protected TypeSerializer createSerializer() { + return typeInfo.createSerializer(new ExecutionConfig()); + } + + @Override + protected Row[] getSortedTestData() { + return data; + } + + @Override + protected boolean supportsNullKeys() { + return true; + } + + private static Row createRow(Object f0, Object f1, Object f2, Object f3, Object f4) { + Row row = new Row(5); + row.setField(0, f0); + row.setField(1, f1); + row.setField(2, f2); + row.setField(3, f3); + row.setField(4, f4); + return row; + } + + public static class MyPojo implements Serializable, Comparable { + // we cannot use null because the PojoComparator does not support null properly + public String name = ""; + + @Override + public int compareTo(MyPojo o) { + if (name == null && o.name == null) { + return 0; + } else if (name == null) { + return -1; + } else if (o.name == null) { + return 1; + } else { + return name.compareTo(o.name); + } + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + MyPojo myPojo = (MyPojo) o; + + return name != null ? name.equals(myPojo.name) : myPojo.name == null; + + } + + } +} diff --git a/flink-core/src/test/java/org/apache/flink/api/common/typeutils/base/RowComparatorWithManyFieldsTests.java b/flink-core/src/test/java/org/apache/flink/api/common/typeutils/base/RowComparatorWithManyFieldsTests.java new file mode 100644 index 0000000000000..ffaf94303a25a --- /dev/null +++ b/flink-core/src/test/java/org/apache/flink/api/common/typeutils/base/RowComparatorWithManyFieldsTests.java @@ -0,0 +1,103 @@ +/* + * 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.ExecutionConfig; +import org.apache.flink.api.common.typeinfo.BasicTypeInfo; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeutils.ComparatorTestBase; +import org.apache.flink.api.common.typeutils.TypeComparator; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.api.java.typeutils.runtime.RowComparator; +import org.apache.flink.types.Row; +import org.junit.Before; + +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkNotNull; +import static org.junit.Assert.assertEquals; + +/** + * Tests {@link RowComparator} for wide rows. + */ +public class RowComparatorWithManyFieldsTests extends ComparatorTestBase { + + private final int numberOfFields = 10; + private RowTypeInfo typeInfo; + private final Row[] data = new Row[]{ + createRow(null, "b0", "c0", "d0", "e0", "f0", "g0", "h0", "i0", "j0"), + createRow("a1", "b1", "c1", "d1", "e1", "f1", "g1", "h1", "i1", "j1"), + createRow("a2", "b2", "c2", "d2", "e2", "f2", "g2", "h2", "i2", "j2"), + createRow("a3", "b3", "c3", "d3", "e3", "f3", "g3", "h3", "i3", "j3") + }; + + @Before + public void setUp() throws Exception { + TypeInformation[] fieldTypes = new TypeInformation[numberOfFields]; + for (int i = 0; i < numberOfFields; i++) { + fieldTypes[i] = BasicTypeInfo.STRING_TYPE_INFO; + } + typeInfo = new RowTypeInfo(fieldTypes); + + } + + @Override + protected void deepEquals(String message, Row should, Row is) { + int arity = should.getArity(); + assertEquals(message, arity, is.getArity()); + for (int i = 0; i < arity; i++) { + Object copiedValue = should.getField(i); + Object element = is.getField(i); + assertEquals(message, element, copiedValue); + } + } + + @Override + protected TypeComparator createComparator(boolean ascending) { + return typeInfo.createComparator( + new int[]{0}, + new boolean[]{ascending}, + 0, + new ExecutionConfig()); + } + + @Override + protected TypeSerializer createSerializer() { + return typeInfo.createSerializer(new ExecutionConfig()); + } + + @Override + protected Row[] getSortedTestData() { + return data; + } + + @Override + protected boolean supportsNullKeys() { + return true; + } + + private Row createRow(Object... values) { + checkNotNull(values); + checkArgument(values.length == numberOfFields); + Row row = new Row(numberOfFields); + for (int i = 0; i < values.length; i++) { + row.setField(i, values[i]); + } + return row; + } +} diff --git a/flink-core/src/test/java/org/apache/flink/api/common/typeutils/base/RowSerializerTest.java b/flink-core/src/test/java/org/apache/flink/api/common/typeutils/base/RowSerializerTest.java new file mode 100644 index 0000000000000..41e142ea29f5d --- /dev/null +++ b/flink-core/src/test/java/org/apache/flink/api/common/typeutils/base/RowSerializerTest.java @@ -0,0 +1,197 @@ +/* + * 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.ExecutionConfig; +import org.apache.flink.api.common.typeinfo.BasicTypeInfo; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeutils.SerializerTestInstance; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.api.java.typeutils.TupleTypeInfo; +import org.apache.flink.api.java.typeutils.TypeExtractor; +import org.apache.flink.types.Row; +import org.junit.Test; + +import java.io.Serializable; + +import static org.junit.Assert.assertEquals; + +public class RowSerializerTest { + + @Test + public void testRowSerializer() { + TypeInformation typeInfo = new RowTypeInfo( + BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.STRING_TYPE_INFO); + Row row1 = new Row(2); + row1.setField(0, 1); + row1.setField(1, "a"); + + Row row2 = new Row(2); + row2.setField(0, 2); + row2.setField(1, null); + + TypeSerializer serializer = typeInfo.createSerializer(new ExecutionConfig()); + RowSerializerTestInstance instance = new RowSerializerTestInstance(serializer, row1, row2); + instance.testAll(); + } + + @Test + public void testLargeRowSerializer() { + TypeInformation typeInfo = new RowTypeInfo( + BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.STRING_TYPE_INFO); + + Row row = new Row(13); + row.setField(0, 2); + row.setField(1, null); + row.setField(3, null); + row.setField(4, null); + row.setField(5, null); + row.setField(6, null); + row.setField(7, null); + row.setField(8, null); + row.setField(9, null); + row.setField(10, null); + row.setField(11, null); + row.setField(12, "Test"); + + TypeSerializer serializer = typeInfo.createSerializer(new ExecutionConfig()); + RowSerializerTestInstance testInstance = new RowSerializerTestInstance(serializer, row); + testInstance.testAll(); + } + + @Test + public void testRowSerializerWithComplexTypes() { + TypeInformation typeInfo = new RowTypeInfo( + BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.DOUBLE_TYPE_INFO, + BasicTypeInfo.STRING_TYPE_INFO, + new TupleTypeInfo>( + BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.BOOLEAN_TYPE_INFO, + BasicTypeInfo.SHORT_TYPE_INFO), + TypeExtractor.createTypeInfo(MyPojo.class)); + + MyPojo testPojo1 = new MyPojo(); + testPojo1.name = null; + MyPojo testPojo2 = new MyPojo(); + testPojo2.name = "Test1"; + MyPojo testPojo3 = new MyPojo(); + testPojo3.name = "Test2"; + + Row[] data = new Row[]{ + createRow(null, null, null, null, null), + createRow(0, null, null, null, null), + createRow(0, 0.0, null, null, null), + createRow(0, 0.0, "a", null, null), + createRow(1, 0.0, "a", null, null), + createRow(1, 1.0, "a", null, null), + createRow(1, 1.0, "b", null, null), + createRow(1, 1.0, "b", new Tuple3<>(1, false, (short) 2), null), + createRow(1, 1.0, "b", new Tuple3<>(2, false, (short) 2), null), + createRow(1, 1.0, "b", new Tuple3<>(2, true, (short) 2), null), + createRow(1, 1.0, "b", new Tuple3<>(2, true, (short) 3), null), + createRow(1, 1.0, "b", new Tuple3<>(2, true, (short) 3), testPojo1), + createRow(1, 1.0, "b", new Tuple3<>(2, true, (short) 3), testPojo2), + createRow(1, 1.0, "b", new Tuple3<>(2, true, (short) 3), testPojo3) + }; + + TypeSerializer serializer = typeInfo.createSerializer(new ExecutionConfig()); + RowSerializerTestInstance testInstance = new RowSerializerTestInstance(serializer, data); + testInstance.testAll(); + } + + // ---------------------------------------------------------------------------------------------- + + private static Row createRow(Object f0, Object f1, Object f2, Object f3, Object f4) { + Row row = new Row(5); + row.setField(0, f0); + row.setField(1, f1); + row.setField(2, f2); + row.setField(3, f3); + row.setField(4, f4); + return row; + } + + + private class RowSerializerTestInstance extends SerializerTestInstance { + + RowSerializerTestInstance( + TypeSerializer serializer, + Row... testData) { + super(serializer, Row.class, -1, testData); + } + + @Override + protected void deepEquals(String message, Row should, Row is) { + int arity = should.getArity(); + assertEquals(message, arity, is.getArity()); + for (int i = 0; i < arity; i++) { + Object copiedValue = should.getField(i); + Object element = is.getField(i); + assertEquals(message, element, copiedValue); + } + } + } + + public static class MyPojo implements Serializable, Comparable { + public String name = null; + + @Override + public int compareTo(MyPojo o) { + if (name == null && o.name == null) { + return 0; + } else if (name == null) { + return -1; + } else if (o.name == null) { + return 1; + } else { + return name.compareTo(o.name); + } + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + MyPojo myPojo = (MyPojo) o; + + return name != null ? name.equals(myPojo.name) : myPojo.name == null; + } + + } +} From d0ee79b4694acee3e87c2341bb4ec398cf6c25af Mon Sep 17 00:00:00 2001 From: Jark Wu Date: Fri, 9 Dec 2016 11:43:09 +0800 Subject: [PATCH 2/7] address review comments and add RowTypeInfoTest --- .../flink/api/java/typeutils/RowTypeInfo.java | 26 +++++++ .../java/typeutils/runtime/RowComparator.java | 13 ++-- .../main/java/org/apache/flink/types/Row.java | 18 ++--- .../api/java/typeutils/RowTypeInfoTest.java | 69 +++++++++++++++++++ 4 files changed, 105 insertions(+), 21 deletions(-) create mode 100644 flink-core/src/test/java/org/apache/flink/api/java/typeutils/RowTypeInfoTest.java diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/RowTypeInfo.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/RowTypeInfo.java index 8315792a49432..e3bb9b79401b1 100644 --- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/RowTypeInfo.java +++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/RowTypeInfo.java @@ -27,6 +27,7 @@ import org.apache.flink.types.Row; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; import static org.apache.flink.util.Preconditions.checkState; @@ -102,6 +103,31 @@ public TypeSerializer createSerializer(ExecutionConfig config) { return new RowSerializer(fieldSerializers); } + @Override + public boolean canEqual(Object obj) { + return obj instanceof RowTypeInfo; + } + + @Override + public int hashCode() { + return 31 * super.hashCode() + Arrays.hashCode(fieldNames); + } + + @Override + public String toString() { + StringBuilder bld = new StringBuilder("Row"); + if (types.length > 0) { + bld.append('(').append(fieldNames[0]).append(": ").append(types[0]); + + for (int i = 1; i < types.length; i++) { + bld.append(", ").append(fieldNames[i]).append(": ").append(types[i]); + } + + bld.append(')'); + } + return bld.toString(); + } + private class RowTypeComparatorBuilder implements TypeComparatorBuilder { private final ArrayList fieldComparators = new ArrayList(); diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/RowComparator.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/RowComparator.java index aaf942e491143..4918d249c654d 100644 --- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/RowComparator.java +++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/RowComparator.java @@ -326,8 +326,8 @@ public int getNormalizeKeyLen() { @Override public boolean isNormalizedKeyPrefixOnly(int keyBytes) { return numLeadingNormalizableKeys < keyPositions.length || - normalizableKeyPrefixLen == Integer.MAX_VALUE || - normalizableKeyPrefixLen > keyBytes; + normalizableKeyPrefixLen == Integer.MAX_VALUE || + normalizableKeyPrefixLen > keyBytes; } @Override @@ -336,7 +336,7 @@ public void putNormalizedKey( int bytesLeft = numBytes; int currentOffset = offset; - for (int i=0;i 0;i++){ + for (int i = 0; i < numLeadingNormalizableKeys && bytesLeft > 0; i++) { int len = normalizedKeyLengths[i]; len = bytesLeft >= len ? len : bytesLeft; @@ -352,12 +352,9 @@ public void putNormalizedKey( } @Override - public void writeWithKeyNormalization( - Row record, - DataOutputView target) throws IOException { + public void writeWithKeyNormalization(Row record, DataOutputView target) throws IOException { throw new UnsupportedOperationException( "Record serialization with leading normalized keys not supported."); - } @Override @@ -398,7 +395,7 @@ public TypeComparator duplicate() { public int extractKeys(Object record, Object[] target, int index) { int len = comparators.length; int localIndex = index; - for(int i=0;i Date: Fri, 9 Dec 2016 19:29:58 +0800 Subject: [PATCH 3/7] fix tabs --- .../flink/api/java/typeutils/runtime/RowComparator.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/RowComparator.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/RowComparator.java index 4918d249c654d..164e3f0aec101 100644 --- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/RowComparator.java +++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/RowComparator.java @@ -326,8 +326,8 @@ public int getNormalizeKeyLen() { @Override public boolean isNormalizedKeyPrefixOnly(int keyBytes) { return numLeadingNormalizableKeys < keyPositions.length || - normalizableKeyPrefixLen == Integer.MAX_VALUE || - normalizableKeyPrefixLen > keyBytes; + normalizableKeyPrefixLen == Integer.MAX_VALUE || + normalizableKeyPrefixLen > keyBytes; } @Override From 3f6bff32ca63782ff5150bdc5155f87a91523fdb Mon Sep 17 00:00:00 2001 From: Jark Wu Date: Mon, 12 Dec 2016 22:48:38 +0800 Subject: [PATCH 4/7] address review comments --- docs/dev/types_serialization.md | 8 +- .../flink/api/java/typeutils/RowTypeInfo.java | 1 + .../runtime/NullAwareComparator.java | 240 ++++++++++++++++++ .../java/typeutils/runtime/NullMaskUtils.java | 1 + .../java/typeutils/runtime/RowComparator.java | 226 +---------------- .../main/java/org/apache/flink/types/Row.java | 28 +- .../typeutils/base/RowComparatorTest.java | 16 +- .../RowComparatorWithManyFieldsTests.java | 14 +- .../java/org/apache/flink/types/RowTest.java | 37 +++ 9 files changed, 325 insertions(+), 246 deletions(-) create mode 100644 flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/NullAwareComparator.java create mode 100644 flink-core/src/test/java/org/apache/flink/types/RowTest.java diff --git a/docs/dev/types_serialization.md b/docs/dev/types_serialization.md index 63bf7c264e258..7636697014da9 100644 --- a/docs/dev/types_serialization.md +++ b/docs/dev/types_serialization.md @@ -86,13 +86,13 @@ Internally, Flink makes the following distinctions between types: * Composite types - * Flink Java Tuples (part of the Flink Java API) + * Flink Java Tuples (part of the Flink Java API): max 25 fields, null fields not supported - * Scala *case classes* (including Scala tuples) + * Scala *case classes* (including Scala tuples): max 22 fields, null fields not supported - * POJOs: classes that follow a certain bean-like pattern + * Row: tuples with arbitrary number of fields and support for null fields - * Row (unlimited length of fields) + * POJOs: classes that follow a certain bean-like pattern * Auxiliary types (Option, Either, Lists, Maps, ...) diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/RowTypeInfo.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/RowTypeInfo.java index e3bb9b79401b1..03cbe61aaf356 100644 --- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/RowTypeInfo.java +++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/RowTypeInfo.java @@ -60,6 +60,7 @@ public TypeComparator createComparator( boolean[] orders, int logicalFieldOffset, ExecutionConfig config) { + comparatorOrders = orders; TypeComparator comparator = super.createComparator( logicalKeyFields, diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/NullAwareComparator.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/NullAwareComparator.java new file mode 100644 index 0000000000000..35878114be92d --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/NullAwareComparator.java @@ -0,0 +1,240 @@ +/* + * 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.java.typeutils.runtime; + +import org.apache.flink.api.common.typeutils.CompositeTypeComparator; +import org.apache.flink.api.common.typeutils.TypeComparator; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; +import org.apache.flink.core.memory.MemorySegment; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +/** + * Null-aware comparator that wraps a comparator which does not support null references. + *

+ * NOTE: This class assumes to be used within a composite type comparator (such + * as {@link RowComparator}) that handles serialized comparison. + */ +public class NullAwareComparator extends TypeComparator { + private static final long serialVersionUID = 1L; + + private final TypeComparator wrappedComparator; + private final boolean order; + + // number of flat fields + private final int flatFields; + + // stores the null for reference comparison + private boolean nullReference = false; + + public NullAwareComparator(TypeComparator wrappedComparator, boolean order) { + this.wrappedComparator = wrappedComparator; + this.order = order; + this.flatFields = wrappedComparator.getFlatComparators().length; + } + + @Override + public int hash(T record) { + if (record != null) { + return wrappedComparator.hash(record); + } else { + return 0; + } + } + + @Override + public void setReference(T toCompare) { + if (toCompare == null) { + nullReference = true; + } else { + nullReference = false; + wrappedComparator.setReference(toCompare); + } + } + + @Override + public boolean equalToReference(T candidate) { + // both values are null + if (candidate == null && nullReference) { + return true; + } + // one value is null + else if (candidate == null || nullReference) { + return false; + } + // no null value + else { + return wrappedComparator.equalToReference(candidate); + } + } + + @Override + public int compareToReference(TypeComparator referencedComparator) { + NullAwareComparator otherComparator = (NullAwareComparator) referencedComparator; + boolean otherNullReference = otherComparator.nullReference; + // both values are null -> equality + if (nullReference && otherNullReference) { + return 0; + } + // first value is null -> inequality + // but order is considered + else if (nullReference) { + return order ? 1 : -1; + } + // second value is null -> inequality + // but order is considered + else if (otherNullReference) { + return order ? -1 : 1; + } + // no null values + else { + return wrappedComparator.compareToReference(otherComparator.wrappedComparator); + } + } + + @Override + public int compare(T first, T second) { + // both values are null -> equality + if (first == null && second == null) { + return 0; + } + // first value is null -> inequality + // but order is considered + else if (first == null) { + return order ? -1 : 1; + } + // second value is null -> inequality + // but order is considered + else if (second == null) { + return order ? 1 : -1; + } + // no null values + else { + return wrappedComparator.compare(first, second); + } + } + + @Override + public int compareSerialized( + DataInputView firstSource, + DataInputView secondSource) throws IOException { + + throw new UnsupportedOperationException( + "Comparator does not support null-aware serialized comparision."); + } + + @Override + public boolean supportsNormalizedKey() { + return wrappedComparator.supportsNormalizedKey(); + } + + @Override + public boolean supportsSerializationWithKeyNormalization() { + return false; + } + + @Override + public int getNormalizeKeyLen() { + int len = wrappedComparator.getNormalizeKeyLen(); + if (len == Integer.MAX_VALUE) { + return Integer.MAX_VALUE; + } else { + // add one for a null byte + return len + 1; + } + } + + @Override + public boolean isNormalizedKeyPrefixOnly(int keyBytes) { + return wrappedComparator.isNormalizedKeyPrefixOnly(keyBytes - 1); + } + + @Override + public void putNormalizedKey(T record, MemorySegment target, int offset, int numBytes) { + if (numBytes > 0) { + // write a null byte with padding + if (record == null) { + target.putBoolean(offset, false); + // write padding + for (int j = 0; j < numBytes - 1; j++) { + target.put(offset + 1 + j, (byte) 0); + } + } + // write a non-null byte with key + else { + target.putBoolean(offset, true); + // write key + wrappedComparator.putNormalizedKey(record, target, offset + 1, numBytes - 1); + } + } + } + + @Override + public void writeWithKeyNormalization(T record, DataOutputView target) throws IOException { + throw new UnsupportedOperationException( + "Record serialization with leading normalized keys not supported."); + } + + @Override + public T readWithKeyDenormalization(T reuse, DataInputView source) throws IOException { + throw new UnsupportedOperationException( + "Record deserialization with leading normalized keys not supported."); + } + + @Override + public boolean invertNormalizedKey() { + return wrappedComparator.invertNormalizedKey(); + } + + @Override + public TypeComparator duplicate() { + return new NullAwareComparator(wrappedComparator.duplicate(), order); + } + + @Override + public int extractKeys(Object record, Object[] target, int index) { + if (record == null) { + for (int i = 0; i < flatFields; i++) { + target[index + i] = null; + } + return flatFields; + } else { + return wrappedComparator.extractKeys(record, target, index); + } + } + + @Override + public TypeComparator[] getFlatComparators() { + // determine the flat comparators and wrap them again in null-aware comparators + List> flatComparators = new ArrayList<>(); + if (wrappedComparator instanceof CompositeTypeComparator) { + ((CompositeTypeComparator) wrappedComparator).getFlatComparator(flatComparators); + } else { + flatComparators.add(wrappedComparator); + } + + TypeComparator[] result = new TypeComparator[flatComparators.size()]; + for (int i = 0; i < result.length; i++) { + result[i] = new NullAwareComparator<>(flatComparators.get(i), order); + } + return result; + } +} diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/NullMaskUtils.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/NullMaskUtils.java index 18820917ca628..010af7f4256e2 100644 --- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/NullMaskUtils.java +++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/NullMaskUtils.java @@ -56,6 +56,7 @@ public static void readIntoNullMask( int len, DataInputView source, boolean[] nullMask) throws IOException { + int b = 0x00; int bytePos = 0; diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/RowComparator.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/RowComparator.java index 164e3f0aec101..1113a6bb13fcd 100644 --- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/RowComparator.java +++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/RowComparator.java @@ -79,6 +79,7 @@ public RowComparator( TypeComparator[] comparators, TypeSerializer[] serializers, boolean[] orders) { + this(arity, keyPositions, makeNullAware(comparators, orders), serializers); } @@ -91,6 +92,7 @@ private RowComparator( int[] keyPositions, NullAwareComparator[] comparators, TypeSerializer[] serializers) { + this( arity, keyPositions, @@ -108,6 +110,7 @@ private RowComparator( NullAwareComparator[] comparators, TypeSerializer[] serializers, Tuple4 auxiliaryFields) { + this( arity, keyPositions, @@ -128,7 +131,10 @@ private RowComparator( NullAwareComparator[] comparators, TypeSerializer[] serializers, int[] normalizedKeyLengths, - int numLeadingNormalizableKeys, int normalizableKeyPrefixLen, boolean invertNormKey) { + int numLeadingNormalizableKeys, + int normalizableKeyPrefixLen, + boolean invertNormKey) { + this.arity = arity; this.keyPositions = keyPositions; this.comparators = comparators; @@ -331,8 +337,7 @@ public boolean isNormalizedKeyPrefixOnly(int keyBytes) { } @Override - public void putNormalizedKey( - Row record, MemorySegment target, int offset, int numBytes) { + public void putNormalizedKey(Row record, MemorySegment target, int offset, int numBytes) { int bytesLeft = numBytes; int currentOffset = offset; @@ -416,6 +421,7 @@ private Object[] instantiateDeserializationFields() { */ private static Tuple4 createAuxiliaryFields(int[] keyPositions, NullAwareComparator[] comparators) { + int[] normalizedKeyLengths = new int[keyPositions.length]; int numLeadingNormalizableKeys = 0; int normalizableKeyPrefixLen = 0; @@ -472,6 +478,7 @@ private Object[] instantiateDeserializationFields() { private static NullAwareComparator[] makeNullAware( TypeComparator[] comparators, boolean[] orders) { + checkArgument(comparators.length == orders.length); NullAwareComparator[] result = new NullAwareComparator[comparators.length]; for (int i = 0; i < comparators.length; i++) { @@ -479,217 +486,4 @@ private static NullAwareComparator[] makeNullAware( } return (NullAwareComparator[]) result; } - - - /** - * Null-aware comparator that wraps a comparator which does not support null references. - *

- * NOTE: This class assumes to be used within a composite type comparator (such - * as {@link RowComparator}) that handles serialized comparison. - */ - public static class NullAwareComparator extends TypeComparator { - private static final long serialVersionUID = 1L; - - private final TypeComparator wrappedComparator; - private final boolean order; - - // number of flat fields - private final int flatFields; - - // stores the null for reference comparison - private boolean nullReference = false; - - public NullAwareComparator(TypeComparator wrappedComparator, boolean order) { - this.wrappedComparator = wrappedComparator; - this.order = order; - this.flatFields = wrappedComparator.getFlatComparators().length; - } - - @Override - public int hash(T record) { - if (record != null) { - return wrappedComparator.hash(record); - } else { - return 0; - } - } - - @Override - public void setReference(T toCompare) { - if (toCompare == null) { - nullReference = true; - } else { - nullReference = false; - wrappedComparator.setReference(toCompare); - } - } - - @Override - public boolean equalToReference(T candidate) { - // both values are null - if (candidate == null && nullReference) { - return true; - } - // one value is null - else if (candidate == null || nullReference) { - return false; - } - // no null value - else { - return wrappedComparator.equalToReference(candidate); - } - } - - @Override - public int compareToReference(TypeComparator referencedComparator) { - NullAwareComparator otherComparator = (NullAwareComparator) referencedComparator; - boolean otherNullReference = otherComparator.nullReference; - // both values are null -> equality - if (nullReference && otherNullReference) { - return 0; - } - // first value is null -> inequality - // but order is considered - else if (nullReference) { - return order ? 1 : -1; - } - // second value is null -> inequality - // but order is considered - else if (otherNullReference) { - return order ? -1 : 1; - } - // no null values - else { - return wrappedComparator.compareToReference(otherComparator.wrappedComparator); - } - } - - @Override - public int compare(T first, T second) { - // both values are null -> equality - if (first == null && second == null) { - return 0; - } - // first value is null -> inequality - // but order is considered - else if (first == null) { - return order ? -1 : 1; - } - // second value is null -> inequality - // but order is considered - else if (second == null) { - return order ? 1 : -1; - } - // no null values - else { - return wrappedComparator.compare(first, second); - } - } - - @Override - public int compareSerialized( - DataInputView firstSource, - DataInputView secondSource) throws IOException { - - throw new UnsupportedOperationException( - "Comparator does not support null-aware serialized comparision."); - } - - @Override - public boolean supportsNormalizedKey() { - return wrappedComparator.supportsNormalizedKey(); - } - - @Override - public boolean supportsSerializationWithKeyNormalization() { - return false; - } - - @Override - public int getNormalizeKeyLen() { - int len = wrappedComparator.getNormalizeKeyLen(); - if (len == Integer.MAX_VALUE) { - return Integer.MAX_VALUE; - } else { - // add one for a null byte - return len + 1; - } - } - - @Override - public boolean isNormalizedKeyPrefixOnly(int keyBytes) { - return wrappedComparator.isNormalizedKeyPrefixOnly(keyBytes - 1); - } - - @Override - public void putNormalizedKey(T record, MemorySegment target, int offset, int numBytes) { - if (numBytes > 0) { - // write a null byte with padding - if (record == null) { - target.putBoolean(offset, false); - // write padding - for (int j = 0; j < numBytes - 1; j++) { - target.put(offset + 1 + j, (byte) 0); - } - } - // write a non-null byte with key - else { - target.putBoolean(offset, true); - // write key - wrappedComparator.putNormalizedKey(record, target, offset + 1, numBytes - 1); - } - } - } - - @Override - public void writeWithKeyNormalization(T record, DataOutputView target) throws IOException { - throw new UnsupportedOperationException( - "Record serialization with leading normalized keys not supported."); - } - - @Override - public T readWithKeyDenormalization(T reuse, DataInputView source) throws IOException { - throw new UnsupportedOperationException( - "Record deserialization with leading normalized keys not supported."); - } - - @Override - public boolean invertNormalizedKey() { - return wrappedComparator.invertNormalizedKey(); - } - - @Override - public TypeComparator duplicate() { - return new NullAwareComparator(wrappedComparator.duplicate(), order); - } - - @Override - public int extractKeys(Object record, Object[] target, int index) { - if (record == null) { - for (int i = 0; i < flatFields; i++) { - target[index + i] = null; - } - return flatFields; - } else { - return wrappedComparator.extractKeys(record, target, index); - } - } - - @Override - public TypeComparator[] getFlatComparators() { - // determine the flat comparators and wrap them again in null-aware comparators - List> flatComparators = new ArrayList<>(); - if (wrappedComparator instanceof CompositeTypeComparator) { - ((CompositeTypeComparator) wrappedComparator).getFlatComparator(flatComparators); - } else { - flatComparators.add(wrappedComparator); - } - - TypeComparator[] result = new TypeComparator[flatComparators.size()]; - for (int i = 0; i < result.length; i++) { - result[i] = new NullAwareComparator<>(flatComparators.get(i), order); - } - return result; - } - } } diff --git a/flink-core/src/main/java/org/apache/flink/types/Row.java b/flink-core/src/main/java/org/apache/flink/types/Row.java index 08f67644d0704..6825b71f0a9a8 100644 --- a/flink-core/src/main/java/org/apache/flink/types/Row.java +++ b/flink-core/src/main/java/org/apache/flink/types/Row.java @@ -19,18 +19,19 @@ import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.util.StringUtils; import java.io.Serializable; import java.util.Arrays; /** - * A Row has no limited length and contain a set of fields, which may all be different types. - * Because Row is not strongly typed, Flink's type extraction mechanism can't extract correct field - * types. So that users should manually tell Flink the type information via creating a - * {@link RowTypeInfo}. + * A Row can have arbitrary number of fields and contain a set of fields, which may all be + * different types. The fields in Row can be null. Due to Row is not strongly typed, Flink's + * type extraction mechanism can't extract correct field types. So that users should manually + * tell Flink the type information via creating a {@link RowTypeInfo}. * *

- * The fields in the Row may be accessed by position (zero-based) {@link #getField(int)}. And can + * The fields in the Row can be accessed by position (zero-based) {@link #getField(int)}. And can * set fields by {@link #setField(int, Object)}. *

* Row is in principle serializable. However, it may contain non-serializable fields, @@ -47,15 +48,15 @@ public class Row implements Serializable{ /** * Create a new Row instance. - * @param arity The number of field in the Row + * @param arity The number of fields in the Row */ public Row(int arity) { this.fields = new Object[arity]; } /** - * Get the number of field in the Row. - * @return The number of field in the Row. + * Get the number of fields in the Row. + * @return The number of fields in the Row. */ public int getArity() { return fields.length; @@ -84,7 +85,14 @@ public void setField(int pos, Object value) { @Override public String toString() { - return Arrays.deepToString(fields); + StringBuilder sb = new StringBuilder(); + for (int i = 0; i < fields.length; i++) { + if (i > 0) { + sb.append(","); + } + sb.append(StringUtils.arrayAwareToString(fields[i])); + } + return sb.toString(); } @Override @@ -98,9 +106,7 @@ public boolean equals(Object o) { Row row = (Row) o; - // Probably incorrect - comparing Object[] arrays with Arrays.equals return Arrays.equals(fields, row.fields); - } @Override diff --git a/flink-core/src/test/java/org/apache/flink/api/common/typeutils/base/RowComparatorTest.java b/flink-core/src/test/java/org/apache/flink/api/common/typeutils/base/RowComparatorTest.java index a56ff421083a8..2530c72fdce3b 100644 --- a/flink-core/src/test/java/org/apache/flink/api/common/typeutils/base/RowComparatorTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/common/typeutils/base/RowComparatorTest.java @@ -27,7 +27,7 @@ import org.apache.flink.api.java.typeutils.TupleTypeInfo; import org.apache.flink.api.java.typeutils.TypeExtractor; import org.apache.flink.types.Row; -import org.junit.Before; +import org.junit.BeforeClass; import java.io.Serializable; @@ -35,7 +35,7 @@ public class RowComparatorTest extends ComparatorTestBase { - private final RowTypeInfo typeInfo = new RowTypeInfo( + private static final RowTypeInfo typeInfo = new RowTypeInfo( BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.DOUBLE_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO, @@ -46,11 +46,11 @@ public class RowComparatorTest extends ComparatorTestBase { ), TypeExtractor.createTypeInfo(MyPojo.class)); - private MyPojo testPojo1 = new MyPojo(); - private MyPojo testPojo2 = new MyPojo(); - private MyPojo testPojo3 = new MyPojo(); + private static MyPojo testPojo1 = new MyPojo(); + private static MyPojo testPojo2 = new MyPojo(); + private static MyPojo testPojo3 = new MyPojo(); - private final Row[] data = new Row[]{ + private static final Row[] data = new Row[]{ createRow(null, null, null, null, null), createRow(0, null, null, null, null), createRow(0, 0.0, null, null, null), @@ -67,8 +67,8 @@ public class RowComparatorTest extends ComparatorTestBase { createRow(1, 1.0, "b", new Tuple3<>(2, true, (short) 3), testPojo3) }; - @Before - public void init() { + @BeforeClass + public static void init() { // TODO we cannot test null here as PojoComparator has no support for null keys testPojo1.name = ""; testPojo2.name = "Test1"; diff --git a/flink-core/src/test/java/org/apache/flink/api/common/typeutils/base/RowComparatorWithManyFieldsTests.java b/flink-core/src/test/java/org/apache/flink/api/common/typeutils/base/RowComparatorWithManyFieldsTests.java index ffaf94303a25a..6bc3165b8c077 100644 --- a/flink-core/src/test/java/org/apache/flink/api/common/typeutils/base/RowComparatorWithManyFieldsTests.java +++ b/flink-core/src/test/java/org/apache/flink/api/common/typeutils/base/RowComparatorWithManyFieldsTests.java @@ -26,7 +26,7 @@ import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.api.java.typeutils.runtime.RowComparator; import org.apache.flink.types.Row; -import org.junit.Before; +import org.junit.BeforeClass; import static org.apache.flink.util.Preconditions.checkArgument; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -37,17 +37,17 @@ */ public class RowComparatorWithManyFieldsTests extends ComparatorTestBase { - private final int numberOfFields = 10; - private RowTypeInfo typeInfo; - private final Row[] data = new Row[]{ + private static final int numberOfFields = 10; + private static RowTypeInfo typeInfo; + private static final Row[] data = new Row[]{ createRow(null, "b0", "c0", "d0", "e0", "f0", "g0", "h0", "i0", "j0"), createRow("a1", "b1", "c1", "d1", "e1", "f1", "g1", "h1", "i1", "j1"), createRow("a2", "b2", "c2", "d2", "e2", "f2", "g2", "h2", "i2", "j2"), createRow("a3", "b3", "c3", "d3", "e3", "f3", "g3", "h3", "i3", "j3") }; - @Before - public void setUp() throws Exception { + @BeforeClass + public static void setUp() throws Exception { TypeInformation[] fieldTypes = new TypeInformation[numberOfFields]; for (int i = 0; i < numberOfFields; i++) { fieldTypes[i] = BasicTypeInfo.STRING_TYPE_INFO; @@ -91,7 +91,7 @@ protected boolean supportsNullKeys() { return true; } - private Row createRow(Object... values) { + private static Row createRow(Object... values) { checkNotNull(values); checkArgument(values.length == numberOfFields); Row row = new Row(numberOfFields); diff --git a/flink-core/src/test/java/org/apache/flink/types/RowTest.java b/flink-core/src/test/java/org/apache/flink/types/RowTest.java new file mode 100644 index 0000000000000..35ba32dd8c5fb --- /dev/null +++ b/flink-core/src/test/java/org/apache/flink/types/RowTest.java @@ -0,0 +1,37 @@ +/* + * 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.api.java.tuple.Tuple2; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; + +public class RowTest { + @Test + public void testRowToString() { + Row row = new Row(5); + row.setField(0, 1); + row.setField(1, "hello"); + row.setField(2, null); + row.setField(3, new Tuple2<>(2, "hi")); + row.setField(4, "hello world"); + + assertEquals("1,hello,null,(2,hi),hello world", row.toString()); + } +} From a88a8f4478886c529db5b3eb938ea562ef18aa9b Mon Sep 17 00:00:00 2001 From: Jark Wu Date: Tue, 13 Dec 2016 13:36:06 +0800 Subject: [PATCH 5/7] remove unused ArrayList import --- .../apache/flink/api/java/typeutils/runtime/RowComparator.java | 1 - 1 file changed, 1 deletion(-) diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/RowComparator.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/RowComparator.java index 1113a6bb13fcd..d6c5195a11541 100644 --- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/RowComparator.java +++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/RowComparator.java @@ -28,7 +28,6 @@ import org.apache.flink.types.Row; import java.io.IOException; -import java.util.ArrayList; import java.util.Collections; import java.util.List; From b94b5a765fe4bfbc09e1769ce5f297fae3fc0628 Mon Sep 17 00:00:00 2001 From: tonycox Date: Fri, 9 Dec 2016 21:41:36 +0400 Subject: [PATCH 6/7] [FLINK-5188] Create analog of RowCsvInputFormat in java and adjust Row and RowTypeInfo imports --- .../kafka/Kafka010JsonTableSource.java | 2 +- .../connectors/kafka/Kafka010TableSource.java | 2 +- .../kafka/Kafka08JsonTableSink.java | 2 +- .../kafka/Kafka08JsonTableSource.java | 2 +- .../connectors/kafka/Kafka08TableSource.java | 2 +- .../kafka/Kafka08JsonTableSinkTest.java | 2 +- .../kafka/Kafka08JsonTableSourceTest.java | 2 +- .../kafka/Kafka09JsonTableSink.java | 2 +- .../kafka/Kafka09JsonTableSource.java | 2 +- .../connectors/kafka/Kafka09TableSource.java | 2 +- .../kafka/Kafka09JsonTableSinkTest.java | 2 +- .../kafka/Kafka09JsonTableSourceTest.java | 2 +- .../connectors/kafka/KafkaJsonTableSink.java | 2 +- .../connectors/kafka/KafkaTableSink.java | 4 +- .../connectors/kafka/KafkaTableSource.java | 4 +- .../JsonRowDeserializationSchema.java | 4 +- .../JsonRowSerializationSchema.java | 8 +- .../JsonRowDeserializationSchemaTest.java | 14 +- .../kafka/JsonRowSerializationSchemaTest.java | 8 +- .../kafka/KafkaTableSinkTestBase.java | 4 +- .../kafka/KafkaTableSourceTestBase.java | 2 +- .../api/java/io/jdbc/JDBCInputFormat.java | 6 +- .../api/java/io/jdbc/JDBCOutputFormat.java | 44 +- .../flink/api/java/io/jdbc/JDBCFullTest.java | 2 +- .../api/java/io/jdbc/JDBCInputFormatTest.java | 38 +- .../java/io/jdbc/JDBCOutputFormatTest.java | 24 +- .../flink/api/java/io/jdbc/JDBCTestBase.java | 2 +- .../flink/api/java/io/RowCsvInputFormat.java | 173 ++++ .../api/java/io/RowCsvInputFormatTest.java | 879 ++++++++++++++++++ .../java/table/BatchTableEnvironment.scala | 4 +- .../java/table/StreamTableEnvironment.scala | 4 +- .../scala/table/BatchTableEnvironment.scala | 2 +- .../scala/table/StreamTableEnvironment.scala | 2 +- .../flink/api/scala/table/package.scala | 3 +- .../api/table/BatchTableEnvironment.scala | 6 +- .../api/table/StreamTableEnvironment.scala | 4 +- .../flink/api/table/TableEnvironment.scala | 7 +- .../api/table/codegen/CodeGenUtils.scala | 3 +- .../api/table/codegen/CodeGenerator.scala | 5 +- .../api/table/codegen/ExpressionReducer.scala | 10 +- .../plan/nodes/dataset/DataSetAggregate.scala | 8 +- .../datastream/DataStreamAggregate.scala | 8 +- .../DataSetAggregateWithNullValuesRule.scala | 1 + .../table/plan/schema/TableSourceTable.scala | 6 +- .../table/runtime/aggregate/Aggregate.scala | 2 +- .../AggregateAllTimeWindowFunction.scala | 2 +- .../AggregateAllWindowFunction.scala | 2 +- .../aggregate/AggregateMapFunction.scala | 6 +- .../AggregateReduceCombineFunction.scala | 4 +- .../AggregateReduceGroupFunction.scala | 4 +- .../AggregateTimeWindowFunction.scala | 2 +- .../runtime/aggregate/AggregateUtil.scala | 7 +- .../aggregate/AggregateWindowFunction.scala | 2 +- .../runtime/aggregate/AvgAggregate.scala | 62 +- .../runtime/aggregate/CountAggregate.scala | 8 +- ...mentalAggregateAllTimeWindowFunction.scala | 2 +- ...ncrementalAggregateAllWindowFunction.scala | 4 +- .../IncrementalAggregateReduceFunction.scala | 4 +- ...crementalAggregateTimeWindowFunction.scala | 2 +- .../IncrementalAggregateWindowFunction.scala | 4 +- .../runtime/aggregate/MaxAggregate.scala | 14 +- .../runtime/aggregate/MinAggregate.scala | 14 +- .../runtime/aggregate/SumAggregate.scala | 14 +- .../TimeWindowPropertyCollector.scala | 4 +- .../table/runtime/io/RowCsvInputFormat.scala | 4 +- .../flink/api/table/sinks/CsvTableSink.scala | 12 +- .../api/table/sources/CsvTableSource.scala | 9 +- .../api/table/typeutils/RowComparator.scala | 1 + .../api/table/typeutils/TypeConverter.scala | 10 +- .../java/batch/TableEnvironmentITCase.java | 2 +- .../api/java/batch/TableSourceITCase.java | 4 +- .../flink/api/java/batch/sql/SqlITCase.java | 2 +- .../java/batch/table/AggregationsITCase.java | 2 +- .../api/java/batch/table/CalcITCase.java | 2 +- .../api/java/batch/table/CastingITCase.java | 2 +- .../api/java/batch/table/JoinITCase.java | 2 +- .../flink/api/java/stream/sql/SqlITCase.java | 2 +- .../scala/batch/TableEnvironmentITCase.scala | 3 +- .../api/scala/batch/TableSourceITCase.scala | 9 +- .../scala/batch/sql/AggregationsITCase.scala | 3 +- .../api/scala/batch/sql/CalcITCase.scala | 3 +- .../api/scala/batch/sql/JoinITCase.scala | 3 +- .../scala/batch/sql/SetOperatorsITCase.scala | 3 +- .../api/scala/batch/sql/SortITCase.scala | 23 +- .../scala/batch/sql/TableWithSQLITCase.scala | 3 +- .../batch/table/AggregationsITCase.scala | 3 +- .../api/scala/batch/table/CalcITCase.scala | 3 +- .../api/scala/batch/table/JoinITCase.scala | 3 +- .../batch/table/SetOperatorsITCase.scala | 3 +- .../api/scala/batch/table/SortITCase.scala | 33 +- .../table/UserDefinedTableFunctionTest.scala | 7 +- .../api/scala/stream/TableSourceITCase.scala | 7 +- .../api/scala/stream/sql/SqlITCase.scala | 3 +- .../stream/table/AggregationsITCase.scala | 3 +- .../api/scala/stream/table/CalcITCase.scala | 3 +- .../api/scala/stream/table/UnionITCase.scala | 3 +- .../table/UserDefinedTableFunctionTest.scala | 8 +- .../api/scala/stream/utils/StreamITCase.scala | 2 +- .../api/table/expressions/ArrayTypeTest.scala | 9 +- .../expressions/CompositeAccessTest.scala | 9 +- .../table/expressions/DecimalTypeTest.scala | 9 +- .../expressions/NonDeterministicTests.scala | 6 +- .../expressions/ScalarFunctionsTest.scala | 9 +- .../expressions/ScalarOperatorsTest.scala | 9 +- .../table/expressions/SqlExpressionTest.scala | 6 +- .../table/expressions/TemporalTypesTest.scala | 9 +- .../UserDefinedScalarFunctionTest.scala | 9 +- .../utils/ExpressionTestBase.scala | 8 +- .../runtime/aggregate/AggregateTestBase.scala | 2 +- .../dataset/DataSetCorrelateITCase.scala | 3 +- .../DataStreamCorrelateITCase.scala | 3 +- .../runtime/io/RowCsvInputFormatTest.scala | 275 +++--- .../table/typeutils/RowComparatorTest.scala | 16 +- .../RowComparatorWithManyFieldsTest.scala | 13 +- .../table/typeutils/RowSerializerTest.scala | 25 +- .../utils/UserDefinedTableFunctions.scala | 8 +- 116 files changed, 1611 insertions(+), 480 deletions(-) create mode 100644 flink-java/src/main/java/org/apache/flink/api/java/io/RowCsvInputFormat.java create mode 100644 flink-java/src/test/java/org/apache/flink/api/java/io/RowCsvInputFormatTest.java diff --git a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSource.java b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSource.java index ddf1ad3b8166a..920d71846dd19 100644 --- a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSource.java +++ b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSource.java @@ -19,7 +19,7 @@ package org.apache.flink.streaming.connectors.kafka; import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.table.Row; +import org.apache.flink.types.Row; import org.apache.flink.api.table.sources.StreamTableSource; import org.apache.flink.streaming.util.serialization.DeserializationSchema; diff --git a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010TableSource.java b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010TableSource.java index 732440b11a45f..127dafc5d0051 100644 --- a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010TableSource.java +++ b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010TableSource.java @@ -19,7 +19,7 @@ package org.apache.flink.streaming.connectors.kafka; import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.table.Row; +import org.apache.flink.types.Row; import org.apache.flink.api.table.sources.StreamTableSource; import org.apache.flink.streaming.util.serialization.DeserializationSchema; diff --git a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSink.java b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSink.java index b155576bce93b..839388fd79845 100644 --- a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSink.java +++ b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSink.java @@ -17,7 +17,7 @@ */ package org.apache.flink.streaming.connectors.kafka; -import org.apache.flink.api.table.Row; +import org.apache.flink.types.Row; import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner; import org.apache.flink.streaming.util.serialization.SerializationSchema; diff --git a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSource.java b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSource.java index 63bb57e121010..6c7d7273b888e 100644 --- a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSource.java +++ b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSource.java @@ -19,7 +19,7 @@ package org.apache.flink.streaming.connectors.kafka; import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.table.Row; +import org.apache.flink.types.Row; import org.apache.flink.api.table.sources.StreamTableSource; import org.apache.flink.streaming.util.serialization.DeserializationSchema; diff --git a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08TableSource.java b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08TableSource.java index 8f51237fbb680..0e3791c615a65 100644 --- a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08TableSource.java +++ b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08TableSource.java @@ -19,7 +19,7 @@ package org.apache.flink.streaming.connectors.kafka; import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.table.Row; +import org.apache.flink.types.Row; import org.apache.flink.api.table.sources.StreamTableSource; import org.apache.flink.streaming.util.serialization.DeserializationSchema; diff --git a/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSinkTest.java b/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSinkTest.java index 6d0b1406ff20c..0ac452efb3f85 100644 --- a/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSinkTest.java +++ b/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSinkTest.java @@ -17,7 +17,7 @@ */ package org.apache.flink.streaming.connectors.kafka; -import org.apache.flink.api.table.Row; +import org.apache.flink.types.Row; import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner; import org.apache.flink.streaming.util.serialization.JsonRowSerializationSchema; import org.apache.flink.streaming.util.serialization.SerializationSchema; diff --git a/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSourceTest.java b/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSourceTest.java index a2d66ac2f6413..f9ef2ce7efb56 100644 --- a/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSourceTest.java +++ b/flink-connectors/flink-connector-kafka-0.8/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSourceTest.java @@ -20,7 +20,7 @@ import java.util.Properties; import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.table.Row; +import org.apache.flink.types.Row; import org.apache.flink.streaming.util.serialization.DeserializationSchema; import org.apache.flink.streaming.util.serialization.JsonRowDeserializationSchema; diff --git a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSink.java b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSink.java index 38ea47c237d28..edbebd08dfa61 100644 --- a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSink.java +++ b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSink.java @@ -17,7 +17,7 @@ */ package org.apache.flink.streaming.connectors.kafka; -import org.apache.flink.api.table.Row; +import org.apache.flink.types.Row; import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner; import org.apache.flink.streaming.util.serialization.SerializationSchema; diff --git a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSource.java b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSource.java index 975ef5800a07f..dfcba5fb252fc 100644 --- a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSource.java +++ b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSource.java @@ -19,7 +19,7 @@ package org.apache.flink.streaming.connectors.kafka; import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.table.Row; +import org.apache.flink.types.Row; import org.apache.flink.api.table.sources.StreamTableSource; import org.apache.flink.streaming.util.serialization.DeserializationSchema; diff --git a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09TableSource.java b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09TableSource.java index 03b504024645f..f42300304f55f 100644 --- a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09TableSource.java +++ b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09TableSource.java @@ -19,7 +19,7 @@ package org.apache.flink.streaming.connectors.kafka; import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.table.Row; +import org.apache.flink.types.Row; import org.apache.flink.api.table.sources.StreamTableSource; import org.apache.flink.streaming.util.serialization.DeserializationSchema; diff --git a/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSinkTest.java b/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSinkTest.java index 45f70acf8c6fb..df84a0fd6ece8 100644 --- a/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSinkTest.java +++ b/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSinkTest.java @@ -17,7 +17,7 @@ */ package org.apache.flink.streaming.connectors.kafka; -import org.apache.flink.api.table.Row; +import org.apache.flink.types.Row; import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner; import org.apache.flink.streaming.util.serialization.JsonRowSerializationSchema; import org.apache.flink.streaming.util.serialization.SerializationSchema; diff --git a/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSourceTest.java b/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSourceTest.java index 4a75f50740c94..10b9acc19dc2d 100644 --- a/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSourceTest.java +++ b/flink-connectors/flink-connector-kafka-0.9/src/test/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSourceTest.java @@ -20,7 +20,7 @@ import java.util.Properties; import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.table.Row; +import org.apache.flink.types.Row; import org.apache.flink.streaming.util.serialization.DeserializationSchema; import org.apache.flink.streaming.util.serialization.JsonRowDeserializationSchema; diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSink.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSink.java index ee987838c143e..27c4de729a4a7 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSink.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSink.java @@ -17,7 +17,7 @@ */ package org.apache.flink.streaming.connectors.kafka; -import org.apache.flink.api.table.Row; +import org.apache.flink.types.Row; import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner; import org.apache.flink.streaming.util.serialization.JsonRowSerializationSchema; import org.apache.flink.streaming.util.serialization.SerializationSchema; diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSink.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSink.java index 714d9cd9085ea..6c42943d3cba1 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSink.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSink.java @@ -18,9 +18,9 @@ package org.apache.flink.streaming.connectors.kafka; import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.table.Row; +import org.apache.flink.types.Row; import org.apache.flink.api.table.sinks.StreamTableSink; -import org.apache.flink.api.table.typeutils.RowTypeInfo; +import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner; import org.apache.flink.streaming.util.serialization.SerializationSchema; diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSource.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSource.java index fd423d7432b8f..498e918574f9d 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSource.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSource.java @@ -19,9 +19,9 @@ package org.apache.flink.streaming.connectors.kafka; import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.table.Row; +import org.apache.flink.types.Row; import org.apache.flink.api.table.sources.StreamTableSource; -import org.apache.flink.api.table.typeutils.RowTypeInfo; +import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.util.serialization.DeserializationSchema; diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/JsonRowDeserializationSchema.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/JsonRowDeserializationSchema.java index 434481095cc7c..b4b3341fe75f6 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/JsonRowDeserializationSchema.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/JsonRowDeserializationSchema.java @@ -22,8 +22,8 @@ import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.typeutils.TypeExtractor; -import org.apache.flink.api.table.Row; -import org.apache.flink.api.table.typeutils.RowTypeInfo; +import org.apache.flink.types.Row; +import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.util.Preconditions; import java.io.IOException; diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/JsonRowSerializationSchema.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/JsonRowSerializationSchema.java index 077ff132642aa..1998aa6b7f568 100644 --- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/JsonRowSerializationSchema.java +++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/util/serialization/JsonRowSerializationSchema.java @@ -19,7 +19,7 @@ import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.node.ObjectNode; -import org.apache.flink.api.table.Row; +import org.apache.flink.types.Row; import org.apache.flink.util.Preconditions; @@ -49,15 +49,15 @@ public JsonRowSerializationSchema(String[] fieldNames) { @Override public byte[] serialize(Row row) { - if (row.productArity() != fieldNames.length) { + if (row.getArity() != fieldNames.length) { throw new IllegalStateException(String.format( "Number of elements in the row %s is different from number of field names: %d", row, fieldNames.length)); } ObjectNode objectNode = mapper.createObjectNode(); - for (int i = 0; i < row.productArity(); i++) { - JsonNode node = mapper.valueToTree(row.productElement(i)); + for (int i = 0; i < row.getArity(); i++) { + JsonNode node = mapper.valueToTree(row.getField(i)); objectNode.set(fieldNames[i], node); } diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/JsonRowDeserializationSchemaTest.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/JsonRowDeserializationSchemaTest.java index 68225e2ae461f..88f62f031ec2d 100644 --- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/JsonRowDeserializationSchemaTest.java +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/JsonRowDeserializationSchemaTest.java @@ -20,7 +20,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.node.ObjectNode; -import org.apache.flink.api.table.Row; +import org.apache.flink.types.Row; import org.apache.flink.streaming.util.serialization.JsonRowDeserializationSchema; import org.junit.Test; @@ -61,10 +61,10 @@ public void testDeserialization() throws Exception { Row deserialized = deserializationSchema.deserialize(serializedJson); - assertEquals(3, deserialized.productArity()); - assertEquals(id, deserialized.productElement(0)); - assertEquals(name, deserialized.productElement(1)); - assertArrayEquals(bytes, (byte[]) deserialized.productElement(2)); + assertEquals(3, deserialized.getArity()); + assertEquals(id, deserialized.getField(0)); + assertEquals(name, deserialized.getField(1)); + assertArrayEquals(bytes, (byte[]) deserialized.getField(2)); } /** @@ -85,8 +85,8 @@ public void testMissingNode() throws Exception { Row row = deserializationSchema.deserialize(serializedJson); - assertEquals(1, row.productArity()); - assertNull("Missing field not null", row.productElement(0)); + assertEquals(1, row.getArity()); + assertNull("Missing field not null", row.getField(0)); deserializationSchema.setFailOnMissingField(true); diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/JsonRowSerializationSchemaTest.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/JsonRowSerializationSchemaTest.java index 92af15de2336c..78dedf4f9ed78 100644 --- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/JsonRowSerializationSchemaTest.java +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/JsonRowSerializationSchemaTest.java @@ -16,7 +16,7 @@ */ package org.apache.flink.streaming.connectors.kafka; -import org.apache.flink.api.table.Row; +import org.apache.flink.types.Row; import org.apache.flink.streaming.util.serialization.JsonRowDeserializationSchema; import org.apache.flink.streaming.util.serialization.JsonRowSerializationSchema; import org.junit.Test; @@ -88,10 +88,10 @@ private Row serializeAndDeserialize(String[] fieldNames, Class[] fieldTypes, Row private void assertEqualRows(Row expectedRow, Row resultRow) { assertEquals("Deserialized row should have expected number of fields", - expectedRow.productArity(), resultRow.productArity()); - for (int i = 0; i < expectedRow.productArity(); i++) { + expectedRow.getArity(), resultRow.getArity()); + for (int i = 0; i < expectedRow.getArity(); i++) { assertEquals(String.format("Field number %d should be as in the original row", i), - expectedRow.productElement(i), resultRow.productElement(i)); + expectedRow.getField(i), resultRow.getField(i)); } } diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSinkTestBase.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSinkTestBase.java index ae0af527ebb78..cc1c16601633a 100644 --- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSinkTestBase.java +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSinkTestBase.java @@ -18,8 +18,8 @@ package org.apache.flink.streaming.connectors.kafka; import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.table.Row; -import org.apache.flink.api.table.typeutils.RowTypeInfo; +import org.apache.flink.types.Row; +import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.connectors.kafka.internals.TypeUtil; import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner; diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceTestBase.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceTestBase.java index 2a281e8e40939..ad5199362dc6e 100644 --- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceTestBase.java +++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSourceTestBase.java @@ -21,7 +21,7 @@ import java.util.Properties; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.table.Row; +import org.apache.flink.types.Row; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.util.serialization.DeserializationSchema; import org.junit.Test; diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCInputFormat.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCInputFormat.java index b4246f5a9c7bb..3153f961b391e 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCInputFormat.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCInputFormat.java @@ -37,8 +37,8 @@ import org.apache.flink.api.common.io.statistics.BaseStatistics; import org.apache.flink.api.java.io.jdbc.split.ParameterValuesProvider; import org.apache.flink.api.java.typeutils.ResultTypeQueryable; -import org.apache.flink.api.table.Row; -import org.apache.flink.api.table.typeutils.RowTypeInfo; +import org.apache.flink.types.Row; +import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.io.GenericInputSplit; import org.apache.flink.core.io.InputSplit; @@ -276,7 +276,7 @@ public Row nextRecord(Row row) throws IOException { if (!hasNext) { return null; } - for (int pos = 0; pos < row.productArity(); pos++) { + for (int pos = 0; pos < row.getArity(); pos++) { row.setField(pos, resultSet.getObject(pos + 1)); } //update hasNext after we've read the record diff --git a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCOutputFormat.java b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCOutputFormat.java index da4b1ade75005..c5585e2841add 100644 --- a/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCOutputFormat.java +++ b/flink-connectors/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCOutputFormat.java @@ -26,7 +26,7 @@ import org.apache.flink.api.common.io.RichOutputFormat; import org.apache.flink.api.java.tuple.Tuple; -import org.apache.flink.api.table.Row; +import org.apache.flink.types.Row; import org.apache.flink.configuration.Configuration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -108,22 +108,22 @@ private void establishConnection() throws SQLException, ClassNotFoundException { @Override public void writeRecord(Row row) throws IOException { - if (typesArray != null && typesArray.length > 0 && typesArray.length != row.productArity()) { + if (typesArray != null && typesArray.length > 0 && typesArray.length != row.getArity()) { LOG.warn("Column SQL types array doesn't match arity of passed Row! Check the passed array..."); } try { if (typesArray == null ) { // no types provided - for (int index = 0; index < row.productArity(); index++) { - LOG.warn("Unknown column type for column %s. Best effort approach to set its value: %s.", index + 1, row.productElement(index)); - upload.setObject(index + 1, row.productElement(index)); + for (int index = 0; index < row.getArity(); index++) { + LOG.warn("Unknown column type for column %s. Best effort approach to set its value: %s.", index + 1, row.getField(index)); + upload.setObject(index + 1, row.getField(index)); } } else { // types provided - for (int index = 0; index < row.productArity(); index++) { + for (int index = 0; index < row.getArity(); index++) { - if (row.productElement(index) == null) { + if (row.getField(index) == null) { upload.setNull(index + 1, typesArray[index]); } else { // casting values as suggested by http://docs.oracle.com/javase/1.5.0/docs/guide/jdbc/getstart/mapping.html @@ -133,56 +133,56 @@ public void writeRecord(Row row) throws IOException { break; case java.sql.Types.BOOLEAN: case java.sql.Types.BIT: - upload.setBoolean(index + 1, (boolean) row.productElement(index)); + upload.setBoolean(index + 1, (boolean) row.getField(index)); break; case java.sql.Types.CHAR: case java.sql.Types.NCHAR: case java.sql.Types.VARCHAR: case java.sql.Types.LONGVARCHAR: case java.sql.Types.LONGNVARCHAR: - upload.setString(index + 1, (String) row.productElement(index)); + upload.setString(index + 1, (String) row.getField(index)); break; case java.sql.Types.TINYINT: - upload.setByte(index + 1, (byte) row.productElement(index)); + upload.setByte(index + 1, (byte) row.getField(index)); break; case java.sql.Types.SMALLINT: - upload.setShort(index + 1, (short) row.productElement(index)); + upload.setShort(index + 1, (short) row.getField(index)); break; case java.sql.Types.INTEGER: - upload.setInt(index + 1, (int) row.productElement(index)); + upload.setInt(index + 1, (int) row.getField(index)); break; case java.sql.Types.BIGINT: - upload.setLong(index + 1, (long) row.productElement(index)); + upload.setLong(index + 1, (long) row.getField(index)); break; case java.sql.Types.REAL: - upload.setFloat(index + 1, (float) row.productElement(index)); + upload.setFloat(index + 1, (float) row.getField(index)); break; case java.sql.Types.FLOAT: case java.sql.Types.DOUBLE: - upload.setDouble(index + 1, (double) row.productElement(index)); + upload.setDouble(index + 1, (double) row.getField(index)); break; case java.sql.Types.DECIMAL: case java.sql.Types.NUMERIC: - upload.setBigDecimal(index + 1, (java.math.BigDecimal) row.productElement(index)); + upload.setBigDecimal(index + 1, (java.math.BigDecimal) row.getField(index)); break; case java.sql.Types.DATE: - upload.setDate(index + 1, (java.sql.Date) row.productElement(index)); + upload.setDate(index + 1, (java.sql.Date) row.getField(index)); break; case java.sql.Types.TIME: - upload.setTime(index + 1, (java.sql.Time) row.productElement(index)); + upload.setTime(index + 1, (java.sql.Time) row.getField(index)); break; case java.sql.Types.TIMESTAMP: - upload.setTimestamp(index + 1, (java.sql.Timestamp) row.productElement(index)); + upload.setTimestamp(index + 1, (java.sql.Timestamp) row.getField(index)); break; case java.sql.Types.BINARY: case java.sql.Types.VARBINARY: case java.sql.Types.LONGVARBINARY: - upload.setBytes(index + 1, (byte[]) row.productElement(index)); + upload.setBytes(index + 1, (byte[]) row.getField(index)); break; default: - upload.setObject(index + 1, row.productElement(index)); + upload.setObject(index + 1, row.getField(index)); LOG.warn("Unmanaged sql type (%s) for column %s. Best effort approach to set its value: %s.", - typesArray[index], index + 1, row.productElement(index)); + typesArray[index], index + 1, row.getField(index)); // case java.sql.Types.SQLXML // case java.sql.Types.ARRAY: // case java.sql.Types.JAVA_OBJECT: diff --git a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCFullTest.java b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCFullTest.java index da9469b2f6541..88aa4fa52b7f8 100644 --- a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCFullTest.java +++ b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCFullTest.java @@ -29,7 +29,7 @@ import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.java.io.jdbc.JDBCInputFormat.JDBCInputFormatBuilder; import org.apache.flink.api.java.io.jdbc.split.NumericBetweenParametersProvider; -import org.apache.flink.api.table.Row; +import org.apache.flink.types.Row; import org.junit.Assert; import org.junit.Test; diff --git a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCInputFormatTest.java b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCInputFormatTest.java index efae076c3d2a0..b08aa3abc03d5 100644 --- a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCInputFormatTest.java +++ b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCInputFormatTest.java @@ -25,7 +25,7 @@ import org.apache.flink.api.java.io.jdbc.split.GenericParameterValuesProvider; import org.apache.flink.api.java.io.jdbc.split.NumericBetweenParametersProvider; import org.apache.flink.api.java.io.jdbc.split.ParameterValuesProvider; -import org.apache.flink.api.table.Row; +import org.apache.flink.types.Row; import org.apache.flink.core.io.InputSplit; import org.junit.After; import org.junit.Assert; @@ -116,15 +116,15 @@ public void testJDBCInputFormatWithoutParallelism() throws IOException, Instanti break; } - if(next.productElement(0)!=null) { Assert.assertEquals("Field 0 should be int", Integer.class, next.productElement(0).getClass());} - if(next.productElement(1)!=null) { Assert.assertEquals("Field 1 should be String", String.class, next.productElement(1).getClass());} - if(next.productElement(2)!=null) { Assert.assertEquals("Field 2 should be String", String.class, next.productElement(2).getClass());} - if(next.productElement(3)!=null) { Assert.assertEquals("Field 3 should be float", Double.class, next.productElement(3).getClass());} - if(next.productElement(4)!=null) { Assert.assertEquals("Field 4 should be int", Integer.class, next.productElement(4).getClass());} + if(next.getField(0)!=null) { Assert.assertEquals("Field 0 should be int", Integer.class, next.getField(0).getClass());} + if(next.getField(1)!=null) { Assert.assertEquals("Field 1 should be String", String.class, next.getField(1).getClass());} + if(next.getField(2)!=null) { Assert.assertEquals("Field 2 should be String", String.class, next.getField(2).getClass());} + if(next.getField(3)!=null) { Assert.assertEquals("Field 3 should be float", Double.class, next.getField(3).getClass());} + if(next.getField(4)!=null) { Assert.assertEquals("Field 4 should be int", Integer.class, next.getField(4).getClass());} for (int x = 0; x < 5; x++) { if(testData[recordCount][x]!=null) { - Assert.assertEquals(testData[recordCount][x], next.productElement(x)); + Assert.assertEquals(testData[recordCount][x], next.getField(x)); } } recordCount++; @@ -162,15 +162,15 @@ public void testJDBCInputFormatWithParallelismAndNumericColumnSplitting() throws if (next == null) { break; } - if(next.productElement(0)!=null) { Assert.assertEquals("Field 0 should be int", Integer.class, next.productElement(0).getClass());} - if(next.productElement(1)!=null) { Assert.assertEquals("Field 1 should be String", String.class, next.productElement(1).getClass());} - if(next.productElement(2)!=null) { Assert.assertEquals("Field 2 should be String", String.class, next.productElement(2).getClass());} - if(next.productElement(3)!=null) { Assert.assertEquals("Field 3 should be float", Double.class, next.productElement(3).getClass());} - if(next.productElement(4)!=null) { Assert.assertEquals("Field 4 should be int", Integer.class, next.productElement(4).getClass());} + if(next.getField(0)!=null) { Assert.assertEquals("Field 0 should be int", Integer.class, next.getField(0).getClass());} + if(next.getField(1)!=null) { Assert.assertEquals("Field 1 should be String", String.class, next.getField(1).getClass());} + if(next.getField(2)!=null) { Assert.assertEquals("Field 2 should be String", String.class, next.getField(2).getClass());} + if(next.getField(3)!=null) { Assert.assertEquals("Field 3 should be float", Double.class, next.getField(3).getClass());} + if(next.getField(4)!=null) { Assert.assertEquals("Field 4 should be int", Integer.class, next.getField(4).getClass());} for (int x = 0; x < 5; x++) { if(testData[recordCount][x]!=null) { - Assert.assertEquals(testData[recordCount][x], next.productElement(x)); + Assert.assertEquals(testData[recordCount][x], next.getField(x)); } } recordCount++; @@ -208,11 +208,11 @@ public void testJDBCInputFormatWithParallelismAndGenericSplitting() throws IOExc if (next == null) { break; } - if(next.productElement(0)!=null) { Assert.assertEquals("Field 0 should be int", Integer.class, next.productElement(0).getClass());} - if(next.productElement(1)!=null) { Assert.assertEquals("Field 1 should be String", String.class, next.productElement(1).getClass());} - if(next.productElement(2)!=null) { Assert.assertEquals("Field 2 should be String", String.class, next.productElement(2).getClass());} - if(next.productElement(3)!=null) { Assert.assertEquals("Field 3 should be float", Double.class, next.productElement(3).getClass());} - if(next.productElement(4)!=null) { Assert.assertEquals("Field 4 should be int", Integer.class, next.productElement(4).getClass());} + if(next.getField(0)!=null) { Assert.assertEquals("Field 0 should be int", Integer.class, next.getField(0).getClass());} + if(next.getField(1)!=null) { Assert.assertEquals("Field 1 should be String", String.class, next.getField(1).getClass());} + if(next.getField(2)!=null) { Assert.assertEquals("Field 2 should be String", String.class, next.getField(2).getClass());} + if(next.getField(3)!=null) { Assert.assertEquals("Field 3 should be float", Double.class, next.getField(3).getClass());} + if(next.getField(4)!=null) { Assert.assertEquals("Field 4 should be int", Integer.class, next.getField(4).getClass());} recordCount++; } @@ -244,4 +244,4 @@ public void testEmptyResults() throws IOException, InstantiationException, Illeg Assert.assertEquals(0, recordsCnt); } -} \ No newline at end of file +} diff --git a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCOutputFormatTest.java b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCOutputFormatTest.java index 086a84cfba20a..8de0c34b263ee 100644 --- a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCOutputFormatTest.java +++ b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCOutputFormatTest.java @@ -26,7 +26,7 @@ import java.sql.SQLException; import org.apache.flink.api.java.tuple.Tuple5; -import org.apache.flink.api.table.Row; +import org.apache.flink.types.Row; import org.junit.After; import org.junit.Assert; import org.junit.Test; @@ -137,25 +137,25 @@ public void testJDBCOutputFormat() throws IOException, InstantiationException, I for (int i = 0; i < tuple5.getArity(); i++) { row.setField(i, resultSet.getObject(i + 1)); } - if (row.productElement(0) != null) { - Assert.assertEquals("Field 0 should be int", Integer.class, row.productElement(0).getClass()); + if (row.getField(0) != null) { + Assert.assertEquals("Field 0 should be int", Integer.class, row.getField(0).getClass()); } - if (row.productElement(1) != null) { - Assert.assertEquals("Field 1 should be String", String.class, row.productElement(1).getClass()); + if (row.getField(1) != null) { + Assert.assertEquals("Field 1 should be String", String.class, row.getField(1).getClass()); } - if (row.productElement(2) != null) { - Assert.assertEquals("Field 2 should be String", String.class, row.productElement(2).getClass()); + if (row.getField(2) != null) { + Assert.assertEquals("Field 2 should be String", String.class, row.getField(2).getClass()); } - if (row.productElement(3) != null) { - Assert.assertEquals("Field 3 should be float", Double.class, row.productElement(3).getClass()); + if (row.getField(3) != null) { + Assert.assertEquals("Field 3 should be float", Double.class, row.getField(3).getClass()); } - if (row.productElement(4) != null) { - Assert.assertEquals("Field 4 should be int", Integer.class, row.productElement(4).getClass()); + if (row.getField(4) != null) { + Assert.assertEquals("Field 4 should be int", Integer.class, row.getField(4).getClass()); } for (int x = 0; x < tuple5.getArity(); x++) { if (JDBCTestBase.testData[recordCount][x] != null) { - Assert.assertEquals(JDBCTestBase.testData[recordCount][x], row.productElement(x)); + Assert.assertEquals(JDBCTestBase.testData[recordCount][x], row.getField(x)); } } diff --git a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCTestBase.java b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCTestBase.java index 69ad69350cc2c..ffcb26fa7ed8b 100644 --- a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCTestBase.java +++ b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCTestBase.java @@ -25,7 +25,7 @@ import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.table.typeutils.RowTypeInfo; +import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.junit.AfterClass; import org.junit.Assert; import org.junit.BeforeClass; diff --git a/flink-java/src/main/java/org/apache/flink/api/java/io/RowCsvInputFormat.java b/flink-java/src/main/java/org/apache/flink/api/java/io/RowCsvInputFormat.java new file mode 100644 index 0000000000000..1b5fb4a106874 --- /dev/null +++ b/flink-java/src/main/java/org/apache/flink/api/java/io/RowCsvInputFormat.java @@ -0,0 +1,173 @@ +/* + * 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.java.io; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.io.ParseException; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.core.fs.Path; +import org.apache.flink.types.Row; +import org.apache.flink.types.parser.FieldParser; + +@Internal +public class RowCsvInputFormat extends CsvInputFormat { + + private static final long serialVersionUID = 1L; + + private int arity; + private boolean[] fieldsMask; + private boolean emptyColumnAsNull; + + public RowCsvInputFormat(Path filePath, RowTypeInfo rowTypeInfo, String lineDelimiter, String fieldDelimiter, boolean[] includedFieldsMask, boolean emptyColumnAsNull) { + super(filePath); + if (rowTypeInfo.getArity() == 0) { + throw new IllegalArgumentException("Row arity must be greater than 0."); + } + this.arity = rowTypeInfo.getArity(); + if (includedFieldsMask != null) { + this.fieldsMask = includedFieldsMask; + } else { + this.fieldsMask = createDefaultMask(arity); + } + this.emptyColumnAsNull = emptyColumnAsNull; + setDelimiter(lineDelimiter); + setFieldDelimiter(fieldDelimiter); + setFieldsGeneric(fieldsMask, extractTypeClasses(rowTypeInfo)); + } + + public RowCsvInputFormat(Path filePath, RowTypeInfo rowTypeInfo, String lineDelimiter, String fieldDelimiter, int[] includedFieldsMask) { + this(filePath, rowTypeInfo, lineDelimiter, fieldDelimiter, toBoolMask(includedFieldsMask), false); + } + + public RowCsvInputFormat(Path filePath, RowTypeInfo rowTypeInfo, String lineDelimiter, String fieldDelimiter) { + this(filePath, rowTypeInfo, lineDelimiter, fieldDelimiter, null, false); + } + + public RowCsvInputFormat(Path filePath, RowTypeInfo rowTypeInfo, boolean[] includedFieldsMask) { + this(filePath, rowTypeInfo, DEFAULT_LINE_DELIMITER, DEFAULT_FIELD_DELIMITER, includedFieldsMask, false); + } + + public RowCsvInputFormat(Path filePath, RowTypeInfo rowTypeInfo, int[] includedFieldsMask) { + this(filePath, rowTypeInfo, DEFAULT_LINE_DELIMITER, DEFAULT_FIELD_DELIMITER, includedFieldsMask); + } + + public RowCsvInputFormat(Path filePath, RowTypeInfo rowTypeInfo, boolean emptyColumnAsNull) { + this(filePath, rowTypeInfo, DEFAULT_LINE_DELIMITER, DEFAULT_FIELD_DELIMITER, null, emptyColumnAsNull); + } + + public RowCsvInputFormat(Path filePath, RowTypeInfo rowTypeInfo) { + this(filePath, rowTypeInfo, DEFAULT_LINE_DELIMITER, DEFAULT_FIELD_DELIMITER, null); + } + + private static Class[] extractTypeClasses(RowTypeInfo rowTypeInfo) { + Class[] classes = new Class[rowTypeInfo.getArity()]; + for (int i = 0; i < rowTypeInfo.getArity(); i++) { + classes[i] = rowTypeInfo.getTypeAt(i).getTypeClass(); + } + return classes; + } + + private static boolean[] toBoolMask(int[] includedFieldsMask) { + if (includedFieldsMask == null) { + return null; + } else { + return toBooleanMask(includedFieldsMask); + } + } + + @Override + protected Row fillRecord(Row reuse, Object[] parsedValues) { + Row reuseRow; + if (reuse == null) { + reuseRow = new Row(arity); + } else { + reuseRow = reuse; + } + for (int i = 0; i < parsedValues.length; i++) { + reuseRow.setField(i, parsedValues[i]); + } + return reuseRow; + } + + @Override + protected boolean parseRecord(Object[] holders, byte[] bytes, int offset, int numBytes) throws ParseException { + byte[] fieldDelimiter = this.getFieldDelimiter(); + boolean[] fieldIncluded = this.fieldIncluded; + + int startPos = offset; + int limit = offset + numBytes; + + int field = 0; + int output = 0; + while (field < fieldIncluded.length) { + + // check valid start position + if (startPos >= limit) { + if (isLenient()) { + return false; + } else { + throw new ParseException("Row too short: " + new String(bytes, offset, numBytes)); + } + } + + if (fieldIncluded[field]) { + // parse field + FieldParser parser = (FieldParser) this.getFieldParsers()[output]; + int latestValidPos = startPos; + startPos = parser.resetErrorStateAndParse( + bytes, + startPos, + limit, + fieldDelimiter, + holders[output]); + + if (!isLenient() && (parser.getErrorState() != FieldParser.ParseErrorState.NONE)) { + // the error state EMPTY_COLUMN is ignored + if (parser.getErrorState() != FieldParser.ParseErrorState.EMPTY_COLUMN) { + throw new ParseException(String.format("Parsing error for column %1$s of row '%2$s' originated by %3$s: %4$s.", + field, new String(bytes, offset, numBytes), parser.getClass().getSimpleName(), parser.getErrorState())); + } + } + holders[output] = parser.getLastResult(); + + // check parse result: + // the result is null if it is invalid + // or empty with emptyColumnAsNull enabled + if (startPos < 0 || + (emptyColumnAsNull && (parser.getErrorState().equals(FieldParser.ParseErrorState.EMPTY_COLUMN)))) { + holders[output] = null; + startPos = skipFields(bytes, latestValidPos, limit, fieldDelimiter); + } + output++; + } else { + // skip field + startPos = skipFields(bytes, startPos, limit, fieldDelimiter); + } + + // check if something went wrong + if (startPos < 0) { + throw new ParseException(String.format("Unexpected parser position for column %1$s of row '%2$s'", + field, new String(bytes, offset, numBytes))); + } + + field++; + } + return true; + } +} diff --git a/flink-java/src/test/java/org/apache/flink/api/java/io/RowCsvInputFormatTest.java b/flink-java/src/test/java/org/apache/flink/api/java/io/RowCsvInputFormatTest.java new file mode 100644 index 0000000000000..a68e81ee839a6 --- /dev/null +++ b/flink-java/src/test/java/org/apache/flink/api/java/io/RowCsvInputFormatTest.java @@ -0,0 +1,879 @@ +/* + * 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.java.io; + +import org.apache.flink.api.common.io.ParseException; +import org.apache.flink.api.common.typeinfo.BasicTypeInfo; +import org.apache.flink.api.common.typeinfo.SqlTimeTypeInfo; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.fs.FileInputSplit; +import org.apache.flink.core.fs.Path; +import org.apache.flink.types.Row; +import org.apache.flink.types.parser.FieldParser; +import org.apache.flink.types.parser.StringParser; +import org.junit.Ignore; +import org.junit.Test; + +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.io.OutputStreamWriter; +import java.nio.charset.StandardCharsets; +import java.sql.Date; +import java.sql.Time; +import java.sql.Timestamp; +import java.util.HashMap; +import java.util.Map; + +import static junit.framework.TestCase.assertEquals; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.fail; + +public class RowCsvInputFormatTest { + + private static Path PATH = new Path("an/ignored/file/"); + + // static variables for testing the removal of \r\n to \n + private static String FIRST_PART = "That is the first part"; + private static String SECOND_PART = "That is the second part"; + + @Test + public void ignoreInvalidLines() throws Exception { + String fileContent = + "#description of the data\n" + + "header1|header2|header3|\n" + + "this is|1|2.0|\n" + + "//a comment\n" + + "a test|3|4.0|\n" + + "#next|5|6.0|\n"; + + FileInputSplit split = createTempFile(fileContent); + + RowTypeInfo typeInfo = new RowTypeInfo(BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.DOUBLE_TYPE_INFO); + + RowCsvInputFormat format = new RowCsvInputFormat(PATH, typeInfo, "\n", "|"); + format.setLenient(false); + Configuration parameters = new Configuration(); + format.configure(new Configuration()); + format.open(split); + + Row result = new Row(3); + try { + result = format.nextRecord(result); + fail("Parse Exception was not thrown! (Row too short)"); + } catch (ParseException ignored) { + } // => ok + + try { + result = format.nextRecord(result); + fail("Parse Exception was not thrown! (Invalid int value)"); + } catch (ParseException ignored) { + } // => ok + + result = format.nextRecord(result); + assertNotNull(result); + assertEquals("this is", result.getField(0)); + assertEquals(1, result.getField(1)); + assertEquals(2.0, result.getField(2)); + + try { + result = format.nextRecord(result); + fail("Parse Exception was not thrown! (Row too short)"); + } catch (ParseException ignored) { + } // => ok + + result = format.nextRecord(result); + assertNotNull(result); + assertEquals("a test", result.getField(0)); + assertEquals(3, result.getField(1)); + assertEquals(4.0, result.getField(2)); + + result = format.nextRecord(result); + assertNotNull(result); + assertEquals("#next", result.getField(0)); + assertEquals(5, result.getField(1)); + assertEquals(6.0, result.getField(2)); + + result = format.nextRecord(result); + assertNull(result); + + // re-open with lenient = true + format.setLenient(true); + format.configure(parameters); + format.open(split); + + result = new Row(3); + + result = format.nextRecord(result); + assertNotNull(result); + assertEquals("header1", result.getField(0)); + assertNull(result.getField(1)); + assertNull(result.getField(2)); + + result = format.nextRecord(result); + assertNotNull(result); + assertEquals("this is", result.getField(0)); + assertEquals(1, result.getField(1)); + assertEquals(2.0, result.getField(2)); + + result = format.nextRecord(result); + assertNotNull(result); + assertEquals("a test", result.getField(0)); + assertEquals(3, result.getField(1)); + assertEquals(4.0, result.getField(2)); + + result = format.nextRecord(result); + assertNotNull(result); + assertEquals("#next", result.getField(0)); + assertEquals(5, result.getField(1)); + assertEquals(6.0, result.getField(2)); + result = format.nextRecord(result); + assertNull(result); + } + + @Test + public void ignoreSingleCharPrefixComments() throws Exception { + String fileContent = + "#description of the data\n" + + "#successive commented line\n" + + "this is|1|2.0|\n" + + "a test|3|4.0|\n" + + "#next|5|6.0|\n"; + + FileInputSplit split = createTempFile(fileContent); + + RowTypeInfo typeInfo = new RowTypeInfo( + BasicTypeInfo.STRING_TYPE_INFO, + BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.DOUBLE_TYPE_INFO); + + RowCsvInputFormat format = new RowCsvInputFormat(PATH, typeInfo, "\n", "|"); + format.setCommentPrefix("#"); + format.configure(new Configuration()); + format.open(split); + + Row result = new Row(3); + + result = format.nextRecord(result); + assertNotNull(result); + assertEquals("this is", result.getField(0)); + assertEquals(1, result.getField(1)); + assertEquals(2.0, result.getField(2)); + + result = format.nextRecord(result); + assertNotNull(result); + assertEquals("a test", result.getField(0)); + assertEquals(3, result.getField(1)); + assertEquals(4.0, result.getField(2)); + + result = format.nextRecord(result); + assertNull(result); + } + + @Test + public void ignoreMultiCharPrefixComments() throws Exception { + String fileContent = + "//description of the data\n" + + "//successive commented line\n" + + "this is|1|2.0|\n" + + "a test|3|4.0|\n" + + "//next|5|6.0|\n"; + + FileInputSplit split = createTempFile(fileContent); + + RowTypeInfo typeInfo = new RowTypeInfo( + BasicTypeInfo.STRING_TYPE_INFO, + BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.DOUBLE_TYPE_INFO); + + RowCsvInputFormat format = new RowCsvInputFormat(PATH, typeInfo, "\n", "|"); + format.setCommentPrefix("//"); + format.configure(new Configuration()); + format.open(split); + + Row result = new Row(3); + + result = format.nextRecord(result); + assertNotNull(result); + assertEquals("this is", result.getField(0)); + assertEquals(1, result.getField(1)); + assertEquals(2.0, result.getField(2)); + + result = format.nextRecord(result); + assertNotNull(result); + assertEquals("a test", result.getField(0)); + assertEquals(3, result.getField(1)); + assertEquals(4.0, result.getField(2)); + result = format.nextRecord(result); + assertNull(result); + } + + @Test + public void readStringFields() throws Exception { + String fileContent = "abc|def|ghijk\nabc||hhg\n|||"; + + FileInputSplit split = createTempFile(fileContent); + + RowTypeInfo typeInfo = new RowTypeInfo( + BasicTypeInfo.STRING_TYPE_INFO, + BasicTypeInfo.STRING_TYPE_INFO, + BasicTypeInfo.STRING_TYPE_INFO); + + RowCsvInputFormat format = new RowCsvInputFormat(PATH, typeInfo, "\n", "|"); + format.configure(new Configuration()); + format.open(split); + + Row result = new Row(3); + + result = format.nextRecord(result); + assertNotNull(result); + assertEquals("abc", result.getField(0)); + assertEquals("def", result.getField(1)); + assertEquals("ghijk", result.getField(2)); + + result = format.nextRecord(result); + assertNotNull(result); + assertEquals("abc", result.getField(0)); + assertEquals("", result.getField(1)); + assertEquals("hhg", result.getField(2)); + + result = format.nextRecord(result); + assertNotNull(result); + assertEquals("", result.getField(0)); + assertEquals("", result.getField(1)); + assertEquals("", result.getField(2)); + + result = format.nextRecord(result); + assertNull(result); + assertTrue(format.reachedEnd()); + } + + @Test + public void readMixedQuotedStringFields() throws Exception { + String fileContent = "@a|b|c@|def|@ghijk@\nabc||@|hhg@\n|||"; + + FileInputSplit split = createTempFile(fileContent); + + RowTypeInfo typeInfo = new RowTypeInfo( + BasicTypeInfo.STRING_TYPE_INFO, + BasicTypeInfo.STRING_TYPE_INFO, + BasicTypeInfo.STRING_TYPE_INFO); + + RowCsvInputFormat format = new RowCsvInputFormat(PATH, typeInfo, "\n", "|"); + format.configure(new Configuration()); + format.enableQuotedStringParsing('@'); + format.open(split); + + Row result = new Row(3); + + result = format.nextRecord(result); + assertNotNull(result); + assertEquals("a|b|c", result.getField(0)); + assertEquals("def", result.getField(1)); + assertEquals("ghijk", result.getField(2)); + + result = format.nextRecord(result); + assertNotNull(result); + assertEquals("abc", result.getField(0)); + assertEquals("", result.getField(1)); + assertEquals("|hhg", result.getField(2)); + + result = format.nextRecord(result); + assertNotNull(result); + assertEquals("", result.getField(0)); + assertEquals("", result.getField(1)); + assertEquals("", result.getField(2)); + + result = format.nextRecord(result); + assertNull(result); + assertTrue(format.reachedEnd()); + } + + @Test + public void readStringFieldsWithTrailingDelimiters() throws Exception { + String fileContent = "abc|-def|-ghijk\nabc|-|-hhg\n|-|-|-\n"; + + FileInputSplit split = createTempFile(fileContent); + + RowTypeInfo typeInfo = new RowTypeInfo( + BasicTypeInfo.STRING_TYPE_INFO, + BasicTypeInfo.STRING_TYPE_INFO, + BasicTypeInfo.STRING_TYPE_INFO); + + RowCsvInputFormat format = new RowCsvInputFormat(PATH, typeInfo, "\n", "|"); + format.setFieldDelimiter("|-"); + format.configure(new Configuration()); + format.open(split); + + Row result = new Row(3); + + result = format.nextRecord(result); + assertNotNull(result); + assertEquals("abc", result.getField(0)); + assertEquals("def", result.getField(1)); + assertEquals("ghijk", result.getField(2)); + + result = format.nextRecord(result); + assertNotNull(result); + assertEquals("abc", result.getField(0)); + assertEquals("", result.getField(1)); + assertEquals("hhg", result.getField(2)); + + result = format.nextRecord(result); + assertNotNull(result); + assertEquals("", result.getField(0)); + assertEquals("", result.getField(1)); + assertEquals("", result.getField(2)); + + result = format.nextRecord(result); + assertNull(result); + assertTrue(format.reachedEnd()); + } + + @Test + public void testIntegerFields() throws Exception { + String fileContent = "111|222|333|444|555\n666|777|888|999|000|\n"; + + FileInputSplit split = createTempFile(fileContent); + + RowTypeInfo typeInfo = new RowTypeInfo( + BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.INT_TYPE_INFO); + + RowCsvInputFormat format = new RowCsvInputFormat(PATH, typeInfo, "\n", "|"); + + format.setFieldDelimiter("|"); + format.configure(new Configuration()); + format.open(split); + + Row result = new Row(5); + + result = format.nextRecord(result); + assertNotNull(result); + assertEquals(111, result.getField(0)); + assertEquals(222, result.getField(1)); + assertEquals(333, result.getField(2)); + assertEquals(444, result.getField(3)); + assertEquals(555, result.getField(4)); + + result = format.nextRecord(result); + assertNotNull(result); + assertEquals(666, result.getField(0)); + assertEquals(777, result.getField(1)); + assertEquals(888, result.getField(2)); + assertEquals(999, result.getField(3)); + assertEquals(0, result.getField(4)); + + result = format.nextRecord(result); + assertNull(result); + assertTrue(format.reachedEnd()); + } + + @Test + public void testEmptyFields() throws Exception { + String fileContent = + ",,,,,,,,\n" + + ",,,,,,,,\n" + + ",,,,,,,,\n" + + ",,,,,,,,\n" + + ",,,,,,,,\n" + + ",,,,,,,,\n" + + ",,,,,,,,\n" + + ",,,,,,,,\n"; + + FileInputSplit split = createTempFile(fileContent); + + RowTypeInfo typeInfo = new RowTypeInfo( + BasicTypeInfo.BOOLEAN_TYPE_INFO, + BasicTypeInfo.BYTE_TYPE_INFO, + BasicTypeInfo.DOUBLE_TYPE_INFO, + BasicTypeInfo.FLOAT_TYPE_INFO, + BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.LONG_TYPE_INFO, + BasicTypeInfo.SHORT_TYPE_INFO, + BasicTypeInfo.STRING_TYPE_INFO); + + RowCsvInputFormat format = new RowCsvInputFormat(PATH, typeInfo, true); + format.setFieldDelimiter(","); + format.configure(new Configuration()); + format.open(split); + + Row result = new Row(8); + int linesCnt = fileContent.split("\n").length; + + for (int i = 0; i < linesCnt; i++) { + result = format.nextRecord(result); + assertNull(result.getField(i)); + } + + // ensure no more rows + assertNull(format.nextRecord(result)); + assertTrue(format.reachedEnd()); + } + + @Test + public void testDoubleFields() throws Exception { + String fileContent = "11.1|22.2|33.3|44.4|55.5\n66.6|77.7|88.8|99.9|00.0|\n"; + + FileInputSplit split = createTempFile(fileContent); + + RowTypeInfo typeInfo = new RowTypeInfo( + BasicTypeInfo.DOUBLE_TYPE_INFO, + BasicTypeInfo.DOUBLE_TYPE_INFO, + BasicTypeInfo.DOUBLE_TYPE_INFO, + BasicTypeInfo.DOUBLE_TYPE_INFO, + BasicTypeInfo.DOUBLE_TYPE_INFO); + + RowCsvInputFormat format = new RowCsvInputFormat(PATH, typeInfo); + format.setFieldDelimiter("|"); + format.configure(new Configuration()); + format.open(split); + + Row result = new Row(5); + + result = format.nextRecord(result); + assertNotNull(result); + assertEquals(11.1, result.getField(0)); + assertEquals(22.2, result.getField(1)); + assertEquals(33.3, result.getField(2)); + assertEquals(44.4, result.getField(3)); + assertEquals(55.5, result.getField(4)); + + result = format.nextRecord(result); + assertNotNull(result); + assertEquals(66.6, result.getField(0)); + assertEquals(77.7, result.getField(1)); + assertEquals(88.8, result.getField(2)); + assertEquals(99.9, result.getField(3)); + assertEquals(0.0, result.getField(4)); + + result = format.nextRecord(result); + assertNull(result); + assertTrue(format.reachedEnd()); + } + + @Test + public void testReadFirstN() throws Exception { + String fileContent = "111|222|333|444|555|\n666|777|888|999|000|\n"; + + FileInputSplit split = createTempFile(fileContent); + + RowTypeInfo typeInfo = new RowTypeInfo( + BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.INT_TYPE_INFO); + + RowCsvInputFormat format = new RowCsvInputFormat(PATH, typeInfo); + format.setFieldDelimiter("|"); + format.configure(new Configuration()); + format.open(split); + + Row result = new Row(2); + + result = format.nextRecord(result); + assertNotNull(result); + assertEquals(111, result.getField(0)); + assertEquals(222, result.getField(1)); + + result = format.nextRecord(result); + assertNotNull(result); + assertEquals(666, result.getField(0)); + assertEquals(777, result.getField(1)); + + result = format.nextRecord(result); + assertNull(result); + assertTrue(format.reachedEnd()); + } + + @Test + public void testReadSparseWithNullFieldsForTypes() throws Exception { + String fileContent = "111|x|222|x|333|x|444|x|555|x|666|x|777|x|888|x|999|x|000|x|\n" + + "000|x|999|x|888|x|777|x|666|x|555|x|444|x|333|x|222|x|111|x|"; + + FileInputSplit split = createTempFile(fileContent); + + RowTypeInfo typeInfo = new RowTypeInfo( + BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.INT_TYPE_INFO); + + RowCsvInputFormat format = new RowCsvInputFormat( + PATH, + typeInfo, + new boolean[]{true, false, false, true, false, false, false, true}); + format.setFieldDelimiter("|x|"); + format.configure(new Configuration()); + format.open(split); + + Row result = new Row(3); + + result = format.nextRecord(result); + assertNotNull(result); + assertEquals(111, result.getField(0)); + assertEquals(444, result.getField(1)); + assertEquals(888, result.getField(2)); + + result = format.nextRecord(result); + assertNotNull(result); + assertEquals(0, result.getField(0)); + assertEquals(777, result.getField(1)); + assertEquals(333, result.getField(2)); + + result = format.nextRecord(result); + assertNull(result); + assertTrue(format.reachedEnd()); + } + + @Test + public void testReadSparseWithPositionSetter() throws Exception { + String fileContent = "111|222|333|444|555|666|777|888|999|000|\n" + + "000|999|888|777|666|555|444|333|222|111|"; + + FileInputSplit split = createTempFile(fileContent); + + RowTypeInfo typeInfo = new RowTypeInfo( + BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.INT_TYPE_INFO); + + RowCsvInputFormat format = new RowCsvInputFormat( + PATH, + typeInfo, + new int[]{0, 3, 7}); + format.setFieldDelimiter("|"); + format.configure(new Configuration()); + format.open(split); + + Row result = new Row(3); + result = format.nextRecord(result); + + assertNotNull(result); + assertEquals(111, result.getField(0)); + assertEquals(444, result.getField(1)); + assertEquals(888, result.getField(2)); + + result = format.nextRecord(result); + assertNotNull(result); + assertEquals(0, result.getField(0)); + assertEquals(777, result.getField(1)); + assertEquals(333, result.getField(2)); + + result = format.nextRecord(result); + assertNull(result); + assertTrue(format.reachedEnd()); + } + + @Test + public void testReadSparseWithMask() throws Exception { + String fileContent = "111&&222&&333&&444&&555&&666&&777&&888&&999&&000&&\n" + + "000&&999&&888&&777&&666&&555&&444&&333&&222&&111&&"; + + FileInputSplit split = RowCsvInputFormatTest.createTempFile(fileContent); + + RowTypeInfo typeInfo = new RowTypeInfo( + BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.INT_TYPE_INFO); + + RowCsvInputFormat format = new RowCsvInputFormat( + PATH, + typeInfo, + new boolean[]{true, false, false, true, false, false, false, true}); + format.setFieldDelimiter("&&"); + format.configure(new Configuration()); + format.open(split); + + Row result = new Row(3); + + result = format.nextRecord(result); + assertNotNull(result); + assertEquals(111, result.getField(0)); + assertEquals(444, result.getField(1)); + assertEquals(888, result.getField(2)); + + result = format.nextRecord(result); + assertNotNull(result); + assertEquals(0, result.getField(0)); + assertEquals(777, result.getField(1)); + assertEquals(333, result.getField(2)); + + result = format.nextRecord(result); + assertNull(result); + assertTrue(format.reachedEnd()); + } + + @Test + public void testParseStringErrors() throws Exception { + StringParser stringParser = new StringParser(); + + stringParser.enableQuotedStringParsing((byte) '"'); + + Map failures = new HashMap<>(); + failures.put("\"string\" trailing", FieldParser.ParseErrorState.UNQUOTED_CHARS_AFTER_QUOTED_STRING); + failures.put("\"unterminated ", FieldParser.ParseErrorState.UNTERMINATED_QUOTED_STRING); + + for (Map.Entry failure : failures.entrySet()) { + int result = stringParser.parseField( + failure.getKey().getBytes(), + 0, + failure.getKey().length(), + new byte[]{(byte) '|'}, + null); + assertEquals(-1, result); + assertEquals(failure.getValue(), stringParser.getErrorState()); + } + } + + // Test disabled because we do not support double-quote escaped quotes right now. + @Test + @Ignore + public void testParserCorrectness() throws Exception { + // RFC 4180 Compliance Test content + // Taken from http://en.wikipedia.org/wiki/Comma-separated_values#Example + String fileContent = "Year,Make,Model,Description,Price\n" + + "1997,Ford,E350,\"ac, abs, moon\",3000.00\n" + + "1999,Chevy,\"Venture \"\"Extended Edition\"\"\",\"\",4900.00\n" + + "1996,Jeep,Grand Cherokee,\"MUST SELL! air, moon roof, loaded\",4799.00\n" + + "1999,Chevy,\"Venture \"\"Extended Edition, Very Large\"\"\",,5000.00\n" + + ",,\"Venture \"\"Extended Edition\"\"\",\"\",4900.00"; + + FileInputSplit split = createTempFile(fileContent); + + RowTypeInfo typeInfo = new RowTypeInfo( + BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.STRING_TYPE_INFO, + BasicTypeInfo.STRING_TYPE_INFO, + BasicTypeInfo.STRING_TYPE_INFO, + BasicTypeInfo.DOUBLE_TYPE_INFO); + + RowCsvInputFormat format = new RowCsvInputFormat(PATH, typeInfo); + format.setSkipFirstLineAsHeader(true); + format.setFieldDelimiter(","); + format.configure(new Configuration()); + format.open(split); + + Row result = new Row(5); + Row r1 = new Row(5); + r1.setField(0, 1997); + r1.setField(1, "Ford"); + r1.setField(2, "E350"); + r1.setField(3, "ac, abs, moon"); + r1.setField(4, 3000.0); + + Row r2 = new Row(5); + r2.setField(0, 1999); + r2.setField(1, "Chevy"); + r2.setField(2, "Venture \"Extended Edition\""); + r2.setField(3, ""); + r2.setField(4, 4900.0); + + Row r3 = new Row(5); + r3.setField(0, 1996); + r3.setField(1, "Jeep"); + r3.setField(2, "Grand Cherokee"); + r3.setField(3, "MUST SELL! air, moon roof, loaded"); + r3.setField(4, 4799.0); + + Row r4 = new Row(5); + r4.setField(0, 1999); + r4.setField(1, "Chevy"); + r4.setField(2, "Venture \"Extended Edition, Very Large\""); + r4.setField(3, ""); + r4.setField(4, 5000.0); + + Row r5 = new Row(5); + r5.setField(0, 0); + r5.setField(1, ""); + r5.setField(2, "Venture \"Extended Edition\""); + r5.setField(3, ""); + r5.setField(4, 4900.0); + + Row[] expectedLines = new Row[]{r1, r2, r3, r4, r5}; + for (Row expected : expectedLines) { + result = format.nextRecord(result); + assertEquals(expected, result); + } + assertNull(format.nextRecord(result)); + assertTrue(format.reachedEnd()); + } + + @Test + public void testWindowsLineEndRemoval() throws Exception { + + // check typical use case -- linux file is correct and it is set up to linux(\n) + testRemovingTrailingCR("\n", "\n"); + + // check typical windows case -- windows file endings and file has windows file endings set up + testRemovingTrailingCR("\r\n", "\r\n"); + + // check problematic case windows file -- windows file endings(\r\n) + // but linux line endings (\n) set up + testRemovingTrailingCR("\r\n", "\n"); + + // check problematic case linux file -- linux file endings (\n) + // but windows file endings set up (\r\n) + // specific setup for windows line endings will expect \r\n because + // it has to be set up and is not standard. + } + + @Test + public void testQuotedStringParsingWithIncludeFields() throws Exception { + String fileContent = "\"20:41:52-1-3-2015\"|\"Re: Taskmanager memory error in Eclipse\"|" + + "\"Blahblah \"|\"blaaa|\"blubb\""; + File tempFile = File.createTempFile("CsvReaderQuotedString", "tmp"); + tempFile.deleteOnExit(); + tempFile.setWritable(true); + + OutputStreamWriter writer = new OutputStreamWriter(new FileOutputStream(tempFile)); + writer.write(fileContent); + writer.close(); + + RowTypeInfo typeInfo = new RowTypeInfo( + BasicTypeInfo.STRING_TYPE_INFO, + BasicTypeInfo.STRING_TYPE_INFO); + + RowCsvInputFormat inputFormat = new RowCsvInputFormat( + new Path(tempFile.toURI().toString()), + typeInfo, + new boolean[]{true, false, true}); + inputFormat.enableQuotedStringParsing('"'); + inputFormat.setFieldDelimiter("|"); + inputFormat.setDelimiter('\n'); + inputFormat.configure(new Configuration()); + + FileInputSplit[] splits = inputFormat.createInputSplits(1); + inputFormat.open(splits[0]); + + Row record = inputFormat.nextRecord(new Row(2)); + assertEquals("20:41:52-1-3-2015", record.getField(0)); + assertEquals("Blahblah ", record.getField(1)); + } + + @Test + public void testQuotedStringParsingWithEscapedQuotes() throws Exception { + String fileContent = "\"\\\"Hello\\\" World\"|\"We are\\\" young\""; + File tempFile = File.createTempFile("CsvReaderQuotedString", "tmp"); + tempFile.deleteOnExit(); + tempFile.setWritable(true); + + OutputStreamWriter writer = new OutputStreamWriter(new FileOutputStream(tempFile)); + writer.write(fileContent); + writer.close(); + + RowTypeInfo typeInfo = new RowTypeInfo( + BasicTypeInfo.STRING_TYPE_INFO, + BasicTypeInfo.STRING_TYPE_INFO); + + RowCsvInputFormat inputFormat = new RowCsvInputFormat(new Path(tempFile.toURI().toString()), typeInfo); + inputFormat.enableQuotedStringParsing('"'); + inputFormat.setFieldDelimiter("|"); + inputFormat.setDelimiter('\n'); + inputFormat.configure(new Configuration()); + + FileInputSplit[] splits = inputFormat.createInputSplits(1); + inputFormat.open(splits[0]); + + Row record = inputFormat.nextRecord(new Row(2)); + assertEquals("\\\"Hello\\\" World", record.getField(0)); + assertEquals("We are\\\" young", record.getField(1)); + } + + @Test + public void testSqlTimeFields() throws Exception { + String fileContent = "1990-10-14|02:42:25|1990-10-14 02:42:25.123|1990-1-4 2:2:5\n" + + "1990-10-14|02:42:25|1990-10-14 02:42:25.123|1990-1-4 2:2:5.3\n"; + + FileInputSplit split = createTempFile(fileContent); + + RowTypeInfo typeInfo = new RowTypeInfo( + SqlTimeTypeInfo.DATE, + SqlTimeTypeInfo.TIME, + SqlTimeTypeInfo.TIMESTAMP, + SqlTimeTypeInfo.TIMESTAMP); + + RowCsvInputFormat format = new RowCsvInputFormat(PATH, typeInfo); + format.setFieldDelimiter("|"); + format.configure(new Configuration()); + format.open(split); + + Row result = new Row(4); + + result = format.nextRecord(result); + assertNotNull(result); + assertEquals(Date.valueOf("1990-10-14"), result.getField(0)); + assertEquals(Time.valueOf("02:42:25"), result.getField(1)); + assertEquals(Timestamp.valueOf("1990-10-14 02:42:25.123"), result.getField(2)); + assertEquals(Timestamp.valueOf("1990-01-04 02:02:05"), result.getField(3)); + + result = format.nextRecord(result); + assertNotNull(result); + assertEquals(Date.valueOf("1990-10-14"), result.getField(0)); + assertEquals(Time.valueOf("02:42:25"), result.getField(1)); + assertEquals(Timestamp.valueOf("1990-10-14 02:42:25.123"), result.getField(2)); + assertEquals(Timestamp.valueOf("1990-01-04 02:02:05.3"), result.getField(3)); + + result = format.nextRecord(result); + assertNull(result); + assertTrue(format.reachedEnd()); + } + + private static FileInputSplit createTempFile(String content) throws IOException { + File tempFile = File.createTempFile("test_contents", "tmp"); + tempFile.deleteOnExit(); + OutputStreamWriter wrt = new OutputStreamWriter(new FileOutputStream(tempFile), StandardCharsets.UTF_8); + wrt.write(content); + wrt.close(); + return new FileInputSplit(0, new Path(tempFile.toURI().toString()), 0, tempFile.length(), new String[]{"localhost"}); + } + + private static void testRemovingTrailingCR(String lineBreakerInFile, String lineBreakerSetup) throws IOException { + String fileContent = FIRST_PART + lineBreakerInFile + SECOND_PART + lineBreakerInFile; + + // create input file + File tempFile = File.createTempFile("CsvInputFormatTest", "tmp"); + tempFile.deleteOnExit(); + tempFile.setWritable(true); + + OutputStreamWriter wrt = new OutputStreamWriter(new FileOutputStream(tempFile)); + wrt.write(fileContent); + wrt.close(); + + RowTypeInfo typeInfo = new RowTypeInfo(BasicTypeInfo.STRING_TYPE_INFO); + + RowCsvInputFormat inputFormat = new RowCsvInputFormat(new Path(tempFile.toURI().toString()), typeInfo); + inputFormat.configure(new Configuration()); + inputFormat.setDelimiter(lineBreakerSetup); + + FileInputSplit[] splits = inputFormat.createInputSplits(1); + inputFormat.open(splits[0]); + + Row result = inputFormat.nextRecord(new Row(1)); + assertNotNull("Expecting to not return null", result); + assertEquals(FIRST_PART, result.getField(0)); + + result = inputFormat.nextRecord(result); + assertNotNull("Expecting to not return null", result); + assertEquals(SECOND_PART, result.getField(0)); + } +} diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/java/table/BatchTableEnvironment.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/java/table/BatchTableEnvironment.scala index 3517338bec11c..0f748c593c78f 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/java/table/BatchTableEnvironment.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/java/table/BatchTableEnvironment.scala @@ -133,7 +133,7 @@ class BatchTableEnvironment( * Converts the given [[Table]] into a [[DataSet]] of a specified type. * * The fields of the [[Table]] are mapped to [[DataSet]] fields as follows: - * - [[org.apache.flink.api.table.Row]] and [[org.apache.flink.api.java.tuple.Tuple]] + * - [[org.apache.flink.types.Row]] and [[org.apache.flink.api.java.tuple.Tuple]] * types: Fields are mapped by position, field types must match. * - POJO [[DataSet]] types: Fields are mapped by field name, field types must match. * @@ -150,7 +150,7 @@ class BatchTableEnvironment( * Converts the given [[Table]] into a [[DataSet]] of a specified type. * * The fields of the [[Table]] are mapped to [[DataSet]] fields as follows: - * - [[org.apache.flink.api.table.Row]] and [[org.apache.flink.api.java.tuple.Tuple]] + * - [[org.apache.flink.types.Row]] and [[org.apache.flink.api.java.tuple.Tuple]] * types: Fields are mapped by position, field types must match. * - POJO [[DataSet]] types: Fields are mapped by field name, field types must match. * diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/java/table/StreamTableEnvironment.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/java/table/StreamTableEnvironment.scala index 83293e3d99c1b..3218ceda65293 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/java/table/StreamTableEnvironment.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/java/table/StreamTableEnvironment.scala @@ -135,7 +135,7 @@ class StreamTableEnvironment( * Converts the given [[Table]] into a [[DataStream]] of a specified type. * * The fields of the [[Table]] are mapped to [[DataStream]] fields as follows: - * - [[org.apache.flink.api.table.Row]] and [[org.apache.flink.api.java.tuple.Tuple]] + * - [[org.apache.flink.types.Row]] and [[org.apache.flink.api.java.tuple.Tuple]] * types: Fields are mapped by position, field types must match. * - POJO [[DataStream]] types: Fields are mapped by field name, field types must match. * @@ -152,7 +152,7 @@ class StreamTableEnvironment( * Converts the given [[Table]] into a [[DataStream]] of a specified type. * * The fields of the [[Table]] are mapped to [[DataStream]] fields as follows: - * - [[org.apache.flink.api.table.Row]] and [[org.apache.flink.api.java.tuple.Tuple]] + * - [[org.apache.flink.types.Row]] and [[org.apache.flink.api.java.tuple.Tuple]] * types: Fields are mapped by position, field types must match. * - POJO [[DataStream]] types: Fields are mapped by field name, field types must match. * diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/scala/table/BatchTableEnvironment.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/scala/table/BatchTableEnvironment.scala index f4bfe31c9a02e..26fe51e029d25 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/scala/table/BatchTableEnvironment.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/scala/table/BatchTableEnvironment.scala @@ -128,7 +128,7 @@ class BatchTableEnvironment( * Converts the given [[Table]] into a [[DataSet]] of a specified type. * * The fields of the [[Table]] are mapped to [[DataSet]] fields as follows: - * - [[org.apache.flink.api.table.Row]] and [[org.apache.flink.api.java.tuple.Tuple]] + * - [[org.apache.flink.types.Row]] and [[org.apache.flink.api.java.tuple.Tuple]] * types: Fields are mapped by position, field types must match. * - POJO [[DataSet]] types: Fields are mapped by field name, field types must match. * diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/scala/table/StreamTableEnvironment.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/scala/table/StreamTableEnvironment.scala index dde69d508934f..044ace81d2e0e 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/scala/table/StreamTableEnvironment.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/scala/table/StreamTableEnvironment.scala @@ -131,7 +131,7 @@ class StreamTableEnvironment( * Converts the given [[Table]] into a [[DataStream]] of a specified type. * * The fields of the [[Table]] are mapped to [[DataStream]] fields as follows: - * - [[org.apache.flink.api.table.Row]] and [[org.apache.flink.api.java.tuple.Tuple]] + * - [[org.apache.flink.types.Row]] and [[org.apache.flink.api.java.tuple.Tuple]] * types: Fields are mapped by position, field types must match. * - POJO [[DataStream]] types: Fields are mapped by field name, field types must match. * diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/scala/table/package.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/scala/table/package.scala index 3bce5cf29d510..1e8bf392459c0 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/scala/table/package.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/scala/table/package.scala @@ -18,7 +18,8 @@ package org.apache.flink.api.scala import org.apache.flink.api.common.typeutils.CompositeType -import org.apache.flink.api.table.{Row, Table} +import org.apache.flink.api.table.Table +import org.apache.flink.types.Row import scala.language.implicitConversions import org.apache.flink.streaming.api.scala._ diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/BatchTableEnvironment.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/BatchTableEnvironment.scala index 918b01b66db05..06afda0d85dc3 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/BatchTableEnvironment.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/BatchTableEnvironment.scala @@ -168,8 +168,10 @@ abstract class BatchTableEnvironment( private[flink] def explain(table: Table, extended: Boolean): String = { val ast = table.getRelNode val optimizedPlan = optimize(ast) - val dataSet = translate[Row](optimizedPlan)(TypeExtractor.createTypeInfo(classOf[Row])) - dataSet.output(new DiscardingOutputFormat[Row]) + val dataSet = translate[org.apache.flink.types.Row](optimizedPlan) ( + TypeExtractor.createTypeInfo( + classOf[org.apache.flink.types.Row])) + dataSet.output(new DiscardingOutputFormat[org.apache.flink.types.Row]) val env = dataSet.getExecutionEnvironment val jasonSqlPlan = env.getExecutionPlan val sqlPlan = PlanJsonParser.getSqlExecutionPlan(jasonSqlPlan, extended) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/StreamTableEnvironment.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/StreamTableEnvironment.scala index 8f00586d700b3..03e36103821a2 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/StreamTableEnvironment.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/StreamTableEnvironment.scala @@ -327,7 +327,9 @@ abstract class StreamTableEnvironment( def explain(table: Table): String = { val ast = table.getRelNode val optimizedPlan = optimize(ast) - val dataStream = translate[Row](optimizedPlan)(TypeExtractor.createTypeInfo(classOf[Row])) + val dataStream = translate[org.apache.flink.types.Row](optimizedPlan)( + TypeExtractor.createTypeInfo( + classOf[org.apache.flink.types.Row])) val env = dataStream.getExecutionEnvironment val jsonSqlPlan = env.getExecutionPlan diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/TableEnvironment.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/TableEnvironment.scala index b6d0e31883cab..07ea86089cb34 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/TableEnvironment.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/TableEnvironment.scala @@ -32,15 +32,15 @@ import org.apache.calcite.sql.util.ChainedSqlOperatorTable import org.apache.calcite.tools.{FrameworkConfig, Frameworks, RuleSet, RuleSets} import org.apache.flink.api.common.typeinfo.{AtomicType, TypeInformation} import org.apache.flink.api.java.table.{BatchTableEnvironment => JavaBatchTableEnv, StreamTableEnvironment => JavaStreamTableEnv} -import org.apache.flink.api.java.typeutils.{PojoTypeInfo, TupleTypeInfo} +import org.apache.flink.api.java.typeutils.{PojoTypeInfo, RowTypeInfo, TupleTypeInfo} import org.apache.flink.api.java.{ExecutionEnvironment => JavaBatchExecEnv} import org.apache.flink.api.scala.table.{BatchTableEnvironment => ScalaBatchTableEnv, StreamTableEnvironment => ScalaStreamTableEnv} import org.apache.flink.api.scala.typeutils.CaseClassTypeInfo import org.apache.flink.api.scala.{ExecutionEnvironment => ScalaBatchExecEnv} import org.apache.flink.api.table.codegen.ExpressionReducer import org.apache.flink.api.table.expressions.{Alias, Expression, UnresolvedFieldReference} -import org.apache.flink.api.table.functions.utils.UserDefinedFunctionUtils.{checkForInstantiation, checkNotSingleton, createTableSqlFunctions, createScalarSqlFunction} -import org.apache.flink.api.table.functions.{TableFunction, ScalarFunction} +import org.apache.flink.api.table.functions.utils.UserDefinedFunctionUtils.{checkForInstantiation, checkNotSingleton, createScalarSqlFunction, createTableSqlFunctions} +import org.apache.flink.api.table.functions.{ScalarFunction, TableFunction} import org.apache.flink.api.table.plan.cost.DataSetCostFactory import org.apache.flink.api.table.plan.schema.RelTable import org.apache.flink.api.table.sinks.TableSink @@ -347,6 +347,7 @@ abstract class TableEnvironment(val config: TableConfig) { case t: TupleTypeInfo[A] => t.getFieldNames case c: CaseClassTypeInfo[A] => c.getFieldNames case p: PojoTypeInfo[A] => p.getFieldNames + case r: RowTypeInfo => r.getFieldNames case tpe => throw new TableException(s"Type $tpe lacks explicit field naming") } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/CodeGenUtils.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/CodeGenUtils.scala index 4092a24c3556b..a706309ab88d7 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/CodeGenUtils.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/CodeGenUtils.scala @@ -28,7 +28,8 @@ import org.apache.flink.api.common.typeinfo.{FractionalTypeInfo, SqlTimeTypeInfo import org.apache.flink.api.common.typeutils.CompositeType import org.apache.flink.api.java.typeutils.{PojoTypeInfo, TupleTypeInfo, TypeExtractor} import org.apache.flink.api.scala.typeutils.CaseClassTypeInfo -import org.apache.flink.api.table.typeutils.{TimeIntervalTypeInfo, RowTypeInfo, TypeCheckUtils} +import org.apache.flink.api.java.typeutils.RowTypeInfo +import org.apache.flink.api.table.typeutils.{TimeIntervalTypeInfo, TypeCheckUtils} object CodeGenUtils { diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/CodeGenerator.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/CodeGenerator.scala index 7caad126acf0f..cdb375308bd64 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/CodeGenerator.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/CodeGenerator.scala @@ -37,7 +37,8 @@ import org.apache.flink.api.table.codegen.Indenter.toISC import org.apache.flink.api.table.codegen.calls.FunctionGenerator import org.apache.flink.api.table.codegen.calls.ScalarOperators._ import org.apache.flink.api.table.functions.UserDefinedFunction -import org.apache.flink.api.table.typeutils.{RowTypeInfo, TypeConverter} +import org.apache.flink.api.java.typeutils.RowTypeInfo +import org.apache.flink.api.table.typeutils.TypeConverter import org.apache.flink.api.table.typeutils.TypeCheckUtils._ import org.apache.flink.api.table.{FlinkTypeFactory, TableConfig} @@ -1139,7 +1140,7 @@ class CodeGenerator( case ProductAccessor(i) => // Object - val inputCode = s"($fieldTypeTerm) $inputTerm.productElement($i)" + val inputCode = s"($fieldTypeTerm) $inputTerm.getField($i)" generateInputFieldUnboxing(fieldType, inputCode) case ObjectPrivateFieldAccessor(field) => diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/ExpressionReducer.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/ExpressionReducer.scala index 731452f5e5718..871264e87ba7e 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/ExpressionReducer.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/ExpressionReducer.scala @@ -25,8 +25,10 @@ import org.apache.calcite.rex.{RexBuilder, RexNode} import org.apache.calcite.sql.`type`.SqlTypeName import org.apache.flink.api.common.functions.MapFunction import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, TypeInformation} -import org.apache.flink.api.table.typeutils.{RowTypeInfo, TypeConverter} -import org.apache.flink.api.table.{FlinkTypeFactory, Row, TableConfig} +import org.apache.flink.api.table.typeutils.TypeConverter +import org.apache.flink.api.java.typeutils.RowTypeInfo +import org.apache.flink.types.Row +import org.apache.flink.api.table.{FlinkTypeFactory, TableConfig} import scala.collection.JavaConverters._ @@ -69,7 +71,7 @@ class ExpressionReducer(config: TableConfig) } val literalTypes = literals.map(e => FlinkTypeFactory.toTypeInfo(e.getType)) - val resultType = new RowTypeInfo(literalTypes) + val resultType = new RowTypeInfo(literalTypes: _*) // generate MapFunction val generator = new CodeGenerator(config, false, EMPTY_ROW_INFO) @@ -105,7 +107,7 @@ class ExpressionReducer(config: TableConfig) reducedValues.add(unreduced) case _ => val literal = rexBuilder.makeLiteral( - reduced.productElement(reducedIdx), + reduced.getField(reducedIdx), unreduced.getType, true) reducedValues.add(literal) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/DataSetAggregate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/DataSetAggregate.scala index e85ade040ac5d..94513d95dd2ac 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/DataSetAggregate.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/DataSetAggregate.scala @@ -28,8 +28,10 @@ import org.apache.flink.api.java.DataSet import org.apache.flink.api.table.plan.nodes.FlinkAggregate import org.apache.flink.api.table.runtime.aggregate.AggregateUtil import org.apache.flink.api.table.runtime.aggregate.AggregateUtil.CalcitePair -import org.apache.flink.api.table.typeutils.{RowTypeInfo, TypeConverter} -import org.apache.flink.api.table.{BatchTableEnvironment, FlinkTypeFactory, Row} +import org.apache.flink.api.table.typeutils.TypeConverter +import org.apache.flink.api.java.typeutils.RowTypeInfo +import org.apache.flink.api.table.{BatchTableEnvironment, FlinkTypeFactory} +import org.apache.flink.types.Row import scala.collection.JavaConverters._ @@ -119,7 +121,7 @@ class DataSetAggregate( .map(mapFunction) .name(prepareOpName) - val rowTypeInfo = new RowTypeInfo(fieldTypes) + val rowTypeInfo = new RowTypeInfo(fieldTypes: _*) val result = { if (groupingKeys.length > 0) { diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/datastream/DataStreamAggregate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/datastream/DataStreamAggregate.scala index c7d51313f1a8e..71337738d1dc1 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/datastream/DataStreamAggregate.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/datastream/DataStreamAggregate.scala @@ -32,8 +32,10 @@ import org.apache.flink.api.table.plan.nodes.datastream.DataStreamAggregate._ import org.apache.flink.api.table.runtime.aggregate.AggregateUtil._ import org.apache.flink.api.table.runtime.aggregate.{Aggregate, _} import org.apache.flink.api.table.typeutils.TypeCheckUtils.isTimeInterval -import org.apache.flink.api.table.typeutils.{RowIntervalTypeInfo, RowTypeInfo, TimeIntervalTypeInfo, TypeConverter} -import org.apache.flink.api.table.{FlinkTypeFactory, Row, StreamTableEnvironment} +import org.apache.flink.api.table.typeutils.{RowIntervalTypeInfo, TimeIntervalTypeInfo, TypeConverter} +import org.apache.flink.api.java.typeutils.RowTypeInfo +import org.apache.flink.api.table.{FlinkTypeFactory, StreamTableEnvironment} +import org.apache.flink.types.Row import org.apache.flink.streaming.api.datastream.{AllWindowedStream, DataStream, KeyedStream, WindowedStream} import org.apache.flink.streaming.api.windowing.assigners._ import org.apache.flink.streaming.api.windowing.time.Time @@ -117,7 +119,7 @@ class DataStreamAggregate( .map(field => FlinkTypeFactory.toTypeInfo(field.getType)) .toArray - val rowTypeInfo = new RowTypeInfo(fieldTypes) + val rowTypeInfo = new RowTypeInfo(fieldTypes: _*) val aggString = aggregationToString( inputType, diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/dataSet/DataSetAggregateWithNullValuesRule.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/dataSet/DataSetAggregateWithNullValuesRule.scala index 54cb8d1e465a9..3bf3e0c64a500 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/dataSet/DataSetAggregateWithNullValuesRule.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/dataSet/DataSetAggregateWithNullValuesRule.scala @@ -25,6 +25,7 @@ import org.apache.calcite.rel.convert.ConverterRule import org.apache.calcite.rel.logical.{LogicalValues, LogicalUnion, LogicalAggregate} import org.apache.calcite.rex.RexLiteral import org.apache.flink.api.table._ +import org.apache.flink.types.Row import org.apache.flink.api.table.plan.nodes.dataset.{DataSetAggregate, DataSetConvention} /** diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/schema/TableSourceTable.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/schema/TableSourceTable.scala index a11e8c1c365e5..72be00c9bbda8 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/schema/TableSourceTable.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/schema/TableSourceTable.scala @@ -18,13 +18,13 @@ package org.apache.flink.api.table.plan.schema -import org.apache.flink.api.table.Row +import org.apache.flink.types.Row import org.apache.flink.api.table.sources.TableSource -import org.apache.flink.api.table.typeutils.RowTypeInfo +import org.apache.flink.api.java.typeutils.RowTypeInfo /** Table which defines an external table via a [[TableSource]] */ class TableSourceTable(val tableSource: TableSource[_]) extends FlinkTable[Row]( - typeInfo = new RowTypeInfo(tableSource.getFieldTypes), + typeInfo = new RowTypeInfo(tableSource.getFieldTypes: _*), fieldIndexes = 0.until(tableSource.getNumberOfFields).toArray, fieldNames = tableSource.getFieldsNames) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/Aggregate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/Aggregate.scala index 1e91711c02eb6..273aa6036f94e 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/Aggregate.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/Aggregate.scala @@ -18,7 +18,7 @@ package org.apache.flink.api.table.runtime.aggregate import org.apache.flink.api.common.typeinfo.TypeInformation -import org.apache.flink.api.table.Row +import org.apache.flink.types.Row /** * The interface for all Flink aggregate functions, which expressed in terms of initiate(), diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/AggregateAllTimeWindowFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/AggregateAllTimeWindowFunction.scala index 7ace2c5adc9d6..4c473d418ac5e 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/AggregateAllTimeWindowFunction.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/AggregateAllTimeWindowFunction.scala @@ -21,7 +21,7 @@ package org.apache.flink.api.table.runtime.aggregate import java.lang.Iterable import org.apache.flink.api.common.functions.RichGroupReduceFunction -import org.apache.flink.api.table.Row +import org.apache.flink.types.Row import org.apache.flink.configuration.Configuration import org.apache.flink.streaming.api.functions.windowing.RichAllWindowFunction import org.apache.flink.streaming.api.windowing.windows.TimeWindow diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/AggregateAllWindowFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/AggregateAllWindowFunction.scala index 4b045bec7470d..db5f47706b325 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/AggregateAllWindowFunction.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/AggregateAllWindowFunction.scala @@ -21,7 +21,7 @@ package org.apache.flink.api.table.runtime.aggregate import java.lang.Iterable import org.apache.flink.api.common.functions.RichGroupReduceFunction -import org.apache.flink.api.table.Row +import org.apache.flink.types.Row import org.apache.flink.configuration.Configuration import org.apache.flink.streaming.api.functions.windowing.RichAllWindowFunction import org.apache.flink.streaming.api.windowing.windows.Window diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/AggregateMapFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/AggregateMapFunction.scala index 7559cec3b4afd..0699bface64f8 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/AggregateMapFunction.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/AggregateMapFunction.scala @@ -20,7 +20,7 @@ package org.apache.flink.api.table.runtime.aggregate import org.apache.flink.api.common.functions.RichMapFunction import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.java.typeutils.ResultTypeQueryable -import org.apache.flink.api.table.Row +import org.apache.flink.types.Row import org.apache.flink.configuration.Configuration import org.apache.flink.util.Preconditions @@ -47,11 +47,11 @@ class AggregateMapFunction[IN, OUT]( val input = value.asInstanceOf[Row] for (i <- 0 until aggregates.length) { - val fieldValue = input.productElement(aggFields(i)) + val fieldValue = input.getField(aggFields(i)) aggregates(i).prepare(fieldValue, output) } for (i <- 0 until groupingKeys.length) { - output.setField(i, input.productElement(groupingKeys(i))) + output.setField(i, input.getField(groupingKeys(i))) } output.asInstanceOf[OUT] } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/AggregateReduceCombineFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/AggregateReduceCombineFunction.scala index ebf0ca73565ce..b2cf07e322225 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/AggregateReduceCombineFunction.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/AggregateReduceCombineFunction.scala @@ -21,7 +21,7 @@ package org.apache.flink.api.table.runtime.aggregate import java.lang.Iterable import org.apache.flink.api.common.functions.{CombineFunction, RichGroupReduceFunction} -import org.apache.flink.api.table.Row +import org.apache.flink.types.Row import org.apache.flink.configuration.Configuration import org.apache.flink.util.{Collector, Preconditions} @@ -74,7 +74,7 @@ class AggregateReduceCombineFunction( // Set group keys to aggregateBuffer. for (i <- groupKeysMapping.indices) { - aggregateBuffer.setField(i, last.productElement(i)) + aggregateBuffer.setField(i, last.getField(i)) } aggregateBuffer diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/AggregateReduceGroupFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/AggregateReduceGroupFunction.scala index 8f096cc65f812..6fe712b43d5a9 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/AggregateReduceGroupFunction.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/AggregateReduceGroupFunction.scala @@ -20,7 +20,7 @@ package org.apache.flink.api.table.runtime.aggregate import java.lang.Iterable import org.apache.flink.api.common.functions.RichGroupReduceFunction -import org.apache.flink.api.table.Row +import org.apache.flink.types.Row import org.apache.flink.configuration.Configuration import org.apache.flink.util.{Collector, Preconditions} @@ -78,7 +78,7 @@ class AggregateReduceGroupFunction( // Set group keys value to final output. groupKeysMapping.foreach { case (after, previous) => - output.setField(after, last.productElement(previous)) + output.setField(after, last.getField(previous)) } // Evaluate final aggregate value and set to output. diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/AggregateTimeWindowFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/AggregateTimeWindowFunction.scala index 9b7ea0b8ec208..ff8f6fb8ee3d1 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/AggregateTimeWindowFunction.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/AggregateTimeWindowFunction.scala @@ -22,7 +22,7 @@ import java.lang.Iterable import org.apache.flink.api.common.functions.RichGroupReduceFunction import org.apache.flink.api.java.tuple.Tuple -import org.apache.flink.api.table.Row +import org.apache.flink.types.Row import org.apache.flink.configuration.Configuration import org.apache.flink.streaming.api.functions.windowing.RichWindowFunction import org.apache.flink.streaming.api.windowing.windows.TimeWindow diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/AggregateUtil.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/AggregateUtil.scala index 4428963bb6a5e..a1810683eb170 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/AggregateUtil.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/AggregateUtil.scala @@ -31,9 +31,10 @@ import org.apache.flink.api.java.tuple.Tuple import org.apache.flink.api.table.FlinkRelBuilder.NamedWindowProperty import org.apache.flink.api.table.expressions.{WindowEnd, WindowStart} import org.apache.flink.api.table.plan.logical._ -import org.apache.flink.api.table.typeutils.RowTypeInfo +import org.apache.flink.api.java.typeutils.RowTypeInfo import org.apache.flink.api.table.typeutils.TypeCheckUtils._ -import org.apache.flink.api.table.{FlinkTypeFactory, Row, TableException} +import org.apache.flink.api.table.{FlinkTypeFactory, TableException} +import org.apache.flink.types.Row import org.apache.flink.streaming.api.functions.windowing.{AllWindowFunction, WindowFunction} import org.apache.flink.streaming.api.windowing.windows.{Window => DataStreamWindow} import scala.collection.JavaConversions._ @@ -529,7 +530,7 @@ object AggregateUtil { // concat group key types and aggregation types val allFieldTypes = groupingTypes ++: aggTypes - val partialType = new RowTypeInfo(allFieldTypes) + val partialType = new RowTypeInfo(allFieldTypes: _*) partialType } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/AggregateWindowFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/AggregateWindowFunction.scala index 6fd890de672a8..4e775494b6155 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/AggregateWindowFunction.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/AggregateWindowFunction.scala @@ -22,7 +22,7 @@ import java.lang.Iterable import org.apache.flink.api.common.functions.RichGroupReduceFunction import org.apache.flink.api.java.tuple.Tuple -import org.apache.flink.api.table.Row +import org.apache.flink.types.Row import org.apache.flink.configuration.Configuration import org.apache.flink.streaming.api.functions.windowing.RichWindowFunction import org.apache.flink.streaming.api.windowing.windows.Window diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/AvgAggregate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/AvgAggregate.scala index ce5bc818f1a64..998ae62062d53 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/AvgAggregate.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/AvgAggregate.scala @@ -19,7 +19,7 @@ package org.apache.flink.api.table.runtime.aggregate import com.google.common.math.LongMath import org.apache.flink.api.common.typeinfo.BasicTypeInfo -import org.apache.flink.api.table.Row +import org.apache.flink.types.Row import java.math.BigDecimal import java.math.BigInteger @@ -52,10 +52,10 @@ abstract class IntegralAvgAggregate[T] extends AvgAggregate[T] { } override def merge(partial: Row, buffer: Row): Unit = { - val partialSum = partial.productElement(partialSumIndex).asInstanceOf[Long] - val partialCount = partial.productElement(partialCountIndex).asInstanceOf[Long] - val bufferSum = buffer.productElement(partialSumIndex).asInstanceOf[Long] - val bufferCount = buffer.productElement(partialCountIndex).asInstanceOf[Long] + val partialSum = partial.getField(partialSumIndex).asInstanceOf[Long] + val partialCount = partial.getField(partialCountIndex).asInstanceOf[Long] + val bufferSum = buffer.getField(partialSumIndex).asInstanceOf[Long] + val bufferCount = buffer.getField(partialCountIndex).asInstanceOf[Long] buffer.setField(partialSumIndex, LongMath.checkedAdd(partialSum, bufferSum)) buffer.setField(partialCountIndex, LongMath.checkedAdd(partialCount, bufferCount)) } @@ -81,8 +81,8 @@ class ByteAvgAggregate extends IntegralAvgAggregate[Byte] { } override def doEvaluate(buffer: Row): Any = { - val bufferSum = buffer.productElement(partialSumIndex).asInstanceOf[Long] - val bufferCount = buffer.productElement(partialCountIndex).asInstanceOf[Long] + val bufferSum = buffer.getField(partialSumIndex).asInstanceOf[Long] + val bufferCount = buffer.getField(partialCountIndex).asInstanceOf[Long] if (bufferCount == 0L) { null } else { @@ -100,8 +100,8 @@ class ShortAvgAggregate extends IntegralAvgAggregate[Short] { } override def doEvaluate(buffer: Row): Any = { - val bufferSum = buffer.productElement(partialSumIndex).asInstanceOf[Long] - val bufferCount = buffer.productElement(partialCountIndex).asInstanceOf[Long] + val bufferSum = buffer.getField(partialSumIndex).asInstanceOf[Long] + val bufferCount = buffer.getField(partialCountIndex).asInstanceOf[Long] if (bufferCount == 0L) { null } else { @@ -119,8 +119,8 @@ class IntAvgAggregate extends IntegralAvgAggregate[Int] { } override def doEvaluate(buffer: Row): Any = { - val bufferSum = buffer.productElement(partialSumIndex).asInstanceOf[Long] - val bufferCount = buffer.productElement(partialCountIndex).asInstanceOf[Long] + val bufferSum = buffer.getField(partialSumIndex).asInstanceOf[Long] + val bufferCount = buffer.getField(partialCountIndex).asInstanceOf[Long] if (bufferCount == 0L) { null } else { @@ -156,17 +156,17 @@ class LongAvgAggregate extends IntegralAvgAggregate[Long] { } override def merge(partial: Row, buffer: Row): Unit = { - val partialSum = partial.productElement(partialSumIndex).asInstanceOf[BigInteger] - val partialCount = partial.productElement(partialCountIndex).asInstanceOf[Long] - val bufferSum = buffer.productElement(partialSumIndex).asInstanceOf[BigInteger] - val bufferCount = buffer.productElement(partialCountIndex).asInstanceOf[Long] + val partialSum = partial.getField(partialSumIndex).asInstanceOf[BigInteger] + val partialCount = partial.getField(partialCountIndex).asInstanceOf[Long] + val bufferSum = buffer.getField(partialSumIndex).asInstanceOf[BigInteger] + val bufferCount = buffer.getField(partialCountIndex).asInstanceOf[Long] buffer.setField(partialSumIndex, partialSum.add(bufferSum)) buffer.setField(partialCountIndex, LongMath.checkedAdd(partialCount, bufferCount)) } override def doEvaluate(buffer: Row): Any = { - val bufferSum = buffer.productElement(partialSumIndex).asInstanceOf[BigInteger] - val bufferCount = buffer.productElement(partialCountIndex).asInstanceOf[Long] + val bufferSum = buffer.getField(partialSumIndex).asInstanceOf[BigInteger] + val bufferCount = buffer.getField(partialCountIndex).asInstanceOf[Long] if (bufferCount == 0L) { null } else { @@ -192,10 +192,10 @@ abstract class FloatingAvgAggregate[T: Numeric] extends AvgAggregate[T] { } override def merge(partial: Row, buffer: Row): Unit = { - val partialSum = partial.productElement(partialSumIndex).asInstanceOf[Double] - val partialCount = partial.productElement(partialCountIndex).asInstanceOf[Long] - val bufferSum = buffer.productElement(partialSumIndex).asInstanceOf[Double] - val bufferCount = buffer.productElement(partialCountIndex).asInstanceOf[Long] + val partialSum = partial.getField(partialSumIndex).asInstanceOf[Double] + val partialCount = partial.getField(partialCountIndex).asInstanceOf[Long] + val bufferSum = buffer.getField(partialSumIndex).asInstanceOf[Double] + val bufferCount = buffer.getField(partialCountIndex).asInstanceOf[Long] buffer.setField(partialSumIndex, partialSum + bufferSum) buffer.setField(partialCountIndex, partialCount + bufferCount) @@ -224,8 +224,8 @@ class FloatAvgAggregate extends FloatingAvgAggregate[Float] { override def doEvaluate(buffer: Row): Any = { - val bufferSum = buffer.productElement(partialSumIndex).asInstanceOf[Double] - val bufferCount = buffer.productElement(partialCountIndex).asInstanceOf[Long] + val bufferSum = buffer.getField(partialSumIndex).asInstanceOf[Double] + val bufferCount = buffer.getField(partialCountIndex).asInstanceOf[Long] if (bufferCount == 0L) { null } else { @@ -243,8 +243,8 @@ class DoubleAvgAggregate extends FloatingAvgAggregate[Double] { } override def doEvaluate(buffer: Row): Any = { - val bufferSum = buffer.productElement(partialSumIndex).asInstanceOf[Double] - val bufferCount = buffer.productElement(partialCountIndex).asInstanceOf[Long] + val bufferSum = buffer.getField(partialSumIndex).asInstanceOf[Double] + val bufferCount = buffer.getField(partialCountIndex).asInstanceOf[Long] if (bufferCount == 0L) { null } else { @@ -275,18 +275,18 @@ class DecimalAvgAggregate extends AvgAggregate[BigDecimal] { } override def merge(partial: Row, buffer: Row): Unit = { - val partialSum = partial.productElement(partialSumIndex).asInstanceOf[BigDecimal] - val partialCount = partial.productElement(partialCountIndex).asInstanceOf[Long] - val bufferSum = buffer.productElement(partialSumIndex).asInstanceOf[BigDecimal] - val bufferCount = buffer.productElement(partialCountIndex).asInstanceOf[Long] + val partialSum = partial.getField(partialSumIndex).asInstanceOf[BigDecimal] + val partialCount = partial.getField(partialCountIndex).asInstanceOf[Long] + val bufferSum = buffer.getField(partialSumIndex).asInstanceOf[BigDecimal] + val bufferCount = buffer.getField(partialCountIndex).asInstanceOf[Long] buffer.setField(partialSumIndex, partialSum.add(bufferSum)) buffer.setField(partialCountIndex, LongMath.checkedAdd(partialCount, bufferCount)) } override def evaluate(buffer: Row): BigDecimal = { - val bufferCount = buffer.productElement(partialCountIndex).asInstanceOf[Long] + val bufferCount = buffer.getField(partialCountIndex).asInstanceOf[Long] if (bufferCount != 0) { - val bufferSum = buffer.productElement(partialSumIndex).asInstanceOf[BigDecimal] + val bufferSum = buffer.getField(partialSumIndex).asInstanceOf[BigDecimal] bufferSum.divide(BigDecimal.valueOf(bufferCount)) } else { null.asInstanceOf[BigDecimal] diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/CountAggregate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/CountAggregate.scala index d9f288a2f7832..4d6d20bd48393 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/CountAggregate.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/CountAggregate.scala @@ -18,7 +18,7 @@ package org.apache.flink.api.table.runtime.aggregate import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, TypeInformation} -import org.apache.flink.api.table.Row +import org.apache.flink.types.Row class CountAggregate extends Aggregate[Long] { private var countIndex: Int = _ @@ -28,13 +28,13 @@ class CountAggregate extends Aggregate[Long] { } override def merge(intermediate: Row, buffer: Row): Unit = { - val partialCount = intermediate.productElement(countIndex).asInstanceOf[Long] - val bufferCount = buffer.productElement(countIndex).asInstanceOf[Long] + val partialCount = intermediate.getField(countIndex).asInstanceOf[Long] + val bufferCount = buffer.getField(countIndex).asInstanceOf[Long] buffer.setField(countIndex, partialCount + bufferCount) } override def evaluate(buffer: Row): Long = { - buffer.productElement(countIndex).asInstanceOf[Long] + buffer.getField(countIndex).asInstanceOf[Long] } override def prepare(value: Any, intermediate: Row): Unit = { diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/IncrementalAggregateAllTimeWindowFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/IncrementalAggregateAllTimeWindowFunction.scala index 85ad8e56cfea4..48e23138c85c5 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/IncrementalAggregateAllTimeWindowFunction.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/IncrementalAggregateAllTimeWindowFunction.scala @@ -19,7 +19,7 @@ package org.apache.flink.api.table.runtime.aggregate import java.lang.Iterable -import org.apache.flink.api.table.Row +import org.apache.flink.types.Row import org.apache.flink.configuration.Configuration import org.apache.flink.streaming.api.windowing.windows.{TimeWindow, Window} import org.apache.flink.util.Collector diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/IncrementalAggregateAllWindowFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/IncrementalAggregateAllWindowFunction.scala index d3f871a5e5c81..1a85dcaf490fa 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/IncrementalAggregateAllWindowFunction.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/IncrementalAggregateAllWindowFunction.scala @@ -19,7 +19,7 @@ package org.apache.flink.api.table.runtime.aggregate import java.lang.Iterable -import org.apache.flink.api.table.Row +import org.apache.flink.types.Row import org.apache.flink.configuration.Configuration import org.apache.flink.streaming.api.functions.windowing.RichAllWindowFunction import org.apache.flink.streaming.api.windowing.windows.Window @@ -66,7 +66,7 @@ class IncrementalAggregateAllWindowFunction[W <: Window]( // Set group keys value to final output. groupKeysMapping.foreach { case (after, previous) => - output.setField(after, record.productElement(previous)) + output.setField(after, record.getField(previous)) } // Evaluate final aggregate value and set to output. aggregateMapping.foreach { diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/IncrementalAggregateReduceFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/IncrementalAggregateReduceFunction.scala index e2830da70f5f8..5c36821b7a317 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/IncrementalAggregateReduceFunction.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/IncrementalAggregateReduceFunction.scala @@ -18,7 +18,7 @@ package org.apache.flink.api.table.runtime.aggregate import org.apache.flink.api.common.functions.ReduceFunction -import org.apache.flink.api.table.Row +import org.apache.flink.types.Row import org.apache.flink.util.Preconditions /** @@ -54,7 +54,7 @@ class IncrementalAggregateReduceFunction( // copy all fields of value1 into accumulatorRow (0 until intermediateRowArity) - .foreach(i => accumulatorRow.setField(i, value1.productElement(i))) + .foreach(i => accumulatorRow.setField(i, value1.getField(i))) // merge value2 to accumulatorRow aggregates.foreach(_.merge(value2, accumulatorRow)) diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/IncrementalAggregateTimeWindowFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/IncrementalAggregateTimeWindowFunction.scala index c880f87dc1521..2513383ab5587 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/IncrementalAggregateTimeWindowFunction.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/IncrementalAggregateTimeWindowFunction.scala @@ -20,7 +20,7 @@ package org.apache.flink.api.table.runtime.aggregate import java.lang.Iterable import org.apache.flink.api.java.tuple.Tuple -import org.apache.flink.api.table.Row +import org.apache.flink.types.Row import org.apache.flink.configuration.Configuration import org.apache.flink.streaming.api.windowing.windows.TimeWindow import org.apache.flink.util.Collector diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/IncrementalAggregateWindowFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/IncrementalAggregateWindowFunction.scala index 81e6890fe2848..d0d71ee09ad21 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/IncrementalAggregateWindowFunction.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/IncrementalAggregateWindowFunction.scala @@ -20,7 +20,7 @@ package org.apache.flink.api.table.runtime.aggregate import java.lang.Iterable import org.apache.flink.api.java.tuple.Tuple -import org.apache.flink.api.table.Row +import org.apache.flink.types.Row import org.apache.flink.configuration.Configuration import org.apache.flink.streaming.api.functions.windowing.RichWindowFunction import org.apache.flink.streaming.api.windowing.windows.Window @@ -68,7 +68,7 @@ class IncrementalAggregateWindowFunction[W <: Window]( // Set group keys value to final output. groupKeysMapping.foreach { case (after, previous) => - output.setField(after, record.productElement(previous)) + output.setField(after, record.getField(previous)) } // Evaluate final aggregate value and set to output. aggregateMapping.foreach { diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/MaxAggregate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/MaxAggregate.scala index 9267527e38e12..2cb3dc75be659 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/MaxAggregate.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/MaxAggregate.scala @@ -20,7 +20,7 @@ package org.apache.flink.api.table.runtime.aggregate import java.math.BigDecimal import org.apache.flink.api.common.typeinfo.BasicTypeInfo -import org.apache.flink.api.table.Row +import org.apache.flink.types.Row abstract class MaxAggregate[T](implicit ord: Ordering[T]) extends Aggregate[T] { @@ -57,9 +57,9 @@ abstract class MaxAggregate[T](implicit ord: Ordering[T]) extends Aggregate[T] { * @param buffer */ override def merge(intermediate: Row, buffer: Row): Unit = { - val partialValue = intermediate.productElement(maxIndex).asInstanceOf[T] + val partialValue = intermediate.getField(maxIndex).asInstanceOf[T] if (partialValue != null) { - val bufferValue = buffer.productElement(maxIndex).asInstanceOf[T] + val bufferValue = buffer.getField(maxIndex).asInstanceOf[T] if (bufferValue != null) { val max : T = if (ord.compare(partialValue, bufferValue) > 0) partialValue else bufferValue buffer.setField(maxIndex, max) @@ -76,7 +76,7 @@ abstract class MaxAggregate[T](implicit ord: Ordering[T]) extends Aggregate[T] { * @return */ override def evaluate(buffer: Row): T = { - buffer.productElement(maxIndex).asInstanceOf[T] + buffer.getField(maxIndex).asInstanceOf[T] } override def supportPartial: Boolean = true @@ -147,9 +147,9 @@ class DecimalMaxAggregate extends Aggregate[BigDecimal] { } override def merge(partial: Row, buffer: Row): Unit = { - val partialValue = partial.productElement(minIndex).asInstanceOf[BigDecimal] + val partialValue = partial.getField(minIndex).asInstanceOf[BigDecimal] if (partialValue != null) { - val bufferValue = buffer.productElement(minIndex).asInstanceOf[BigDecimal] + val bufferValue = buffer.getField(minIndex).asInstanceOf[BigDecimal] if (bufferValue != null) { val min = if (partialValue.compareTo(bufferValue) > 0) partialValue else bufferValue buffer.setField(minIndex, min) @@ -160,7 +160,7 @@ class DecimalMaxAggregate extends Aggregate[BigDecimal] { } override def evaluate(buffer: Row): BigDecimal = { - buffer.productElement(minIndex).asInstanceOf[BigDecimal] + buffer.getField(minIndex).asInstanceOf[BigDecimal] } override def supportPartial: Boolean = true diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/MinAggregate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/MinAggregate.scala index 7e2ebf4010412..c1c79ece14cef 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/MinAggregate.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/MinAggregate.scala @@ -19,7 +19,7 @@ package org.apache.flink.api.table.runtime.aggregate import java.math.BigDecimal import org.apache.flink.api.common.typeinfo.BasicTypeInfo -import org.apache.flink.api.table.Row +import org.apache.flink.types.Row abstract class MinAggregate[T](implicit ord: Ordering[T]) extends Aggregate[T] { @@ -56,9 +56,9 @@ abstract class MinAggregate[T](implicit ord: Ordering[T]) extends Aggregate[T] { * @param buffer */ override def merge(partial: Row, buffer: Row): Unit = { - val partialValue = partial.productElement(minIndex).asInstanceOf[T] + val partialValue = partial.getField(minIndex).asInstanceOf[T] if (partialValue != null) { - val bufferValue = buffer.productElement(minIndex).asInstanceOf[T] + val bufferValue = buffer.getField(minIndex).asInstanceOf[T] if (bufferValue != null) { val min : T = if (ord.compare(partialValue, bufferValue) < 0) partialValue else bufferValue buffer.setField(minIndex, min) @@ -75,7 +75,7 @@ abstract class MinAggregate[T](implicit ord: Ordering[T]) extends Aggregate[T] { * @return */ override def evaluate(buffer: Row): T = { - buffer.productElement(minIndex).asInstanceOf[T] + buffer.getField(minIndex).asInstanceOf[T] } override def supportPartial: Boolean = true @@ -146,9 +146,9 @@ class DecimalMinAggregate extends Aggregate[BigDecimal] { } override def merge(partial: Row, buffer: Row): Unit = { - val partialValue = partial.productElement(minIndex).asInstanceOf[BigDecimal] + val partialValue = partial.getField(minIndex).asInstanceOf[BigDecimal] if (partialValue != null) { - val bufferValue = buffer.productElement(minIndex).asInstanceOf[BigDecimal] + val bufferValue = buffer.getField(minIndex).asInstanceOf[BigDecimal] if (bufferValue != null) { val min = if (partialValue.compareTo(bufferValue) < 0) partialValue else bufferValue buffer.setField(minIndex, min) @@ -159,7 +159,7 @@ class DecimalMinAggregate extends Aggregate[BigDecimal] { } override def evaluate(buffer: Row): BigDecimal = { - buffer.productElement(minIndex).asInstanceOf[BigDecimal] + buffer.getField(minIndex).asInstanceOf[BigDecimal] } override def supportPartial: Boolean = true diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/SumAggregate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/SumAggregate.scala index 7ff23404c9140..16f1608ecd73a 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/SumAggregate.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/SumAggregate.scala @@ -19,7 +19,7 @@ package org.apache.flink.api.table.runtime.aggregate import java.math.BigDecimal import org.apache.flink.api.common.typeinfo.BasicTypeInfo -import org.apache.flink.api.table.Row +import org.apache.flink.types.Row abstract class SumAggregate[T: Numeric] extends Aggregate[T] { @@ -32,9 +32,9 @@ abstract class SumAggregate[T: Numeric] } override def merge(partial1: Row, buffer: Row): Unit = { - val partialValue = partial1.productElement(sumIndex).asInstanceOf[T] + val partialValue = partial1.getField(sumIndex).asInstanceOf[T] if (partialValue != null) { - val bufferValue = buffer.productElement(sumIndex).asInstanceOf[T] + val bufferValue = buffer.getField(sumIndex).asInstanceOf[T] if (bufferValue != null) { buffer.setField(sumIndex, numeric.plus(partialValue, bufferValue)) } else { @@ -44,7 +44,7 @@ abstract class SumAggregate[T: Numeric] } override def evaluate(buffer: Row): T = { - buffer.productElement(sumIndex).asInstanceOf[T] + buffer.getField(sumIndex).asInstanceOf[T] } override def prepare(value: Any, partial: Row): Unit = { @@ -98,9 +98,9 @@ class DecimalSumAggregate extends Aggregate[BigDecimal] { } override def merge(partial1: Row, buffer: Row): Unit = { - val partialValue = partial1.productElement(sumIndex).asInstanceOf[BigDecimal] + val partialValue = partial1.getField(sumIndex).asInstanceOf[BigDecimal] if (partialValue != null) { - val bufferValue = buffer.productElement(sumIndex).asInstanceOf[BigDecimal] + val bufferValue = buffer.getField(sumIndex).asInstanceOf[BigDecimal] if (bufferValue != null) { buffer.setField(sumIndex, partialValue.add(bufferValue)) } else { @@ -110,7 +110,7 @@ class DecimalSumAggregate extends Aggregate[BigDecimal] { } override def evaluate(buffer: Row): BigDecimal = { - buffer.productElement(sumIndex).asInstanceOf[BigDecimal] + buffer.getField(sumIndex).asInstanceOf[BigDecimal] } override def prepare(value: Any, partial: Row): Unit = { diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/TimeWindowPropertyCollector.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/TimeWindowPropertyCollector.scala index 9f1c23b3b70a0..417c1f1aa1307 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/TimeWindowPropertyCollector.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/TimeWindowPropertyCollector.scala @@ -19,7 +19,7 @@ package org.apache.flink.api.table.runtime.aggregate import org.apache.calcite.runtime.SqlFunctions -import org.apache.flink.api.table.Row +import org.apache.flink.types.Row import org.apache.flink.streaming.api.windowing.windows.TimeWindow import org.apache.flink.util.Collector @@ -35,7 +35,7 @@ class TimeWindowPropertyCollector(windowStartOffset: Option[Int], windowEndOffse override def collect(record: Row): Unit = { - val lastFieldPos = record.productArity - 1 + val lastFieldPos = record.getArity - 1 if (windowStartOffset.isDefined) { record.setField( diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/io/RowCsvInputFormat.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/io/RowCsvInputFormat.scala index b0ab801fbf011..8ea7f7acc3ab9 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/io/RowCsvInputFormat.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/io/RowCsvInputFormat.scala @@ -22,9 +22,9 @@ import org.apache.flink.annotation.Internal import org.apache.flink.api.common.io.ParseException import org.apache.flink.api.java.io.CsvInputFormat import org.apache.flink.api.java.io.CsvInputFormat.{DEFAULT_FIELD_DELIMITER, DEFAULT_LINE_DELIMITER, createDefaultMask, toBooleanMask} -import org.apache.flink.api.table.Row +import org.apache.flink.types.Row import org.apache.flink.api.table.runtime.io.RowCsvInputFormat.extractTypeClasses -import org.apache.flink.api.table.typeutils.RowTypeInfo +import org.apache.flink.api.java.typeutils.RowTypeInfo import org.apache.flink.core.fs.Path import org.apache.flink.types.parser.FieldParser import org.apache.flink.types.parser.FieldParser.ParseErrorState diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/sinks/CsvTableSink.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/sinks/CsvTableSink.scala index 7567ba8169c37..5038d9b8b293d 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/sinks/CsvTableSink.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/sinks/CsvTableSink.scala @@ -21,8 +21,8 @@ package org.apache.flink.api.table.sinks import org.apache.flink.api.common.functions.MapFunction import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.java.DataSet -import org.apache.flink.api.table.Row -import org.apache.flink.api.table.typeutils.RowTypeInfo +import org.apache.flink.types.Row +import org.apache.flink.api.java.typeutils.RowTypeInfo import org.apache.flink.streaming.api.datastream.DataStream /** @@ -53,7 +53,7 @@ class CsvTableSink( } override def getOutputType: TypeInformation[Row] = { - new RowTypeInfo(getFieldTypes) + new RowTypeInfo(getFieldTypes: _*) } } @@ -68,15 +68,15 @@ class CsvFormatter(fieldDelim: String) extends MapFunction[Row, String] { val builder = new StringBuilder // write first value - val v = row.productElement(0) + val v = row.getField(0) if (v != null) { builder.append(v.toString) } // write following values - for (i <- 1 until row.productArity) { + for (i <- 1 until row.getArity) { builder.append(fieldDelim) - val v = row.productElement(i) + val v = row.getField(i) if (v != null) { builder.append(v.toString) } diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/sources/CsvTableSource.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/sources/CsvTableSource.scala index 9cf4397f0ce0d..b60575af6c368 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/sources/CsvTableSource.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/sources/CsvTableSource.scala @@ -21,9 +21,10 @@ package org.apache.flink.api.table.sources import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.java.io.CsvInputFormat import org.apache.flink.api.java.{DataSet, ExecutionEnvironment} -import org.apache.flink.api.table.{Row, TableException} -import org.apache.flink.api.table.runtime.io.RowCsvInputFormat -import org.apache.flink.api.table.typeutils.RowTypeInfo +import org.apache.flink.api.table.TableException +import org.apache.flink.types.Row +import org.apache.flink.api.java.io.RowCsvInputFormat +import org.apache.flink.api.java.typeutils.RowTypeInfo import org.apache.flink.core.fs.Path import org.apache.flink.streaming.api.datastream.DataStream import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment @@ -71,7 +72,7 @@ class CsvTableSource( throw TableException("Number of field names and field types must be equal.") } - private val returnType = new RowTypeInfo(fieldTypes) + private val returnType = new RowTypeInfo(fieldTypes: _*) /** * Returns the data of the table as a [[DataSet]] of [[Row]]. diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/typeutils/RowComparator.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/typeutils/RowComparator.scala index 8bbe4d887bca9..a60eba7ac8fce 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/typeutils/RowComparator.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/typeutils/RowComparator.scala @@ -28,6 +28,7 @@ import org.apache.flink.api.table.typeutils.RowComparator.{createAuxiliaryFields import org.apache.flink.core.memory.{DataInputView, DataOutputView, MemorySegment} import org.apache.flink.types.KeyFieldOutOfBoundsException + /** * Comparator for [[Row]]. */ diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/typeutils/TypeConverter.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/typeutils/TypeConverter.scala index a162d9f26bdf3..0a2a0291b2c18 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/typeutils/TypeConverter.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/typeutils/TypeConverter.scala @@ -26,13 +26,15 @@ import org.apache.flink.api.common.typeutils.CompositeType import org.apache.flink.api.java.operators.join.JoinType import org.apache.flink.api.java.tuple.Tuple import org.apache.flink.api.java.typeutils.{PojoTypeInfo, TupleTypeInfo} -import org.apache.flink.api.table.{FlinkTypeFactory, Row, TableException} +import org.apache.flink.api.table.{FlinkTypeFactory, TableException} +import org.apache.flink.types.Row import scala.collection.JavaConversions._ object TypeConverter { - val DEFAULT_ROW_TYPE = new RowTypeInfo(Seq()).asInstanceOf[TypeInformation[Any]] + val DEFAULT_ROW_TYPE = new org.apache.flink.api.java.typeutils.RowTypeInfo( + Seq.empty[TypeInformation[Any]]: _*).asInstanceOf[TypeInformation[Any]] /** * Determines the return type of Flink operators based on the logical fields, the expected @@ -115,7 +117,7 @@ object TypeConverter { // Row is expected, create the arity for it case Some(typeInfo) if typeInfo.getTypeClass == classOf[Row] => - new RowTypeInfo(logicalFieldTypes) + new org.apache.flink.api.java.typeutils.RowTypeInfo(logicalFieldTypes: _*) // no physical type // determine type based on logical fields and configuration parameters @@ -123,7 +125,7 @@ object TypeConverter { // no need for efficient types -> use Row // we cannot use efficient types if row arity > tuple arity or nullable if (!useEfficientTypes || logicalFieldTypes.length > Tuple.MAX_ARITY || nullable) { - new RowTypeInfo(logicalFieldTypes) + new org.apache.flink.api.java.typeutils.RowTypeInfo(logicalFieldTypes: _*) } // use efficient type tuple or atomic type else { diff --git a/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/TableEnvironmentITCase.java b/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/TableEnvironmentITCase.java index 294cba226e93f..75d964ba7d7f1 100644 --- a/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/TableEnvironmentITCase.java +++ b/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/TableEnvironmentITCase.java @@ -36,7 +36,7 @@ import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase; import org.apache.flink.api.table.CalciteConfig; import org.apache.flink.api.table.CalciteConfigBuilder; -import org.apache.flink.api.table.Row; +import org.apache.flink.types.Row; import org.apache.flink.api.table.Table; import org.apache.flink.api.table.TableEnvironment; import org.apache.flink.api.table.TableException; diff --git a/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/TableSourceITCase.java b/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/TableSourceITCase.java index b634d511d98fa..75388080ee10a 100644 --- a/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/TableSourceITCase.java +++ b/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/TableSourceITCase.java @@ -25,11 +25,11 @@ import org.apache.flink.api.java.table.BatchTableEnvironment; import org.apache.flink.api.scala.batch.GeneratingInputFormat; import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase; -import org.apache.flink.api.table.Row; +import org.apache.flink.types.Row; import org.apache.flink.api.table.Table; import org.apache.flink.api.table.TableEnvironment; import org.apache.flink.api.table.sources.BatchTableSource; -import org.apache.flink.api.table.typeutils.RowTypeInfo; +import org.apache.flink.api.java.typeutils.RowTypeInfo; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; diff --git a/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/sql/SqlITCase.java b/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/sql/SqlITCase.java index 1364cbd921c89..89b0d502bed09 100644 --- a/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/sql/SqlITCase.java +++ b/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/sql/SqlITCase.java @@ -24,7 +24,7 @@ import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.api.java.tuple.Tuple5; import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase; -import org.apache.flink.api.table.Row; +import org.apache.flink.types.Row; import org.apache.flink.api.table.Table; import org.apache.flink.api.table.TableEnvironment; import org.apache.flink.test.javaApiOperators.util.CollectionDataSets; diff --git a/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/AggregationsITCase.java b/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/AggregationsITCase.java index 02f6e0b1ab04c..0856a70109abf 100644 --- a/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/AggregationsITCase.java +++ b/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/AggregationsITCase.java @@ -30,7 +30,7 @@ import org.apache.flink.api.java.tuple.Tuple5; import org.apache.flink.api.java.tuple.Tuple7; import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase; -import org.apache.flink.api.table.Row; +import org.apache.flink.types.Row; import org.apache.flink.api.table.Table; import org.apache.flink.api.table.TableEnvironment; import org.apache.flink.api.table.ValidationException; diff --git a/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/CalcITCase.java b/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/CalcITCase.java index 6fc81738bb3f3..1d5c189ca1f4c 100644 --- a/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/CalcITCase.java +++ b/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/CalcITCase.java @@ -23,7 +23,7 @@ import org.apache.flink.api.java.operators.DataSource; import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase; import org.apache.flink.api.table.Table; -import org.apache.flink.api.table.Row; +import org.apache.flink.types.Row; import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.java.table.BatchTableEnvironment; diff --git a/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/CastingITCase.java b/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/CastingITCase.java index 333953b414277..1139837615242 100644 --- a/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/CastingITCase.java +++ b/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/CastingITCase.java @@ -28,7 +28,7 @@ import org.apache.flink.api.java.tuple.Tuple6; import org.apache.flink.api.java.tuple.Tuple8; import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase; -import org.apache.flink.api.table.Row; +import org.apache.flink.types.Row; import org.apache.flink.api.table.Table; import org.apache.flink.api.table.TableEnvironment; import org.junit.Test; diff --git a/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/JoinITCase.java b/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/JoinITCase.java index 9676608bc6014..014c127e3472b 100644 --- a/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/JoinITCase.java +++ b/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/JoinITCase.java @@ -25,7 +25,7 @@ import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.api.java.tuple.Tuple5; import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase; -import org.apache.flink.api.table.Row; +import org.apache.flink.types.Row; import org.apache.flink.api.table.Table; import org.apache.flink.api.table.TableEnvironment; import org.apache.flink.api.table.ValidationException; diff --git a/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/stream/sql/SqlITCase.java b/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/stream/sql/SqlITCase.java index 10ae5d96d291a..53a1a7d7d1aaf 100644 --- a/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/stream/sql/SqlITCase.java +++ b/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/stream/sql/SqlITCase.java @@ -22,7 +22,7 @@ import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.api.java.tuple.Tuple5; import org.apache.flink.api.scala.stream.utils.StreamITCase; -import org.apache.flink.api.table.Row; +import org.apache.flink.types.Row; import org.apache.flink.api.table.Table; import org.apache.flink.api.table.TableEnvironment; import org.apache.flink.streaming.api.datastream.DataStream; diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/TableEnvironmentITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/TableEnvironmentITCase.scala index 4c07615fed9b5..b7c8bc0ffc4ad 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/TableEnvironmentITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/TableEnvironmentITCase.scala @@ -25,7 +25,8 @@ import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase.TableConfigMode import org.apache.flink.api.scala.table._ import org.apache.flink.api.scala.util.CollectionDataSets -import org.apache.flink.api.table.{Row, TableEnvironment, TableException, ValidationException} +import org.apache.flink.api.table.{TableEnvironment, TableException, ValidationException} +import org.apache.flink.types.Row import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode import org.apache.flink.test.util.TestBaseUtils import org.junit._ diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/TableSourceITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/TableSourceITCase.scala index 08bee72601a9d..b5c8adada8a4c 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/TableSourceITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/TableSourceITCase.scala @@ -28,10 +28,11 @@ import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase.TableConfigMode import org.apache.flink.api.scala.table._ import org.apache.flink.api.table.sources.{BatchTableSource, CsvTableSource} -import org.apache.flink.api.table.typeutils.RowTypeInfo -import org.apache.flink.api.table.{Row, TableEnvironment} +import org.apache.flink.api.java.typeutils.RowTypeInfo +import org.apache.flink.api.table.TableEnvironment +import org.apache.flink.types.Row import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode -import org.apache.flink.test.util.{MultipleProgramsTestBase, TestBaseUtils} +import org.apache.flink.test.util.TestBaseUtils import org.junit.Test import org.junit.runner.RunWith import org.junit.runners.Parameterized @@ -153,7 +154,7 @@ class TestBatchTableSource extends BatchTableSource[Row] { override def getFieldsNames: Array[String] = Array("name", "id", "amount") /** Returns the [[TypeInformation]] for the return type. */ - override def getReturnType: TypeInformation[Row] = new RowTypeInfo(fieldTypes) + override def getReturnType: TypeInformation[Row] = new RowTypeInfo(fieldTypes: _*) /** Returns the number of fields of the table. */ override def getNumberOfFields: Int = 3 diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/AggregationsITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/AggregationsITCase.scala index 35bb7dc0f12b6..d5d46babd6531 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/AggregationsITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/AggregationsITCase.scala @@ -23,7 +23,8 @@ import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase import org.apache.flink.api.scala.table._ import org.apache.flink.api.scala.util.CollectionDataSets import TableProgramsTestBase.TableConfigMode -import org.apache.flink.api.table.{TableException, Row, TableEnvironment} +import org.apache.flink.api.table.{TableException, TableEnvironment} +import org.apache.flink.types.Row import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode import org.apache.flink.test.util.TestBaseUtils import org.junit._ diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/CalcITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/CalcITCase.scala index 155833b1c1afc..50374694fc653 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/CalcITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/CalcITCase.scala @@ -29,7 +29,8 @@ import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase.TableConfigM import org.apache.flink.api.scala.table._ import org.apache.flink.api.scala.util.CollectionDataSets import org.apache.flink.api.table.functions.ScalarFunction -import org.apache.flink.api.table.{Row, TableEnvironment, ValidationException} +import org.apache.flink.api.table.{TableEnvironment, ValidationException} +import org.apache.flink.types.Row import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode import org.apache.flink.test.util.TestBaseUtils import org.junit._ diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/JoinITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/JoinITCase.scala index 68f63c3b8f6b6..074f70b496c71 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/JoinITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/JoinITCase.scala @@ -23,7 +23,8 @@ import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase.TableConfigMode import org.apache.flink.api.scala.table._ import org.apache.flink.api.scala.util.CollectionDataSets -import org.apache.flink.api.table.{ValidationException, Row, TableEnvironment, TableException} +import org.apache.flink.api.table.{ValidationException, TableEnvironment, TableException} +import org.apache.flink.types.Row import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode import org.apache.flink.test.util.TestBaseUtils import org.junit._ diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/SetOperatorsITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/SetOperatorsITCase.scala index 7cdb74637cd81..42bd6e8184c77 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/SetOperatorsITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/SetOperatorsITCase.scala @@ -23,7 +23,8 @@ import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase.TableConfigMode import org.apache.flink.api.scala.table._ import org.apache.flink.api.scala.util.CollectionDataSets -import org.apache.flink.api.table.{Row, TableEnvironment} +import org.apache.flink.api.table.TableEnvironment +import org.apache.flink.types.Row import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode import org.apache.flink.test.util.TestBaseUtils import org.junit._ diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/SortITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/SortITCase.scala index f3459848f03ff..1698f76ba32b4 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/SortITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/SortITCase.scala @@ -25,7 +25,8 @@ import org.apache.flink.api.scala.batch.utils.SortTestUtils._ import org.apache.flink.api.scala.util.CollectionDataSets import org.apache.flink.api.scala.table._ import org.apache.flink.api.scala._ -import org.apache.flink.api.table.{Row, TableEnvironment, TableException} +import org.apache.flink.api.table.{TableEnvironment, TableException} +import org.apache.flink.types.Row import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode import org.apache.flink.test.util.TestBaseUtils import org.junit._ @@ -49,13 +50,16 @@ class SortITCase( implicit def rowOrdering[T <: Product] = Ordering.by((x : T) => (- x.productElement(0).asInstanceOf[Int], - x.productElement(1).asInstanceOf[Long])) + implicit def objOrdering[T <: Row] = Ordering.by((x : T) => + (- x.getField(0).asInstanceOf[Int], - x.getField(1).asInstanceOf[Long])) + val ds = CollectionDataSets.get3TupleDataSet(env) tEnv.registerDataSet("MyTable", ds) val expected = sortExpectedly(tupleDataSetStrings) val results = tEnv.sql(sqlQuery).toDataSet[Row].mapPartition(rows => Seq(rows.toSeq)).collect() - val result = results.filterNot(_.isEmpty).sortBy(p => p.head).reduceLeft(_ ++ _) + val result = results.filterNot(_.isEmpty).sortBy(f => f.head).reduceLeft(_ ++ _) TestBaseUtils.compareOrderedResultAsText(result.asJava, expected) } @@ -76,7 +80,10 @@ class SortITCase( val expected = sortExpectedly(tupleDataSetStrings, 2, 21) val results = tEnv.sql(sqlQuery).toDataSet[Row].mapPartition(rows => Seq(rows.toSeq)).collect() - val result = results.filterNot(_.isEmpty).sortBy(p => p.head).reduceLeft(_ ++ _) + val result = results. + filterNot(_.isEmpty) + .sortBy(_.head)(Ordering.by(f=> f.toString)) + .reduceLeft(_ ++ _) TestBaseUtils.compareOrderedResultAsText(result.asJava, expected) } @@ -97,7 +104,10 @@ class SortITCase( val expected = sortExpectedly(tupleDataSetStrings, 2, 7) val results = tEnv.sql(sqlQuery).toDataSet[Row].mapPartition(rows => Seq(rows.toSeq)).collect() - val result = results.filterNot(_.isEmpty).sortBy(p => p.head).reduceLeft(_ ++ _) + val result = results + .filterNot(_.isEmpty) + .sortBy(_.head)(Ordering.by(f=> f.toString)) + .reduceLeft(_ ++ _) TestBaseUtils.compareOrderedResultAsText(result.asJava, expected) } @@ -118,7 +128,10 @@ class SortITCase( val expected = sortExpectedly(tupleDataSetStrings, 0, 5) val results = tEnv.sql(sqlQuery).toDataSet[Row].mapPartition(rows => Seq(rows.toSeq)).collect() - val result = results.filterNot(_.isEmpty).sortBy(p => p.head).reduceLeft(_ ++ _) + val result = results + .filterNot(_.isEmpty) + .sortBy(_.head)(Ordering.by(f=> f.toString)) + .reduceLeft(_ ++ _) TestBaseUtils.compareOrderedResultAsText(result.asJava, expected) } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/TableWithSQLITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/TableWithSQLITCase.scala index a770a6e46ee71..d41f3e0467131 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/TableWithSQLITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/TableWithSQLITCase.scala @@ -23,7 +23,8 @@ import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase.TableConfigMode import org.apache.flink.api.scala.table._ import org.apache.flink.api.scala.util.CollectionDataSets -import org.apache.flink.api.table.{Row, TableEnvironment} +import org.apache.flink.api.table.TableEnvironment +import org.apache.flink.types.Row import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode import org.apache.flink.test.util.TestBaseUtils import org.junit._ diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/AggregationsITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/AggregationsITCase.scala index 16c8ececdb982..3f4e1e510b638 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/AggregationsITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/AggregationsITCase.scala @@ -23,7 +23,8 @@ import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase.TableConfigMode import org.apache.flink.api.scala.table._ import org.apache.flink.api.scala.util.CollectionDataSets -import org.apache.flink.api.table.{Row, TableEnvironment, ValidationException} +import org.apache.flink.api.table.{TableEnvironment, ValidationException} +import org.apache.flink.types.Row import org.apache.flink.examples.scala.WordCountTable.{WC => MyWC} import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode import org.apache.flink.test.util.TestBaseUtils diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/CalcITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/CalcITCase.scala index c3758a430d50b..b01146287d8e1 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/CalcITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/CalcITCase.scala @@ -27,7 +27,8 @@ import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase.TableConfigM import org.apache.flink.api.scala.table._ import org.apache.flink.api.scala.util.CollectionDataSets import org.apache.flink.api.table.expressions.Literal -import org.apache.flink.api.table.{Row, TableEnvironment, TableException, ValidationException} +import org.apache.flink.api.table.{TableEnvironment, TableException, ValidationException} +import org.apache.flink.types.Row import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode import org.apache.flink.test.util.TestBaseUtils import org.junit.Assert._ diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/JoinITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/JoinITCase.scala index 67cac14d655e3..195027d9df0d6 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/JoinITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/JoinITCase.scala @@ -24,7 +24,8 @@ import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase.TableConfigM import org.apache.flink.api.scala.table._ import org.apache.flink.api.scala.util.CollectionDataSets import org.apache.flink.api.table.expressions.Literal -import org.apache.flink.api.table.{Row, TableEnvironment, TableException, ValidationException} +import org.apache.flink.api.table.{TableEnvironment, TableException, ValidationException} +import org.apache.flink.types.Row import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode import org.apache.flink.test.util.TestBaseUtils import org.junit._ diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/SetOperatorsITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/SetOperatorsITCase.scala index 283ba10082b81..0d32cb47674d2 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/SetOperatorsITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/SetOperatorsITCase.scala @@ -23,7 +23,8 @@ import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase.TableConfigMode import org.apache.flink.api.scala.table._ import org.apache.flink.api.scala.util.CollectionDataSets -import org.apache.flink.api.table.{Row, TableEnvironment, ValidationException} +import org.apache.flink.api.table.{TableEnvironment, ValidationException} +import org.apache.flink.types.Row import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode import org.apache.flink.test.util.TestBaseUtils import org.junit._ diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/SortITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/SortITCase.scala index d4a1d8d1ab76b..2f5398c2bc7eb 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/SortITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/SortITCase.scala @@ -24,7 +24,8 @@ import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase.TableConfigM import org.apache.flink.api.scala.table._ import org.apache.flink.api.scala.util.CollectionDataSets import org.apache.flink.api.scala.{ExecutionEnvironment, _} -import org.apache.flink.api.table.{Row, TableEnvironment, ValidationException} +import org.apache.flink.api.table.{TableEnvironment, ValidationException} +import org.apache.flink.types.Row import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode import org.apache.flink.test.util.TestBaseUtils import org.junit._ @@ -58,7 +59,10 @@ class SortITCase( val expected = sortExpectedly(tupleDataSetStrings) val results = t.toDataSet[Row].mapPartition(rows => Seq(rows.toSeq)).collect() - val result = results.filterNot(_.isEmpty).sortBy(p => p.head).reduceLeft(_ ++ _) + val result = results + .filterNot(_.isEmpty) + .sortBy(_.head)(Ordering.by(f=> f.toString)) + .reduceLeft(_ ++ _) TestBaseUtils.compareOrderedResultAsText(result.asJava, expected) } @@ -76,7 +80,10 @@ class SortITCase( val expected = sortExpectedly(tupleDataSetStrings) val results = t.toDataSet[Row].mapPartition(rows => Seq(rows.toSeq)).collect() - val result = results.filterNot(_.isEmpty).sortBy(p => p.head).reduceLeft(_ ++ _) + val result = results + .filterNot(_.isEmpty) + .sortBy(_.head)(Ordering.by(f=> f.toString)) + .reduceLeft(_ ++ _) TestBaseUtils.compareOrderedResultAsText(result.asJava, expected) } @@ -94,7 +101,10 @@ class SortITCase( val expected = sortExpectedly(tupleDataSetStrings) val results = t.toDataSet[Row].mapPartition(rows => Seq(rows.toSeq)).collect() - val result = results.filterNot(_.isEmpty).sortBy(p => p.head).reduceLeft(_ ++ _) + val result = results + .filterNot(_.isEmpty) + .sortBy(_.head)(Ordering.by(f=> f.toString)) + .reduceLeft(_ ++ _) TestBaseUtils.compareOrderedResultAsText(result.asJava, expected) } @@ -112,7 +122,10 @@ class SortITCase( val expected = sortExpectedly(tupleDataSetStrings, 3, 21) val results = t.toDataSet[Row].mapPartition(rows => Seq(rows.toSeq)).collect() - val result = results.filterNot(_.isEmpty).sortBy(p => p.head).reduceLeft(_ ++ _) + val result = results + .filterNot(_.isEmpty) + .sortBy(_.head)(Ordering.by(f=> f.toString)) + .reduceLeft(_ ++ _) TestBaseUtils.compareOrderedResultAsText(result.asJava, expected) } @@ -130,7 +143,10 @@ class SortITCase( val expected = sortExpectedly(tupleDataSetStrings, 3, 8) val results = t.toDataSet[Row].mapPartition(rows => Seq(rows.toSeq)).collect() - val result = results.filterNot(_.isEmpty).sortBy(p => p.head).reduceLeft(_ ++ _) + val result = results + .filterNot(_.isEmpty) + .sortBy(_.head)(Ordering.by(f=> f.toString)) + .reduceLeft(_ ++ _) TestBaseUtils.compareOrderedResultAsText(result.asJava, expected) } @@ -148,7 +164,10 @@ class SortITCase( val expected = sortExpectedly(tupleDataSetStrings, 0, 5) val results = t.toDataSet[Row].mapPartition(rows => Seq(rows.toSeq)).collect() - val result = results.filterNot(_.isEmpty).sortBy(p => p.head).reduceLeft(_ ++ _) + val result = results + .filterNot(_.isEmpty) + .sortBy(_.head)(Ordering.by(f=> f.toString)) + .reduceLeft(_ ++ _) TestBaseUtils.compareOrderedResultAsText(result.asJava, expected) } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/UserDefinedTableFunctionTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/UserDefinedTableFunctionTest.scala index 7e170d428a322..285a181fb69f0 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/UserDefinedTableFunctionTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/UserDefinedTableFunctionTest.scala @@ -21,10 +21,11 @@ import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.java.{DataSet => JDataSet, ExecutionEnvironment => JavaExecutionEnv} import org.apache.flink.api.scala.table._ import org.apache.flink.api.scala.{DataSet, ExecutionEnvironment => ScalaExecutionEnv, _} -import org.apache.flink.api.table.typeutils.RowTypeInfo +import org.apache.flink.api.java.typeutils.RowTypeInfo import org.apache.flink.api.table.utils.TableTestUtil._ import org.apache.flink.api.table.utils.{PojoTableFunc, TableFunc2, _} -import org.apache.flink.api.table.{Row, TableEnvironment, Types} +import org.apache.flink.api.table.{TableEnvironment, Types} +import org.apache.flink.types.Row import org.junit.Test import org.mockito.Mockito._ @@ -35,7 +36,7 @@ class UserDefinedTableFunctionTest extends TableTestBase { // mock val ds = mock(classOf[DataSet[Row]]) val jDs = mock(classOf[JDataSet[Row]]) - val typeInfo: TypeInformation[Row] = new RowTypeInfo(Seq(Types.INT, Types.LONG, Types.STRING)) + val typeInfo = new RowTypeInfo(Seq(Types.INT, Types.LONG, Types.STRING): _*) when(ds.javaSet).thenReturn(jDs) when(jDs.getType).thenReturn(typeInfo) diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/TableSourceITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/TableSourceITCase.scala index c14ad979636c6..1c9311271e320 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/TableSourceITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/TableSourceITCase.scala @@ -25,8 +25,9 @@ import org.apache.flink.api.scala._ import org.apache.flink.api.scala.stream.utils.StreamITCase import org.apache.flink.api.scala.table._ import org.apache.flink.api.table.sources.{CsvTableSource, StreamTableSource} -import org.apache.flink.api.table.typeutils.RowTypeInfo -import org.apache.flink.api.table.{Row, TableEnvironment} +import org.apache.flink.api.java.typeutils.RowTypeInfo +import org.apache.flink.api.table.TableEnvironment +import org.apache.flink.types.Row import org.apache.flink.streaming.api.datastream.DataStream import org.apache.flink.streaming.api.environment import org.apache.flink.streaming.api.functions.source.SourceFunction @@ -163,7 +164,7 @@ class TestStreamTableSource(val numRecords: Int) extends StreamTableSource[Row] override def getFieldsNames: Array[String] = Array("name", "id", "amount") /** Returns the [[TypeInformation]] for the return type. */ - override def getReturnType: TypeInformation[Row] = new RowTypeInfo(fieldTypes) + override def getReturnType: TypeInformation[Row] = new RowTypeInfo(fieldTypes: _*) /** Returns the number of fields of the table. */ override def getNumberOfFields: Int = 3 diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/sql/SqlITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/sql/SqlITCase.scala index 5b278c1e2018a..c4ca9647562e3 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/sql/SqlITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/sql/SqlITCase.scala @@ -21,7 +21,8 @@ package org.apache.flink.api.scala.stream.sql import org.apache.flink.api.scala._ import org.apache.flink.api.scala.stream.utils.{StreamTestData, StreamITCase} import org.apache.flink.api.scala.table._ -import org.apache.flink.api.table.{Row, TableEnvironment} +import org.apache.flink.api.table.TableEnvironment +import org.apache.flink.types.Row import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase import org.junit.Assert._ diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/table/AggregationsITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/table/AggregationsITCase.scala index 075348419d5bf..d398556bc9152 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/table/AggregationsITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/table/AggregationsITCase.scala @@ -22,7 +22,8 @@ import org.apache.flink.api.scala._ import org.apache.flink.api.scala.stream.table.GroupWindowITCase.TimestampWithEqualWatermark import org.apache.flink.api.scala.stream.utils.StreamITCase import org.apache.flink.api.scala.table._ -import org.apache.flink.api.table.{Row, _} +import org.apache.flink.api.table._ +import org.apache.flink.types.Row import org.apache.flink.streaming.api.TimeCharacteristic import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/table/CalcITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/table/CalcITCase.scala index 578ad30f2836f..3eee4d4b93301 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/table/CalcITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/table/CalcITCase.scala @@ -22,7 +22,8 @@ import org.apache.flink.api.scala._ import org.apache.flink.api.scala.stream.utils.{StreamITCase, StreamTestData} import org.apache.flink.api.scala.table._ import org.apache.flink.api.table.expressions.Literal -import org.apache.flink.api.table.{Row, TableEnvironment, TableException} +import org.apache.flink.api.table.{TableEnvironment, TableException} +import org.apache.flink.types.Row import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase import org.junit.Assert._ diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/table/UnionITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/table/UnionITCase.scala index 131974eca0221..5096b53a1db59 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/table/UnionITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/table/UnionITCase.scala @@ -21,7 +21,8 @@ package org.apache.flink.api.scala.stream.table import org.apache.flink.api.scala._ import org.apache.flink.api.scala.stream.utils.{StreamITCase, StreamTestData} import org.apache.flink.api.scala.table._ -import org.apache.flink.api.table.{Row, TableEnvironment, ValidationException} +import org.apache.flink.api.table.{TableEnvironment, ValidationException} +import org.apache.flink.types.Row import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase import org.junit.Assert._ diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/table/UserDefinedTableFunctionTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/table/UserDefinedTableFunctionTest.scala index b45ae8e436ee3..305f1db6f5966 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/table/UserDefinedTableFunctionTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/table/UserDefinedTableFunctionTest.scala @@ -17,12 +17,12 @@ */ package org.apache.flink.api.scala.stream.table -import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.scala._ import org.apache.flink.api.scala.table._ -import org.apache.flink.api.table._ +import org.apache.flink.types.Row import org.apache.flink.api.table.expressions.utils._ -import org.apache.flink.api.table.typeutils.RowTypeInfo +import org.apache.flink.api.java.typeutils.RowTypeInfo +import org.apache.flink.api.table.{TableEnvironment, TableException, Types, ValidationException} import org.apache.flink.api.table.utils.TableTestUtil._ import org.apache.flink.api.table.utils._ import org.apache.flink.streaming.api.datastream.{DataStream => JDataStream} @@ -39,7 +39,7 @@ class UserDefinedTableFunctionTest extends TableTestBase { // mock val ds = mock(classOf[DataStream[Row]]) val jDs = mock(classOf[JDataStream[Row]]) - val typeInfo: TypeInformation[Row] = new RowTypeInfo(Seq(Types.INT, Types.LONG, Types.STRING)) + val typeInfo = new RowTypeInfo(Seq(Types.INT, Types.LONG, Types.STRING): _*) when(ds.javaStream).thenReturn(jDs) when(jDs.getType).thenReturn(typeInfo) diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/utils/StreamITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/utils/StreamITCase.scala index 48600055770cd..4fd3cd7596bde 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/utils/StreamITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/utils/StreamITCase.scala @@ -20,7 +20,7 @@ package org.apache.flink.api.scala.stream.utils import java.util.Collections -import org.apache.flink.api.table.Row +import org.apache.flink.types.Row import org.junit.Assert._ import scala.collection.mutable import org.apache.flink.streaming.api.functions.sink.RichSinkFunction diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/ArrayTypeTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/ArrayTypeTest.scala index 034ce0ba191e5..2ba76ad04bdf9 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/ArrayTypeTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/ArrayTypeTest.scala @@ -24,8 +24,9 @@ import org.apache.flink.api.common.typeinfo.{PrimitiveArrayTypeInfo, TypeInforma import org.apache.flink.api.java.typeutils.ObjectArrayTypeInfo import org.apache.flink.api.scala.table._ import org.apache.flink.api.table.expressions.utils.ExpressionTestBase -import org.apache.flink.api.table.typeutils.RowTypeInfo -import org.apache.flink.api.table.{Row, Types, ValidationException} +import org.apache.flink.api.java.typeutils.RowTypeInfo +import org.apache.flink.api.table.{Types, ValidationException} +import org.apache.flink.types.Row import org.junit.Test class ArrayTypeTest extends ExpressionTestBase { @@ -342,7 +343,7 @@ class ArrayTypeTest extends ExpressionTestBase { } override def typeInfo: TypeInformation[Any] = { - new RowTypeInfo(Seq( + new RowTypeInfo( Types.INT, Types.INT, PrimitiveArrayTypeInfo.INT_PRIMITIVE_ARRAY_TYPE_INFO, @@ -354,6 +355,6 @@ class ArrayTypeTest extends ExpressionTestBase { PrimitiveArrayTypeInfo.DOUBLE_PRIMITIVE_ARRAY_TYPE_INFO, ObjectArrayTypeInfo.getInfoFor(Types.INT), ObjectArrayTypeInfo.getInfoFor(Types.INT) - )).asInstanceOf[TypeInformation[Any]] + ).asInstanceOf[TypeInformation[Any]] } } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/CompositeAccessTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/CompositeAccessTest.scala index 3121c58540413..879f68ddc3d72 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/CompositeAccessTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/CompositeAccessTest.scala @@ -22,10 +22,11 @@ import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.java.typeutils.{TupleTypeInfo, TypeExtractor} import org.apache.flink.api.scala.createTypeInformation import org.apache.flink.api.scala.table._ -import org.apache.flink.api.table.{Row, Types, ValidationException} +import org.apache.flink.api.table.{Types, ValidationException} +import org.apache.flink.types.Row import org.apache.flink.api.table.expressions.CompositeAccessTest.{MyCaseClass, MyCaseClass2, MyPojo} import org.apache.flink.api.table.expressions.utils.ExpressionTestBase -import org.apache.flink.api.table.typeutils.RowTypeInfo +import org.apache.flink.api.java.typeutils.RowTypeInfo import org.junit.Test @@ -154,7 +155,7 @@ class CompositeAccessTest extends ExpressionTestBase { } def typeInfo = { - new RowTypeInfo(Seq( + new RowTypeInfo( createTypeInformation[MyCaseClass], createTypeInformation[MyCaseClass2], createTypeInformation[(String, String)], @@ -163,7 +164,7 @@ class CompositeAccessTest extends ExpressionTestBase { Types.INT, createTypeInformation[MyCaseClass2], createTypeInformation[Tuple1[Boolean]] - )).asInstanceOf[TypeInformation[Any]] + ).asInstanceOf[TypeInformation[Any]] } } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/DecimalTypeTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/DecimalTypeTest.scala index 20a8af843443f..a9863651584e8 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/DecimalTypeTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/DecimalTypeTest.scala @@ -20,9 +20,10 @@ package org.apache.flink.api.table.expressions import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.scala.table._ -import org.apache.flink.api.table.{Row, Types} +import org.apache.flink.api.table.Types +import org.apache.flink.types.Row import org.apache.flink.api.table.expressions.utils.ExpressionTestBase -import org.apache.flink.api.table.typeutils.RowTypeInfo +import org.apache.flink.api.java.typeutils.RowTypeInfo import org.junit.Test class DecimalTypeTest extends ExpressionTestBase { @@ -298,13 +299,13 @@ class DecimalTypeTest extends ExpressionTestBase { } def typeInfo = { - new RowTypeInfo(Seq( + new RowTypeInfo( Types.DECIMAL, Types.DECIMAL, Types.INT, Types.DOUBLE, Types.DECIMAL, - Types.DECIMAL)).asInstanceOf[TypeInformation[Any]] + Types.DECIMAL).asInstanceOf[TypeInformation[Any]] } } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/NonDeterministicTests.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/NonDeterministicTests.scala index de48849159d13..afefb3cf372d4 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/NonDeterministicTests.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/NonDeterministicTests.scala @@ -20,9 +20,9 @@ package org.apache.flink.api.table.expressions import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.scala.table._ -import org.apache.flink.api.table.Row +import org.apache.flink.types.Row import org.apache.flink.api.table.expressions.utils.ExpressionTestBase -import org.apache.flink.api.table.typeutils.RowTypeInfo +import org.apache.flink.api.java.typeutils.RowTypeInfo import org.junit.{Ignore, Test} /** @@ -85,5 +85,5 @@ class NonDeterministicTests extends ExpressionTestBase { override def testData: Any = new Row(0) override def typeInfo: TypeInformation[Any] = - new RowTypeInfo(Seq[TypeInformation[_]]()).asInstanceOf[TypeInformation[Any]] + new RowTypeInfo(Seq.empty[TypeInformation[Any]]: _*).asInstanceOf[TypeInformation[Any]] } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/ScalarFunctionsTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/ScalarFunctionsTest.scala index 1d2a1b7dfd159..3ef02a96a02ca 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/ScalarFunctionsTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/ScalarFunctionsTest.scala @@ -23,8 +23,9 @@ import java.sql.{Date, Time, Timestamp} import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.scala.table._ import org.apache.flink.api.table.expressions.utils.ExpressionTestBase -import org.apache.flink.api.table.typeutils.RowTypeInfo -import org.apache.flink.api.table.{Row, Types, ValidationException} +import org.apache.flink.api.java.typeutils.RowTypeInfo +import org.apache.flink.api.table.{Types, ValidationException} +import org.apache.flink.types.Row import org.junit.Test class ScalarFunctionsTest extends ExpressionTestBase { @@ -1134,7 +1135,7 @@ class ScalarFunctionsTest extends ExpressionTestBase { } def typeInfo = { - new RowTypeInfo(Seq( + new RowTypeInfo( Types.STRING, Types.BOOLEAN, Types.BYTE, @@ -1159,7 +1160,7 @@ class ScalarFunctionsTest extends ExpressionTestBase { Types.BOOLEAN, Types.DECIMAL, Types.STRING, - Types.STRING)).asInstanceOf[TypeInformation[Any]] + Types.STRING).asInstanceOf[TypeInformation[Any]] } } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/ScalarOperatorsTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/ScalarOperatorsTest.scala index 7ad22120feaa7..86f884ffa8b85 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/ScalarOperatorsTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/ScalarOperatorsTest.scala @@ -21,8 +21,9 @@ package org.apache.flink.api.table.expressions import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.scala.table._ import org.apache.flink.api.table.expressions.utils.ExpressionTestBase -import org.apache.flink.api.table.typeutils.RowTypeInfo -import org.apache.flink.api.table.{Row, Types, ValidationException} +import org.apache.flink.api.java.typeutils.RowTypeInfo +import org.apache.flink.api.table.{Types, ValidationException} +import org.apache.flink.types.Row import org.junit.Test class ScalarOperatorsTest extends ExpressionTestBase { @@ -201,7 +202,7 @@ class ScalarOperatorsTest extends ExpressionTestBase { } def typeInfo = { - new RowTypeInfo(Seq( + new RowTypeInfo( Types.BYTE, Types.SHORT, Types.INT, @@ -213,7 +214,7 @@ class ScalarOperatorsTest extends ExpressionTestBase { Types.INT, Types.INT, Types.STRING - )).asInstanceOf[TypeInformation[Any]] + ).asInstanceOf[TypeInformation[Any]] } } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/SqlExpressionTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/SqlExpressionTest.scala index 52dc848a93d23..97267306862e8 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/SqlExpressionTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/SqlExpressionTest.scala @@ -19,9 +19,9 @@ package org.apache.flink.api.table.expressions import org.apache.flink.api.common.typeinfo.TypeInformation -import org.apache.flink.api.table.Row +import org.apache.flink.types.Row import org.apache.flink.api.table.expressions.utils.ExpressionTestBase -import org.apache.flink.api.table.typeutils.RowTypeInfo +import org.apache.flink.api.java.typeutils.RowTypeInfo import org.junit.{Ignore, Test} /** @@ -166,5 +166,5 @@ class SqlExpressionTest extends ExpressionTestBase { override def testData: Any = new Row(0) override def typeInfo: TypeInformation[Any] = - new RowTypeInfo(Seq()).asInstanceOf[TypeInformation[Any]] + new RowTypeInfo(Seq.empty[TypeInformation[Any]]: _*).asInstanceOf[TypeInformation[Any]] } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/TemporalTypesTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/TemporalTypesTest.scala index 0547552cb17e8..bd771baa81cc3 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/TemporalTypesTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/TemporalTypesTest.scala @@ -23,8 +23,9 @@ import java.sql.{Date, Time, Timestamp} import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.scala.table._ import org.apache.flink.api.table.expressions.utils.ExpressionTestBase -import org.apache.flink.api.table.typeutils.RowTypeInfo -import org.apache.flink.api.table.{Row, Types} +import org.apache.flink.api.java.typeutils.RowTypeInfo +import org.apache.flink.api.table.Types +import org.apache.flink.types.Row import org.junit.Test class TemporalTypesTest extends ExpressionTestBase { @@ -556,7 +557,7 @@ class TemporalTypesTest extends ExpressionTestBase { } def typeInfo = { - new RowTypeInfo(Seq( + new RowTypeInfo( Types.DATE, Types.TIME, Types.TIMESTAMP, @@ -567,6 +568,6 @@ class TemporalTypesTest extends ExpressionTestBase { Types.INT, Types.LONG, Types.INTERVAL_MONTHS, - Types.INTERVAL_MILLIS)).asInstanceOf[TypeInformation[Any]] + Types.INTERVAL_MILLIS).asInstanceOf[TypeInformation[Any]] } } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/UserDefinedScalarFunctionTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/UserDefinedScalarFunctionTest.scala index ffe3cd30fb4bd..567cca1e9d1eb 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/UserDefinedScalarFunctionTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/UserDefinedScalarFunctionTest.scala @@ -25,8 +25,9 @@ import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.scala.table._ import org.apache.flink.api.table.expressions.utils._ import org.apache.flink.api.table.functions.ScalarFunction -import org.apache.flink.api.table.typeutils.RowTypeInfo -import org.apache.flink.api.table.{Row, Types} +import org.apache.flink.api.java.typeutils.RowTypeInfo +import org.apache.flink.api.table.Types +import org.apache.flink.types.Row import org.junit.Test class UserDefinedScalarFunctionTest extends ExpressionTestBase { @@ -195,7 +196,7 @@ class UserDefinedScalarFunctionTest extends ExpressionTestBase { } override def typeInfo: TypeInformation[Any] = { - new RowTypeInfo(Seq( + new RowTypeInfo( Types.INT, Types.STRING, Types.BOOLEAN, @@ -205,7 +206,7 @@ class UserDefinedScalarFunctionTest extends ExpressionTestBase { Types.TIMESTAMP, Types.INTERVAL_MONTHS, Types.INTERVAL_MILLIS - )).asInstanceOf[TypeInformation[Any]] + ).asInstanceOf[TypeInformation[Any]] } override def functions: Map[String, ScalarFunction] = Map( diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/utils/ExpressionTestBase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/utils/ExpressionTestBase.scala index 958fd259e6a39..a884c550c62b1 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/utils/ExpressionTestBase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/utils/ExpressionTestBase.scala @@ -33,7 +33,7 @@ import org.apache.flink.api.table.expressions.{Expression, ExpressionParser} import org.apache.flink.api.table.functions.ScalarFunction import org.apache.flink.api.table.plan.nodes.dataset.{DataSetCalc, DataSetConvention} import org.apache.flink.api.table.plan.rules.FlinkRuleSets -import org.apache.flink.api.table.typeutils.RowTypeInfo +import org.apache.flink.api.java.typeutils.RowTypeInfo import org.junit.Assert._ import org.junit.{After, Before} import org.mockito.Mockito._ @@ -96,7 +96,7 @@ abstract class ExpressionTestBase { val stringTestExprs = testExprs.map(expr => relBuilder.cast(expr._1, VARCHAR)).toSeq // generate code - val resultType = new RowTypeInfo(Seq.fill(testExprs.size)(STRING_TYPE_INFO)) + val resultType = new RowTypeInfo(Seq.fill(testExprs.size)(STRING_TYPE_INFO): _*) val genExpr = generator.generateResultExpression( resultType, resultType.getFieldNames, @@ -117,14 +117,14 @@ abstract class ExpressionTestBase { // compile and evaluate val clazz = new TestCompiler[MapFunction[Any, String]]().compile(genFunc) val mapper = clazz.newInstance() - val result = mapper.map(testData).asInstanceOf[Row] + val result = mapper.map(testData).asInstanceOf[org.apache.flink.types.Row] // compare testExprs .zipWithIndex .foreach { case ((expr, expected), index) => - val actual = result.productElement(index) + val actual = result.getField(index) assertEquals( s"Wrong result for: $expr", expected, diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/runtime/aggregate/AggregateTestBase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/runtime/aggregate/AggregateTestBase.scala index 54911a55742fc..4e33a61c1c49b 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/runtime/aggregate/AggregateTestBase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/runtime/aggregate/AggregateTestBase.scala @@ -19,7 +19,7 @@ package org.apache.flink.api.table.runtime.aggregate import java.math.BigDecimal -import org.apache.flink.api.table.Row +import org.apache.flink.types.Row import org.junit.Test import org.junit.Assert.assertEquals diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/runtime/dataset/DataSetCorrelateITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/runtime/dataset/DataSetCorrelateITCase.scala index 32559f16c9b2d..993347fdaf7b9 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/runtime/dataset/DataSetCorrelateITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/runtime/dataset/DataSetCorrelateITCase.scala @@ -23,7 +23,8 @@ import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase.TableConfigM import org.apache.flink.api.scala.table._ import org.apache.flink.api.table.expressions.utils._ import org.apache.flink.api.table.utils._ -import org.apache.flink.api.table.{Row, TableEnvironment} +import org.apache.flink.api.table.TableEnvironment +import org.apache.flink.types.Row import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode import org.apache.flink.test.util.TestBaseUtils import org.junit.Test diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/runtime/datastream/DataStreamCorrelateITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/runtime/datastream/DataStreamCorrelateITCase.scala index 70b0359207485..21fe1573adb7f 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/runtime/datastream/DataStreamCorrelateITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/runtime/datastream/DataStreamCorrelateITCase.scala @@ -21,7 +21,8 @@ import org.apache.flink.api.scala._ import org.apache.flink.api.scala.stream.utils.StreamITCase import org.apache.flink.api.scala.table._ import org.apache.flink.api.table.utils.TableFunc0 -import org.apache.flink.api.table.{Row, TableEnvironment} +import org.apache.flink.api.table.TableEnvironment +import org.apache.flink.types.Row import org.apache.flink.streaming.api.scala.{DataStream, StreamExecutionEnvironment} import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase import org.junit.Assert._ diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/runtime/io/RowCsvInputFormatTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/runtime/io/RowCsvInputFormatTest.scala index d72e7a8eba287..36aaf88f9dde2 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/runtime/io/RowCsvInputFormatTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/runtime/io/RowCsvInputFormatTest.scala @@ -24,14 +24,15 @@ import java.sql.{Date, Time, Timestamp} import org.apache.flink.api.common.io.ParseException import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, SqlTimeTypeInfo} -import org.apache.flink.api.table.Row +import org.apache.flink.types.Row import org.apache.flink.api.table.runtime.io.RowCsvInputFormatTest.{PATH, createTempFile, testRemovingTrailingCR} -import org.apache.flink.api.table.typeutils.RowTypeInfo +import org.apache.flink.api.java.typeutils.RowTypeInfo import org.apache.flink.configuration.Configuration import org.apache.flink.core.fs.{FileInputSplit, Path} import org.apache.flink.types.parser.{FieldParser, StringParser} import org.junit.Assert._ import org.junit.{Ignore, Test} +import scala.collection.JavaConversions._ class RowCsvInputFormatTest { @@ -50,7 +51,7 @@ class RowCsvInputFormatTest { val typeInfo = new RowTypeInfo(Seq( BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.DOUBLE_TYPE_INFO)) + BasicTypeInfo.DOUBLE_TYPE_INFO): _*) val format = new RowCsvInputFormat(PATH, typeInfo, "\n", "|") format.setLenient(false) @@ -77,9 +78,9 @@ class RowCsvInputFormatTest { result = format.nextRecord(result) assertNotNull(result) - assertEquals("this is", result.productElement(0)) - assertEquals(1, result.productElement(1)) - assertEquals(2.0, result.productElement(2)) + assertEquals("this is", result.getField(0)) + assertEquals(1, result.getField(1)) + assertEquals(2.0, result.getField(2)) try { result = format.nextRecord(result) @@ -91,15 +92,15 @@ class RowCsvInputFormatTest { result = format.nextRecord(result) assertNotNull(result) - assertEquals("a test", result.productElement(0)) - assertEquals(3, result.productElement(1)) - assertEquals(4.0, result.productElement(2)) + assertEquals("a test", result.getField(0)) + assertEquals(3, result.getField(1)) + assertEquals(4.0, result.getField(2)) result = format.nextRecord(result) assertNotNull(result) - assertEquals("#next", result.productElement(0)) - assertEquals(5, result.productElement(1)) - assertEquals(6.0, result.productElement(2)) + assertEquals("#next", result.getField(0)) + assertEquals(5, result.getField(1)) + assertEquals(6.0, result.getField(2)) result = format.nextRecord(result) assertNull(result) @@ -113,27 +114,27 @@ class RowCsvInputFormatTest { result = format.nextRecord(result) assertNotNull(result) - assertEquals("header1", result.productElement(0)) - assertNull(result.productElement(1)) - assertNull(result.productElement(2)) + assertEquals("header1", result.getField(0)) + assertNull(result.getField(1)) + assertNull(result.getField(2)) result = format.nextRecord(result) assertNotNull(result) - assertEquals("this is", result.productElement(0)) - assertEquals(1, result.productElement(1)) - assertEquals(2.0, result.productElement(2)) + assertEquals("this is", result.getField(0)) + assertEquals(1, result.getField(1)) + assertEquals(2.0, result.getField(2)) result = format.nextRecord(result) assertNotNull(result) - assertEquals("a test", result.productElement(0)) - assertEquals(3, result.productElement(1)) - assertEquals(4.0, result.productElement(2)) + assertEquals("a test", result.getField(0)) + assertEquals(3, result.getField(1)) + assertEquals(4.0, result.getField(2)) result = format.nextRecord(result) assertNotNull(result) - assertEquals("#next", result.productElement(0)) - assertEquals(5, result.productElement(1)) - assertEquals(6.0, result.productElement(2)) + assertEquals("#next", result.getField(0)) + assertEquals(5, result.getField(1)) + assertEquals(6.0, result.getField(2)) result = format.nextRecord(result) assertNull(result) } @@ -152,7 +153,7 @@ class RowCsvInputFormatTest { val typeInfo = new RowTypeInfo(Seq( BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.DOUBLE_TYPE_INFO)) + BasicTypeInfo.DOUBLE_TYPE_INFO): _*) val format = new RowCsvInputFormat(PATH, typeInfo, "\n", "|") format.setCommentPrefix("#") @@ -163,15 +164,15 @@ class RowCsvInputFormatTest { result = format.nextRecord(result) assertNotNull(result) - assertEquals("this is", result.productElement(0)) - assertEquals(1, result.productElement(1)) - assertEquals(2.0, result.productElement(2)) + assertEquals("this is", result.getField(0)) + assertEquals(1, result.getField(1)) + assertEquals(2.0, result.getField(2)) result = format.nextRecord(result) assertNotNull(result) - assertEquals("a test", result.productElement(0)) - assertEquals(3, result.productElement(1)) - assertEquals(4.0, result.productElement(2)) + assertEquals("a test", result.getField(0)) + assertEquals(3, result.getField(1)) + assertEquals(4.0, result.getField(2)) result = format.nextRecord(result) assertNull(result) @@ -191,7 +192,7 @@ class RowCsvInputFormatTest { val typeInfo = new RowTypeInfo(Seq( BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.DOUBLE_TYPE_INFO)) + BasicTypeInfo.DOUBLE_TYPE_INFO): _*) val format = new RowCsvInputFormat(PATH, typeInfo, "\n", "|") format.setCommentPrefix("//") @@ -202,15 +203,15 @@ class RowCsvInputFormatTest { result = format.nextRecord(result) assertNotNull(result) - assertEquals("this is", result.productElement(0)) - assertEquals(1, result.productElement(1)) - assertEquals(2.0, result.productElement(2)) + assertEquals("this is", result.getField(0)) + assertEquals(1, result.getField(1)) + assertEquals(2.0, result.getField(2)) result = format.nextRecord(result) assertNotNull(result) - assertEquals("a test", result.productElement(0)) - assertEquals(3, result.productElement(1)) - assertEquals(4.0, result.productElement(2)) + assertEquals("a test", result.getField(0)) + assertEquals(3, result.getField(1)) + assertEquals(4.0, result.getField(2)) result = format.nextRecord(result) assertNull(result) } @@ -224,7 +225,7 @@ class RowCsvInputFormatTest { val typeInfo = new RowTypeInfo(Seq( BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO, - BasicTypeInfo.STRING_TYPE_INFO)) + BasicTypeInfo.STRING_TYPE_INFO): _*) val format = new RowCsvInputFormat(PATH, typeInfo, "\n", "|") format.configure(new Configuration) @@ -234,21 +235,21 @@ class RowCsvInputFormatTest { result = format.nextRecord(result) assertNotNull(result) - assertEquals("abc", result.productElement(0)) - assertEquals("def", result.productElement(1)) - assertEquals("ghijk", result.productElement(2)) + assertEquals("abc", result.getField(0)) + assertEquals("def", result.getField(1)) + assertEquals("ghijk", result.getField(2)) result = format.nextRecord(result) assertNotNull(result) - assertEquals("abc", result.productElement(0)) - assertEquals("", result.productElement(1)) - assertEquals("hhg", result.productElement(2)) + assertEquals("abc", result.getField(0)) + assertEquals("", result.getField(1)) + assertEquals("hhg", result.getField(2)) result = format.nextRecord(result) assertNotNull(result) - assertEquals("", result.productElement(0)) - assertEquals("", result.productElement(1)) - assertEquals("", result.productElement(2)) + assertEquals("", result.getField(0)) + assertEquals("", result.getField(1)) + assertEquals("", result.getField(2)) result = format.nextRecord(result) assertNull(result) @@ -263,7 +264,7 @@ class RowCsvInputFormatTest { val typeInfo = new RowTypeInfo(Seq( BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO, - BasicTypeInfo.STRING_TYPE_INFO)) + BasicTypeInfo.STRING_TYPE_INFO): _*) val format = new RowCsvInputFormat(PATH, typeInfo, "\n", "|") format.configure(new Configuration) @@ -274,21 +275,21 @@ class RowCsvInputFormatTest { result = format.nextRecord(result) assertNotNull(result) - assertEquals("a|b|c", result.productElement(0)) - assertEquals("def", result.productElement(1)) - assertEquals("ghijk", result.productElement(2)) + assertEquals("a|b|c", result.getField(0)) + assertEquals("def", result.getField(1)) + assertEquals("ghijk", result.getField(2)) result = format.nextRecord(result) assertNotNull(result) - assertEquals("abc", result.productElement(0)) - assertEquals("", result.productElement(1)) - assertEquals("|hhg", result.productElement(2)) + assertEquals("abc", result.getField(0)) + assertEquals("", result.getField(1)) + assertEquals("|hhg", result.getField(2)) result = format.nextRecord(result) assertNotNull(result) - assertEquals("", result.productElement(0)) - assertEquals("", result.productElement(1)) - assertEquals("", result.productElement(2)) + assertEquals("", result.getField(0)) + assertEquals("", result.getField(1)) + assertEquals("", result.getField(2)) result = format.nextRecord(result) assertNull(result) @@ -303,7 +304,7 @@ class RowCsvInputFormatTest { val typeInfo = new RowTypeInfo(Seq( BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO, - BasicTypeInfo.STRING_TYPE_INFO)) + BasicTypeInfo.STRING_TYPE_INFO): _*) val format = new RowCsvInputFormat(PATH, typeInfo, "\n", "|") format.setFieldDelimiter("|-") @@ -314,21 +315,21 @@ class RowCsvInputFormatTest { result = format.nextRecord(result) assertNotNull(result) - assertEquals("abc", result.productElement(0)) - assertEquals("def", result.productElement(1)) - assertEquals("ghijk", result.productElement(2)) + assertEquals("abc", result.getField(0)) + assertEquals("def", result.getField(1)) + assertEquals("ghijk", result.getField(2)) result = format.nextRecord(result) assertNotNull(result) - assertEquals("abc", result.productElement(0)) - assertEquals("", result.productElement(1)) - assertEquals("hhg", result.productElement(2)) + assertEquals("abc", result.getField(0)) + assertEquals("", result.getField(1)) + assertEquals("hhg", result.getField(2)) result = format.nextRecord(result) assertNotNull(result) - assertEquals("", result.productElement(0)) - assertEquals("", result.productElement(1)) - assertEquals("", result.productElement(2)) + assertEquals("", result.getField(0)) + assertEquals("", result.getField(1)) + assertEquals("", result.getField(2)) result = format.nextRecord(result) assertNull(result) @@ -346,7 +347,7 @@ class RowCsvInputFormatTest { BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO)) + BasicTypeInfo.INT_TYPE_INFO): _*) val format = new RowCsvInputFormat(PATH, typeInfo, "\n", "|") @@ -358,19 +359,19 @@ class RowCsvInputFormatTest { result = format.nextRecord(result) assertNotNull(result) - assertEquals(111, result.productElement(0)) - assertEquals(222, result.productElement(1)) - assertEquals(333, result.productElement(2)) - assertEquals(444, result.productElement(3)) - assertEquals(555, result.productElement(4)) + assertEquals(111, result.getField(0)) + assertEquals(222, result.getField(1)) + assertEquals(333, result.getField(2)) + assertEquals(444, result.getField(3)) + assertEquals(555, result.getField(4)) result = format.nextRecord(result) assertNotNull(result) - assertEquals(666, result.productElement(0)) - assertEquals(777, result.productElement(1)) - assertEquals(888, result.productElement(2)) - assertEquals(999, result.productElement(3)) - assertEquals(0, result.productElement(4)) + assertEquals(666, result.getField(0)) + assertEquals(777, result.getField(1)) + assertEquals(888, result.getField(2)) + assertEquals(999, result.getField(3)) + assertEquals(0, result.getField(4)) result = format.nextRecord(result) assertNull(result) @@ -399,7 +400,7 @@ class RowCsvInputFormatTest { BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.LONG_TYPE_INFO, BasicTypeInfo.SHORT_TYPE_INFO, - BasicTypeInfo.STRING_TYPE_INFO)) + BasicTypeInfo.STRING_TYPE_INFO): _*) val format = new RowCsvInputFormat(PATH, rowTypeInfo = typeInfo, emptyColumnAsNull = true) format.setFieldDelimiter(",") @@ -411,7 +412,7 @@ class RowCsvInputFormatTest { for (i <- 0 until linesCnt) yield { result = format.nextRecord(result) - assertNull(result.productElement(i)) + assertNull(result.getField(i)) } // ensure no more rows @@ -430,7 +431,7 @@ class RowCsvInputFormatTest { BasicTypeInfo.DOUBLE_TYPE_INFO, BasicTypeInfo.DOUBLE_TYPE_INFO, BasicTypeInfo.DOUBLE_TYPE_INFO, - BasicTypeInfo.DOUBLE_TYPE_INFO)) + BasicTypeInfo.DOUBLE_TYPE_INFO): _*) val format = new RowCsvInputFormat(PATH, rowTypeInfo = typeInfo) format.setFieldDelimiter("|") @@ -441,19 +442,19 @@ class RowCsvInputFormatTest { result = format.nextRecord(result) assertNotNull(result) - assertEquals(11.1, result.productElement(0)) - assertEquals(22.2, result.productElement(1)) - assertEquals(33.3, result.productElement(2)) - assertEquals(44.4, result.productElement(3)) - assertEquals(55.5, result.productElement(4)) + assertEquals(11.1, result.getField(0)) + assertEquals(22.2, result.getField(1)) + assertEquals(33.3, result.getField(2)) + assertEquals(44.4, result.getField(3)) + assertEquals(55.5, result.getField(4)) result = format.nextRecord(result) assertNotNull(result) - assertEquals(66.6, result.productElement(0)) - assertEquals(77.7, result.productElement(1)) - assertEquals(88.8, result.productElement(2)) - assertEquals(99.9, result.productElement(3)) - assertEquals(0.0, result.productElement(4)) + assertEquals(66.6, result.getField(0)) + assertEquals(77.7, result.getField(1)) + assertEquals(88.8, result.getField(2)) + assertEquals(99.9, result.getField(3)) + assertEquals(0.0, result.getField(4)) result = format.nextRecord(result) assertNull(result) @@ -468,7 +469,7 @@ class RowCsvInputFormatTest { val typeInfo = new RowTypeInfo(Seq( BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO)) + BasicTypeInfo.INT_TYPE_INFO): _*) val format = new RowCsvInputFormat(PATH, rowTypeInfo = typeInfo) format.setFieldDelimiter("|") @@ -479,13 +480,13 @@ class RowCsvInputFormatTest { result = format.nextRecord(result) assertNotNull(result) - assertEquals(111, result.productElement(0)) - assertEquals(222, result.productElement(1)) + assertEquals(111, result.getField(0)) + assertEquals(222, result.getField(1)) result = format.nextRecord(result) assertNotNull(result) - assertEquals(666, result.productElement(0)) - assertEquals(777, result.productElement(1)) + assertEquals(666, result.getField(0)) + assertEquals(777, result.getField(1)) result = format.nextRecord(result) assertNull(result) @@ -502,7 +503,7 @@ class RowCsvInputFormatTest { val typeInfo: RowTypeInfo = new RowTypeInfo(Seq( BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO)) + BasicTypeInfo.INT_TYPE_INFO): _*) val format = new RowCsvInputFormat( PATH, @@ -516,15 +517,15 @@ class RowCsvInputFormatTest { result = format.nextRecord(result) assertNotNull(result) - assertEquals(111, result.productElement(0)) - assertEquals(444, result.productElement(1)) - assertEquals(888, result.productElement(2)) + assertEquals(111, result.getField(0)) + assertEquals(444, result.getField(1)) + assertEquals(888, result.getField(2)) result = format.nextRecord(result) assertNotNull(result) - assertEquals(0, result.productElement(0)) - assertEquals(777, result.productElement(1)) - assertEquals(333, result.productElement(2)) + assertEquals(0, result.getField(0)) + assertEquals(777, result.getField(1)) + assertEquals(333, result.getField(2)) result = format.nextRecord(result) assertNull(result) @@ -541,7 +542,7 @@ class RowCsvInputFormatTest { val typeInfo = new RowTypeInfo(Seq( BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO)) + BasicTypeInfo.INT_TYPE_INFO): _*) val format = new RowCsvInputFormat( PATH, @@ -555,15 +556,15 @@ class RowCsvInputFormatTest { result = format.nextRecord(result) assertNotNull(result) - assertEquals(111, result.productElement(0)) - assertEquals(444, result.productElement(1)) - assertEquals(888, result.productElement(2)) + assertEquals(111, result.getField(0)) + assertEquals(444, result.getField(1)) + assertEquals(888, result.getField(2)) result = format.nextRecord(result) assertNotNull(result) - assertEquals(0, result.productElement(0)) - assertEquals(777, result.productElement(1)) - assertEquals(333, result.productElement(2)) + assertEquals(0, result.getField(0)) + assertEquals(777, result.getField(1)) + assertEquals(333, result.getField(2)) result = format.nextRecord(result) assertNull(result) @@ -580,7 +581,7 @@ class RowCsvInputFormatTest { val typeInfo = new RowTypeInfo(Seq( BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO)) + BasicTypeInfo.INT_TYPE_INFO): _*) val format = new RowCsvInputFormat( PATH, @@ -594,15 +595,15 @@ class RowCsvInputFormatTest { result = format.nextRecord(result) assertNotNull(result) - assertEquals(111, result.productElement(0)) - assertEquals(444, result.productElement(1)) - assertEquals(888, result.productElement(2)) + assertEquals(111, result.getField(0)) + assertEquals(444, result.getField(1)) + assertEquals(888, result.getField(2)) result = format.nextRecord(result) assertNotNull(result) - assertEquals(0, result.productElement(0)) - assertEquals(777, result.productElement(1)) - assertEquals(333, result.productElement(2)) + assertEquals(0, result.getField(0)) + assertEquals(777, result.getField(1)) + assertEquals(333, result.getField(2)) result = format.nextRecord(result) assertNull(result) @@ -652,7 +653,7 @@ class RowCsvInputFormatTest { BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO, - BasicTypeInfo.DOUBLE_TYPE_INFO)) + BasicTypeInfo.DOUBLE_TYPE_INFO): _*) val format = new RowCsvInputFormat(PATH, typeInfo) format.setSkipFirstLineAsHeader(true) @@ -738,7 +739,7 @@ class RowCsvInputFormatTest { val typeInfo = new RowTypeInfo(Seq( BasicTypeInfo.STRING_TYPE_INFO, - BasicTypeInfo.STRING_TYPE_INFO)) + BasicTypeInfo.STRING_TYPE_INFO): _*) val inputFormat = new RowCsvInputFormat( new Path(tempFile.toURI.toString), @@ -753,8 +754,8 @@ class RowCsvInputFormatTest { inputFormat.open(splits(0)) val record = inputFormat.nextRecord(new Row(2)) - assertEquals("20:41:52-1-3-2015", record.productElement(0)) - assertEquals("Blahblah ", record.productElement(1)) + assertEquals("20:41:52-1-3-2015", record.getField(0)) + assertEquals("Blahblah ", record.getField(1)) } @Test @@ -770,7 +771,7 @@ class RowCsvInputFormatTest { val typeInfo = new RowTypeInfo(Seq( BasicTypeInfo.STRING_TYPE_INFO, - BasicTypeInfo.STRING_TYPE_INFO)) + BasicTypeInfo.STRING_TYPE_INFO): _*) val inputFormat = new RowCsvInputFormat( new Path(tempFile.toURI.toString), @@ -784,8 +785,8 @@ class RowCsvInputFormatTest { inputFormat.open(splits(0)) val record = inputFormat.nextRecord(new Row(2)) - assertEquals("\\\"Hello\\\" World", record.productElement(0)) - assertEquals("We are\\\" young", record.productElement(1)) + assertEquals("\\\"Hello\\\" World", record.getField(0)) + assertEquals("We are\\\" young", record.getField(1)) } @Test @@ -799,7 +800,7 @@ class RowCsvInputFormatTest { SqlTimeTypeInfo.DATE, SqlTimeTypeInfo.TIME, SqlTimeTypeInfo.TIMESTAMP, - SqlTimeTypeInfo.TIMESTAMP)) + SqlTimeTypeInfo.TIMESTAMP): _*) val format = new RowCsvInputFormat(PATH, rowTypeInfo = typeInfo) format.setFieldDelimiter("|") @@ -810,17 +811,17 @@ class RowCsvInputFormatTest { result = format.nextRecord(result) assertNotNull(result) - assertEquals(Date.valueOf("1990-10-14"), result.productElement(0)) - assertEquals(Time.valueOf("02:42:25"), result.productElement(1)) - assertEquals(Timestamp.valueOf("1990-10-14 02:42:25.123"), result.productElement(2)) - assertEquals(Timestamp.valueOf("1990-01-04 02:02:05"), result.productElement(3)) + assertEquals(Date.valueOf("1990-10-14"), result.getField(0)) + assertEquals(Time.valueOf("02:42:25"), result.getField(1)) + assertEquals(Timestamp.valueOf("1990-10-14 02:42:25.123"), result.getField(2)) + assertEquals(Timestamp.valueOf("1990-01-04 02:02:05"), result.getField(3)) result = format.nextRecord(result) assertNotNull(result) - assertEquals(Date.valueOf("1990-10-14"), result.productElement(0)) - assertEquals(Time.valueOf("02:42:25"), result.productElement(1)) - assertEquals(Timestamp.valueOf("1990-10-14 02:42:25.123"), result.productElement(2)) - assertEquals(Timestamp.valueOf("1990-01-04 02:02:05.3"), result.productElement(3)) + assertEquals(Date.valueOf("1990-10-14"), result.getField(0)) + assertEquals(Time.valueOf("02:42:25"), result.getField(1)) + assertEquals(Timestamp.valueOf("1990-10-14 02:42:25.123"), result.getField(2)) + assertEquals(Timestamp.valueOf("1990-01-04 02:02:05.3"), result.getField(3)) result = format.nextRecord(result) assertNull(result) @@ -862,7 +863,7 @@ object RowCsvInputFormatTest { wrt.write(fileContent) wrt.close() - val typeInfo = new RowTypeInfo(Seq(BasicTypeInfo.STRING_TYPE_INFO)) + val typeInfo = new RowTypeInfo(Seq(BasicTypeInfo.STRING_TYPE_INFO): _*) val inputFormat = new RowCsvInputFormat(new Path(tempFile.toURI.toString), typeInfo) inputFormat.configure(new Configuration) @@ -873,10 +874,10 @@ object RowCsvInputFormatTest { var result = inputFormat.nextRecord(new Row(1)) assertNotNull("Expecting to not return null", result) - assertEquals(FIRST_PART, result.productElement(0)) + assertEquals(FIRST_PART, result.getField(0)) result = inputFormat.nextRecord(result) assertNotNull("Expecting to not return null", result) - assertEquals(SECOND_PART, result.productElement(0)) + assertEquals(SECOND_PART, result.getField(0)) } } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/typeutils/RowComparatorTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/typeutils/RowComparatorTest.scala index 557db3abba594..c441a4f8c0795 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/typeutils/RowComparatorTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/typeutils/RowComparatorTest.scala @@ -23,14 +23,14 @@ import org.apache.flink.api.common.typeinfo.BasicTypeInfo import org.apache.flink.api.common.typeutils.{ComparatorTestBase, TypeComparator, TypeSerializer} import org.apache.flink.api.java.tuple import org.apache.flink.api.java.typeutils.{TupleTypeInfo, TypeExtractor} -import org.apache.flink.api.table.Row +import org.apache.flink.types.Row import org.apache.flink.api.table.typeutils.RowComparatorTest.MyPojo +import org.apache.flink.api.java.typeutils.{RowTypeInfo => RowTypeInfoNew} import org.junit.Assert._ class RowComparatorTest extends ComparatorTestBase[Row] { - val typeInfo = new RowTypeInfo( - Array( + val typeInfo = new RowTypeInfoNew( BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.DOUBLE_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO, @@ -38,7 +38,7 @@ class RowComparatorTest extends ComparatorTestBase[Row] { BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.BOOLEAN_TYPE_INFO, BasicTypeInfo.SHORT_TYPE_INFO), - TypeExtractor.createTypeInfo(classOf[MyPojo]))) + TypeExtractor.createTypeInfo(classOf[MyPojo])) val testPojo1 = new MyPojo() // TODO we cannot test null here as PojoComparator has no support for null keys @@ -66,12 +66,12 @@ class RowComparatorTest extends ComparatorTestBase[Row] { ) override protected def deepEquals(message: String, should: Row, is: Row): Unit = { - val arity = should.productArity - assertEquals(message, arity, is.productArity) + val arity = should.getArity + assertEquals(message, arity, is.getArity) var index = 0 while (index < arity) { - val copiedValue: Any = should.productElement(index) - val element: Any = is.productElement(index) + val copiedValue: Any = should.getField(index) + val element: Any = is.getField(index) assertEquals(message, element, copiedValue) index += 1 } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/typeutils/RowComparatorWithManyFieldsTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/typeutils/RowComparatorWithManyFieldsTest.scala index 33715c1835f85..645e18d43ac88 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/typeutils/RowComparatorWithManyFieldsTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/typeutils/RowComparatorWithManyFieldsTest.scala @@ -21,7 +21,8 @@ package org.apache.flink.api.table.typeutils import org.apache.flink.api.common.ExecutionConfig import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, TypeInformation} import org.apache.flink.api.common.typeutils.{ComparatorTestBase, TypeComparator, TypeSerializer} -import org.apache.flink.api.table.Row +import org.apache.flink.types.Row +import org.apache.flink.api.java.typeutils.{RowTypeInfo => RowTypeInfoNew} import org.apache.flink.util.Preconditions import org.junit.Assert._ @@ -34,7 +35,7 @@ class RowComparatorWithManyFieldsTest extends ComparatorTestBase[Row] { for (i <- 0 until numberOfFields) { fieldTypes(i) = BasicTypeInfo.STRING_TYPE_INFO } - val typeInfo = new RowTypeInfo(fieldTypes) + val typeInfo = new RowTypeInfoNew(fieldTypes: _*) val data: Array[Row] = Array( createRow(Array(null, "b0", "c0", "d0", "e0", "f0", "g0", "h0", "i0", "j0")), @@ -44,12 +45,12 @@ class RowComparatorWithManyFieldsTest extends ComparatorTestBase[Row] { ) override protected def deepEquals(message: String, should: Row, is: Row): Unit = { - val arity = should.productArity - assertEquals(message, arity, is.productArity) + val arity = should.getArity + assertEquals(message, arity, is.getArity) var index = 0 while (index < arity) { - val copiedValue: Any = should.productElement(index) - val element: Any = is.productElement(index) + val copiedValue: Any = should.getField(index) + val element: Any = is.getField(index) assertEquals(message, element, copiedValue) index += 1 } diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/typeutils/RowSerializerTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/typeutils/RowSerializerTest.scala index b52dd4d63762c..28dfbc03248c3 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/typeutils/RowSerializerTest.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/typeutils/RowSerializerTest.scala @@ -19,11 +19,11 @@ package org.apache.flink.api.table.typeutils import org.apache.flink.api.common.ExecutionConfig -import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, TypeInformation} +import org.apache.flink.api.common.typeinfo.BasicTypeInfo import org.apache.flink.api.common.typeutils.{SerializerTestInstance, TypeSerializer} import org.apache.flink.api.java.tuple import org.apache.flink.api.java.typeutils.{TupleTypeInfo, TypeExtractor} -import org.apache.flink.api.table.Row +import org.apache.flink.types.Row import org.apache.flink.api.table.typeutils.RowSerializerTest.MyPojo import org.junit.Assert._ import org.junit.Test @@ -36,12 +36,12 @@ class RowSerializerTest { extends SerializerTestInstance[Row](serializer, classOf[Row], -1, testData: _*) { override protected def deepEquals(message: String, should: Row, is: Row): Unit = { - val arity = should.productArity - assertEquals(message, arity, is.productArity) + val arity = should.getArity + assertEquals(message, arity, is.getArity) var index = 0 while (index < arity) { - val copiedValue: Any = should.productElement(index) - val element: Any = is.productElement(index) + val copiedValue: Any = should.getField(index) + val element: Any = is.getField(index) assertEquals(message, element, copiedValue) index += 1 } @@ -50,8 +50,8 @@ class RowSerializerTest { @Test def testRowSerializer(): Unit = { - val rowInfo: TypeInformation[Row] = new RowTypeInfo( - Seq(BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO)) + val rowInfo = new org.apache.flink.api.java.typeutils.RowTypeInfo( + BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO) val row1 = new Row(2) row1.setField(0, 1) @@ -72,7 +72,7 @@ class RowSerializerTest { @Test def testLargeRowSerializer(): Unit = { - val rowInfo: TypeInformation[Row] = new RowTypeInfo(Seq( + val rowInfo = new org.apache.flink.api.java.typeutils.RowTypeInfo( BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO, @@ -85,7 +85,7 @@ class RowSerializerTest { BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.STRING_TYPE_INFO)) + BasicTypeInfo.STRING_TYPE_INFO) val row = new Row(13) row.setField(0, 2) @@ -112,8 +112,7 @@ class RowSerializerTest { @Test def testRowSerializerWithComplexTypes(): Unit = { - val rowInfo = new RowTypeInfo( - Array( + val rowInfo = new org.apache.flink.api.java.typeutils.RowTypeInfo( BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.DOUBLE_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO, @@ -121,7 +120,7 @@ class RowSerializerTest { BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.BOOLEAN_TYPE_INFO, BasicTypeInfo.SHORT_TYPE_INFO), - TypeExtractor.createTypeInfo(classOf[MyPojo]))) + TypeExtractor.createTypeInfo(classOf[MyPojo])) val testPojo1 = new MyPojo() testPojo1.name = null diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/utils/UserDefinedTableFunctions.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/utils/UserDefinedTableFunctions.scala index 3da3857f33ae5..4291b2959b134 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/utils/UserDefinedTableFunctions.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/utils/UserDefinedTableFunctions.scala @@ -21,9 +21,9 @@ import java.lang.Boolean import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, TypeInformation} import org.apache.flink.api.java.tuple.Tuple3 -import org.apache.flink.api.table.Row +import org.apache.flink.types.Row import org.apache.flink.api.table.functions.TableFunction -import org.apache.flink.api.table.typeutils.RowTypeInfo +import org.apache.flink.api.java.typeutils.RowTypeInfo case class SimpleUser(name: String, age: Int) @@ -66,8 +66,8 @@ class TableFunc2 extends TableFunction[Row] { } override def getResultType: TypeInformation[Row] = { - new RowTypeInfo(Seq(BasicTypeInfo.STRING_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO)) + new RowTypeInfo(BasicTypeInfo.STRING_TYPE_INFO, + BasicTypeInfo.INT_TYPE_INFO) } } From 9a0ec915fdb37d5d09464442460fa92ecb95da40 Mon Sep 17 00:00:00 2001 From: tonycox Date: Wed, 14 Dec 2016 12:12:06 +0400 Subject: [PATCH 7/7] [FLINK-5189] Delete Row and its related classes from table api --- .../org/apache/flink/api/table/Row.scala | 38 - .../table/runtime/io/RowCsvInputFormat.scala | 181 ---- .../table/typeutils/NullAwareComparator.scala | 218 ----- .../api/table/typeutils/NullMaskUtils.scala | 98 -- .../api/table/typeutils/RowComparator.scala | 426 --------- .../api/table/typeutils/RowSerializer.scala | 209 ----- .../api/table/typeutils/RowTypeInfo.scala | 108 --- .../runtime/io/RowCsvInputFormatTest.scala | 883 ------------------ .../table/typeutils/RowComparatorTest.scala | 136 --- .../RowComparatorWithManyFieldsTest.scala | 83 -- .../table/typeutils/RowSerializerTest.scala | 193 ---- 11 files changed, 2573 deletions(-) delete mode 100644 flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/Row.scala delete mode 100644 flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/io/RowCsvInputFormat.scala delete mode 100644 flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/typeutils/NullAwareComparator.scala delete mode 100644 flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/typeutils/NullMaskUtils.scala delete mode 100644 flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/typeutils/RowComparator.scala delete mode 100644 flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/typeutils/RowSerializer.scala delete mode 100644 flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/typeutils/RowTypeInfo.scala delete mode 100644 flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/runtime/io/RowCsvInputFormatTest.scala delete mode 100644 flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/typeutils/RowComparatorTest.scala delete mode 100644 flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/typeutils/RowComparatorWithManyFieldsTest.scala delete mode 100644 flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/typeutils/RowSerializerTest.scala diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/Row.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/Row.scala deleted file mode 100644 index e3baab3f9a3ca..0000000000000 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/Row.scala +++ /dev/null @@ -1,38 +0,0 @@ -/* - * 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.table - -/** - * This is used for executing Table API operations. We use manually generated - * TypeInfo to check the field types and create serializers and comparators. - */ -class Row(arity: Int) extends Product { - - private val fields = new Array[Any](arity) - - def productArity = fields.length - - def productElement(i: Int): Any = fields(i) - - def setField(i: Int, value: Any): Unit = fields(i) = value - - def canEqual(that: Any) = false - - override def toString = fields.mkString(",") - -} diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/io/RowCsvInputFormat.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/io/RowCsvInputFormat.scala deleted file mode 100644 index 8ea7f7acc3ab9..0000000000000 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/io/RowCsvInputFormat.scala +++ /dev/null @@ -1,181 +0,0 @@ -/* - * 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.table.runtime.io - -import org.apache.flink.annotation.Internal -import org.apache.flink.api.common.io.ParseException -import org.apache.flink.api.java.io.CsvInputFormat -import org.apache.flink.api.java.io.CsvInputFormat.{DEFAULT_FIELD_DELIMITER, DEFAULT_LINE_DELIMITER, createDefaultMask, toBooleanMask} -import org.apache.flink.types.Row -import org.apache.flink.api.table.runtime.io.RowCsvInputFormat.extractTypeClasses -import org.apache.flink.api.java.typeutils.RowTypeInfo -import org.apache.flink.core.fs.Path -import org.apache.flink.types.parser.FieldParser -import org.apache.flink.types.parser.FieldParser.ParseErrorState - -@Internal -@SerialVersionUID(1L) -class RowCsvInputFormat( - filePath: Path, - rowTypeInfo: RowTypeInfo, - lineDelimiter: String = DEFAULT_LINE_DELIMITER, - fieldDelimiter: String = DEFAULT_FIELD_DELIMITER, - includedFieldsMask: Array[Boolean] = null, - emptyColumnAsNull: Boolean = false) - extends CsvInputFormat[Row](filePath) { - - if (rowTypeInfo.getArity == 0) { - throw new IllegalArgumentException("Row arity must be greater than 0.") - } - private val arity = rowTypeInfo.getArity - private lazy val defaultFieldMask = createDefaultMask(arity) - private val fieldsMask = Option(includedFieldsMask).getOrElse(defaultFieldMask) - - // prepare CsvInputFormat - setDelimiter(lineDelimiter) - setFieldDelimiter(fieldDelimiter) - setFieldsGeneric(fieldsMask, extractTypeClasses(rowTypeInfo)) - - def this( - filePath: Path, - rowTypeInfo: RowTypeInfo, - lineDelimiter: String, - fieldDelimiter: String, - includedFieldsMask: Array[Int]) { - this( - filePath, - rowTypeInfo, - lineDelimiter, - fieldDelimiter, - if (includedFieldsMask == null) { - null - } else { - toBooleanMask(includedFieldsMask) - }) - } - - def this( - filePath: Path, - rowTypeInfo: RowTypeInfo, - includedFieldsMask: Array[Int]) { - this( - filePath, - rowTypeInfo, - DEFAULT_LINE_DELIMITER, - DEFAULT_FIELD_DELIMITER, - includedFieldsMask) - } - - def fillRecord(reuse: Row, parsedValues: Array[AnyRef]): Row = { - val reuseRow = if (reuse == null) { - new Row(arity) - } else { - reuse - } - var i: Int = 0 - while (i < parsedValues.length) { - reuse.setField(i, parsedValues(i)) - i += 1 - } - reuseRow - } - - @throws[ParseException] - override protected def parseRecord( - holders: Array[AnyRef], - bytes: Array[Byte], - offset: Int, - numBytes: Int) - : Boolean = { - val fieldDelimiter = this.getFieldDelimiter - val fieldIncluded: Array[Boolean] = this.fieldIncluded - - var startPos = offset - val limit = offset + numBytes - - var field = 0 - var output = 0 - while (field < fieldIncluded.length) { - - // check valid start position - if (startPos >= limit) { - if (isLenient) { - return false - } else { - throw new ParseException("Row too short: " + new String(bytes, offset, numBytes)) - } - } - - if (fieldIncluded(field)) { - // parse field - val parser: FieldParser[AnyRef] = this.getFieldParsers()(output) - .asInstanceOf[FieldParser[AnyRef]] - val latestValidPos = startPos - startPos = parser.resetErrorStateAndParse( - bytes, - startPos, - limit, - fieldDelimiter, - holders(output)) - - if (!isLenient && (parser.getErrorState ne ParseErrorState.NONE)) { - // the error state EMPTY_COLUMN is ignored - if (parser.getErrorState ne ParseErrorState.EMPTY_COLUMN) { - throw new ParseException(s"Parsing error for column $field of row '" - + new String(bytes, offset, numBytes) - + s"' originated by ${parser.getClass.getSimpleName}: ${parser.getErrorState}.") - } - } - holders(output) = parser.getLastResult - - // check parse result: - // the result is null if it is invalid - // or empty with emptyColumnAsNull enabled - if (startPos < 0 || - (emptyColumnAsNull && (parser.getErrorState eq ParseErrorState.EMPTY_COLUMN))) { - holders(output) = null - startPos = skipFields(bytes, latestValidPos, limit, fieldDelimiter) - } - output += 1 - } else { - // skip field - startPos = skipFields(bytes, startPos, limit, fieldDelimiter) - } - - // check if something went wrong - if (startPos < 0) { - throw new ParseException(s"Unexpected parser position for column $field of row '" - + new String(bytes, offset, numBytes) + "'") - } - - field += 1 - } - true - } -} - -object RowCsvInputFormat { - - private def extractTypeClasses(rowTypeInfo: RowTypeInfo): Array[Class[_]] = { - val classes = for (i <- 0 until rowTypeInfo.getArity) - yield rowTypeInfo.getTypeAt(i).getTypeClass - classes.toArray - } - -} diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/typeutils/NullAwareComparator.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/typeutils/NullAwareComparator.scala deleted file mode 100644 index 86a768dd401ca..0000000000000 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/typeutils/NullAwareComparator.scala +++ /dev/null @@ -1,218 +0,0 @@ -/* - * 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.table.typeutils - -import org.apache.flink.api.common.typeutils.{CompositeTypeComparator, TypeComparator} -import org.apache.flink.core.memory.{DataInputView, DataOutputView, MemorySegment} - -import scala.collection.JavaConversions._ -import scala.collection.mutable.ArrayBuffer - -/** - * Null-aware comparator that wraps a comparator which does not support null references. - * - * NOTE: This class assumes to be used within a composite type comparator (such - * as [[RowComparator]]) that handles serialized comparison. - */ -class NullAwareComparator[T]( - val wrappedComparator: TypeComparator[T], - val order: Boolean) - extends TypeComparator[T] { - - // number of flat fields - private val flatFields = wrappedComparator.getFlatComparators.length - - // stores the null for reference comparison - private var nullReference = false - - override def hash(record: T): Int = { - if (record != null) { - wrappedComparator.hash(record) - } - else { - 0 - } - } - - override def getNormalizeKeyLen: Int = { - val len = wrappedComparator.getNormalizeKeyLen - if (len == Integer.MAX_VALUE) { - Integer.MAX_VALUE - } - else { - len + 1 // add one for a null byte - } - } - - override def putNormalizedKey( - record: T, - target: MemorySegment, - offset: Int, - numBytes: Int) - : Unit = { - if (numBytes > 0) { - // write a null byte with padding - if (record == null) { - target.putBoolean(offset, false) - // write padding - var j = 0 - while (j < numBytes - 1) { - target.put(offset + 1 + j, 0.toByte) - j += 1 - } - } - // write a non-null byte with key - else { - target.putBoolean(offset, true) - // write key - wrappedComparator.putNormalizedKey(record, target, offset + 1, numBytes - 1) - } - } - } - - override def invertNormalizedKey(): Boolean = wrappedComparator.invertNormalizedKey() - - override def supportsSerializationWithKeyNormalization(): Boolean = false - - override def writeWithKeyNormalization(record: T, target: DataOutputView): Unit = - throw new UnsupportedOperationException("Record serialization with leading normalized keys" + - " not supported.") - - override def readWithKeyDenormalization(reuse: T, source: DataInputView): T = - throw new UnsupportedOperationException("Record deserialization with leading normalized keys" + - " not supported.") - - override def isNormalizedKeyPrefixOnly(keyBytes: Int): Boolean = - wrappedComparator.isNormalizedKeyPrefixOnly(keyBytes - 1) - - override def setReference(toCompare: T): Unit = { - if (toCompare == null) { - nullReference = true - } - else { - nullReference = false - wrappedComparator.setReference(toCompare) - } - } - - override def compare(first: T, second: T): Int = { - // both values are null -> equality - if (first == null && second == null) { - 0 - } - // first value is null -> inequality - // but order is considered - else if (first == null) { - if (order) -1 else 1 - } - // second value is null -> inequality - // but order is considered - else if (second == null) { - if (order) 1 else -1 - } - // no null values - else { - wrappedComparator.compare(first, second) - } - } - - override def compareToReference(referencedComparator: TypeComparator[T]): Int = { - val otherComparator = referencedComparator.asInstanceOf[NullAwareComparator[T]] - val otherNullReference = otherComparator.nullReference - // both values are null -> equality - if (nullReference && otherNullReference) { - 0 - } - // first value is null -> inequality - // but order is considered - else if (nullReference) { - if (order) 1 else -1 - } - // second value is null -> inequality - // but order is considered - else if (otherNullReference) { - if (order) -1 else 1 - } - // no null values - else { - wrappedComparator.compareToReference(otherComparator.wrappedComparator) - } - } - - override def supportsNormalizedKey(): Boolean = wrappedComparator.supportsNormalizedKey() - - override def equalToReference(candidate: T): Boolean = { - // both values are null - if (candidate == null && nullReference) { - true - } - // one value is null - else if (candidate == null || nullReference) { - false - } - // no null value - else { - wrappedComparator.equalToReference(candidate) - } - } - - override def duplicate(): TypeComparator[T] = { - new NullAwareComparator[T](wrappedComparator.duplicate(), order) - } - - override def extractKeys(record: Any, target: Array[AnyRef], index: Int): Int = { - if (record == null) { - var i = 0 - while (i < flatFields) { - target(index + i) = null - i += 1 - } - flatFields - } - else { - wrappedComparator.extractKeys(record, target, index) - } - } - - - override def getFlatComparators: Array[TypeComparator[_]] = { - // determine the flat comparators and wrap them again in null-aware comparators - val flatComparators = new ArrayBuffer[TypeComparator[_]]() - wrappedComparator match { - case ctc: CompositeTypeComparator[_] => ctc.getFlatComparator(flatComparators) - case c: TypeComparator[_] => flatComparators += c - } - val wrappedComparators = flatComparators.map { c => - new NullAwareComparator[Any](c.asInstanceOf[TypeComparator[Any]], order) - } - wrappedComparators.toArray[TypeComparator[_]] - } - - /** - * This method is not implemented here. It must be implemented by the comparator this class - * is contained in (e.g. RowComparator). - * - * @param firstSource The input view containing the first record. - * @param secondSource The input view containing the second record. - * @return An integer defining the oder among the objects in the same way as - * { @link java.util.Comparator#compare(Object, Object)}. - */ - override def compareSerialized(firstSource: DataInputView, secondSource: DataInputView): Int = - throw new UnsupportedOperationException("Comparator does not support null-aware serialized " + - "comparision.") -} diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/typeutils/NullMaskUtils.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/typeutils/NullMaskUtils.scala deleted file mode 100644 index dcdc775780aba..0000000000000 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/typeutils/NullMaskUtils.scala +++ /dev/null @@ -1,98 +0,0 @@ -/* - * 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.table.typeutils - -import org.apache.flink.api.table.Row -import org.apache.flink.core.memory.{DataInputView, DataOutputView} - -object NullMaskUtils { - - def writeNullMask(len: Int, value: Row, target: DataOutputView): Unit = { - var b = 0x00 - var bytePos = 0 - - var fieldPos = 0 - var numPos = 0 - while (fieldPos < len) { - b = 0x00 - // set bits in byte - bytePos = 0 - numPos = Math.min(8, len - fieldPos) - while (bytePos < numPos) { - b = b << 1 - // set bit if field is null - if(value.productElement(fieldPos + bytePos) == null) { - b |= 0x01 - } - bytePos += 1 - } - fieldPos += numPos - // shift bits if last byte is not completely filled - b <<= (8 - bytePos) - // write byte - target.writeByte(b) - } - } - - def readIntoNullMask(len: Int, source: DataInputView, nullMask: Array[Boolean]): Unit = { - var b = 0x00 - var bytePos = 0 - - var fieldPos = 0 - var numPos = 0 - while (fieldPos < len) { - // read byte - b = source.readUnsignedByte() - bytePos = 0 - numPos = Math.min(8, len - fieldPos) - while (bytePos < numPos) { - nullMask(fieldPos + bytePos) = (b & 0x80) > 0 - b = b << 1 - bytePos += 1 - } - fieldPos += numPos - } - } - - def readIntoAndCopyNullMask( - len: Int, - source: DataInputView, - target: DataOutputView, - nullMask: Array[Boolean]): Unit = { - var b = 0x00 - var bytePos = 0 - - var fieldPos = 0 - var numPos = 0 - while (fieldPos < len) { - // read byte - b = source.readUnsignedByte() - // copy byte - target.writeByte(b) - bytePos = 0 - numPos = Math.min(8, len - fieldPos) - while (bytePos < numPos) { - nullMask(fieldPos + bytePos) = (b & 0x80) > 0 - b = b << 1 - bytePos += 1 - } - fieldPos += numPos - } - } - -} diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/typeutils/RowComparator.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/typeutils/RowComparator.scala deleted file mode 100644 index a60eba7ac8fce..0000000000000 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/typeutils/RowComparator.scala +++ /dev/null @@ -1,426 +0,0 @@ -/* - * 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.table.typeutils - -import java.util - -import org.apache.flink.api.common.typeutils.{CompositeTypeComparator, TypeComparator, TypeSerializer} -import org.apache.flink.api.java.typeutils.runtime.TupleComparatorBase -import org.apache.flink.api.table.Row -import org.apache.flink.api.table.typeutils.NullMaskUtils.readIntoNullMask -import org.apache.flink.api.table.typeutils.RowComparator.{createAuxiliaryFields, makeNullAware} -import org.apache.flink.core.memory.{DataInputView, DataOutputView, MemorySegment} -import org.apache.flink.types.KeyFieldOutOfBoundsException - - -/** - * Comparator for [[Row]]. - */ -class RowComparator private ( - /** the number of fields of the Row */ - val numberOfFields: Int, - /** key positions describe which fields are keys in what order */ - val keyPositions: Array[Int], - /** null-aware comparators for the key fields, in the same order as the key fields */ - val comparators: Array[NullAwareComparator[Any]], - /** serializers to deserialize the first n fields for comparison */ - val serializers: Array[TypeSerializer[Any]], - /** auxiliary fields for normalized key support */ - private val auxiliaryFields: (Array[Int], Int, Int, Boolean)) - extends CompositeTypeComparator[Row] with Serializable { - - // null masks for serialized comparison - private val nullMask1 = new Array[Boolean](numberOfFields) - private val nullMask2 = new Array[Boolean](numberOfFields) - - // cache for the deserialized key field objects - @transient - private lazy val deserializedKeyFields1: Array[Any] = instantiateDeserializationFields() - - @transient - private lazy val deserializedKeyFields2: Array[Any] = instantiateDeserializationFields() - - // create auxiliary fields - private val normalizedKeyLengths: Array[Int] = auxiliaryFields._1 - private val numLeadingNormalizableKeys: Int = auxiliaryFields._2 - private val normalizableKeyPrefixLen: Int = auxiliaryFields._3 - private val invertNormKey: Boolean = auxiliaryFields._4 - - /** - * Intermediate constructor for creating auxiliary fields. - */ - def this( - numberOfFields: Int, - keyPositions: Array[Int], - comparators: Array[NullAwareComparator[Any]], - serializers: Array[TypeSerializer[Any]]) = { - this( - numberOfFields, - keyPositions, - comparators, - serializers, - createAuxiliaryFields(keyPositions, comparators)) - } - - /** - * General constructor for RowComparator. - * - * @param numberOfFields the number of fields of the Row - * @param keyPositions key positions describe which fields are keys in what order - * @param comparators non-null-aware comparators for the key fields, in the same order as - * the key fields - * @param serializers serializers to deserialize the first n fields for comparison - * @param orders sorting orders for the fields - */ - def this( - numberOfFields: Int, - keyPositions: Array[Int], - comparators: Array[TypeComparator[Any]], - serializers: Array[TypeSerializer[Any]], - orders: Array[Boolean]) = { - this( - numberOfFields, - keyPositions, - makeNullAware(comparators, orders), - serializers) - } - - private def instantiateDeserializationFields(): Array[Any] = { - val newFields = new Array[Any](serializers.length) - var i = 0 - while (i < serializers.length) { - newFields(i) = serializers(i).createInstance() - i += 1 - } - newFields - } - - // -------------------------------------------------------------------------------------------- - // Comparator Methods - // -------------------------------------------------------------------------------------------- - - override def compareToReference(referencedComparator: TypeComparator[Row]): Int = { - val other: RowComparator = referencedComparator.asInstanceOf[RowComparator] - var i = 0 - try { - while (i < keyPositions.length) { - val comparator = comparators(i) - val otherComparator = other.comparators(i) - - val cmp = comparator.compareToReference(otherComparator) - if (cmp != 0) { - return cmp - } - i = i + 1 - } - 0 - } - catch { - case iobex: IndexOutOfBoundsException => - throw new KeyFieldOutOfBoundsException(keyPositions(i)) - } - } - - override def compareSerialized(firstSource: DataInputView, secondSource: DataInputView): Int = { - val len = serializers.length - val keyLen = keyPositions.length - - readIntoNullMask(numberOfFields, firstSource, nullMask1) - readIntoNullMask(numberOfFields, secondSource, nullMask2) - - // deserialize - var i = 0 - while (i < len) { - val serializer = serializers(i) - - // deserialize field 1 - if (!nullMask1(i)) { - deserializedKeyFields1(i) = serializer.deserialize(deserializedKeyFields1(i), firstSource) - } - - // deserialize field 2 - if (!nullMask2(i)) { - deserializedKeyFields2(i) = serializer.deserialize(deserializedKeyFields2(i), secondSource) - } - - i += 1 - } - - // compare - i = 0 - while (i < keyLen) { - val keyPos = keyPositions(i) - val comparator = comparators(i) - - val isNull1 = nullMask1(keyPos) - val isNull2 = nullMask2(keyPos) - - var cmp = 0 - // both values are null -> equality - if (isNull1 && isNull2) { - cmp = 0 - } - // first value is null -> inequality - else if (isNull1) { - cmp = comparator.compare(null, deserializedKeyFields2(keyPos)) - } - // second value is null -> inequality - else if (isNull2) { - cmp = comparator.compare(deserializedKeyFields1(keyPos), null) - } - // no null values - else { - cmp = comparator.compare(deserializedKeyFields1(keyPos), deserializedKeyFields2(keyPos)) - } - - if (cmp != 0) { - return cmp - } - - i += 1 - } - 0 - } - - override def supportsNormalizedKey(): Boolean = numLeadingNormalizableKeys > 0 - - override def getNormalizeKeyLen: Int = normalizableKeyPrefixLen - - override def isNormalizedKeyPrefixOnly(keyBytes: Int): Boolean = - numLeadingNormalizableKeys < keyPositions.length || - normalizableKeyPrefixLen == Integer.MAX_VALUE || - normalizableKeyPrefixLen > keyBytes - - override def invertNormalizedKey(): Boolean = invertNormKey - - override def supportsSerializationWithKeyNormalization(): Boolean = false - - override def writeWithKeyNormalization(record: Row, target: DataOutputView): Unit = - throw new UnsupportedOperationException("Record serialization with leading normalized keys " + - "not supported.") - - override def readWithKeyDenormalization(reuse: Row, source: DataInputView): Row = - throw new UnsupportedOperationException("Record deserialization with leading normalized keys " + - "not supported.") - - override def duplicate(): TypeComparator[Row] = { - // copy comparator and serializer factories - val comparatorsCopy = comparators.map(_.duplicate().asInstanceOf[NullAwareComparator[Any]]) - val serializersCopy = serializers.map(_.duplicate()) - - new RowComparator( - numberOfFields, - keyPositions, - comparatorsCopy, - serializersCopy, - auxiliaryFields) - } - - override def hash(value: Row): Int = { - var code: Int = 0 - var i = 0 - try { - while(i < keyPositions.length) { - code *= TupleComparatorBase.HASH_SALT(i & 0x1F) - val element = value.productElement(keyPositions(i)) // element can be null - code += comparators(i).hash(element) - i += 1 - } - } catch { - case iobex: IndexOutOfBoundsException => - throw new KeyFieldOutOfBoundsException(keyPositions(i)) - } - code - } - - override def setReference(toCompare: Row) { - var i = 0 - try { - while(i < keyPositions.length) { - val comparator = comparators(i) - val element = toCompare.productElement(keyPositions(i)) - comparator.setReference(element) // element can be null - i += 1 - } - } catch { - case iobex: IndexOutOfBoundsException => - throw new KeyFieldOutOfBoundsException(keyPositions(i)) - } - } - - override def equalToReference(candidate: Row): Boolean = { - var i = 0 - try { - while(i < keyPositions.length) { - val comparator = comparators(i) - val element = candidate.productElement(keyPositions(i)) // element can be null - // check if reference is not equal - if (!comparator.equalToReference(element)) { - return false - } - i += 1 - } - } catch { - case iobex: IndexOutOfBoundsException => - throw new KeyFieldOutOfBoundsException(keyPositions(i)) - } - true - } - - override def compare(first: Row, second: Row): Int = { - var i = 0 - try { - while(i < keyPositions.length) { - val keyPos: Int = keyPositions(i) - val comparator = comparators(i) - val firstElement = first.productElement(keyPos) // element can be null - val secondElement = second.productElement(keyPos) // element can be null - - val cmp = comparator.compare(firstElement, secondElement) - if (cmp != 0) { - return cmp - } - i += 1 - } - } catch { - case iobex: IndexOutOfBoundsException => - throw new KeyFieldOutOfBoundsException(keyPositions(i)) - } - 0 - } - - override def putNormalizedKey( - record: Row, - target: MemorySegment, - offset: Int, - numBytes: Int) - : Unit = { - var bytesLeft = numBytes - var currentOffset = offset - - var i = 0 - while (i < numLeadingNormalizableKeys && bytesLeft > 0) { - var len = normalizedKeyLengths(i) - len = if (bytesLeft >= len) len else bytesLeft - - val comparator = comparators(i) - val element = record.productElement(keyPositions(i)) // element can be null - // write key - comparator.putNormalizedKey(element, target, currentOffset, len) - - bytesLeft -= len - currentOffset += len - i += 1 - } - } - - override def getFlatComparator(flatComparators: util.List[TypeComparator[_]]): Unit = - comparators.foreach { c => - c.getFlatComparators.foreach { fc => - flatComparators.add(fc) - } - } - - override def extractKeys(record: Any, target: Array[AnyRef], index: Int): Int = { - val len = comparators.length - var localIndex = index - var i = 0 - while (i < len) { - val element = record.asInstanceOf[Row].productElement(keyPositions(i)) // element can be null - localIndex += comparators(i).extractKeys(element, target, localIndex) - i += 1 - } - localIndex - index - } -} - -object RowComparator { - private def makeNullAware( - comparators: Array[TypeComparator[Any]], - orders: Array[Boolean]) - : Array[NullAwareComparator[Any]] = - comparators - .zip(orders) - .map { case (comp, order) => - new NullAwareComparator[Any]( - comp, - order) - } - - /** - * @return creates auxiliary fields for normalized key support - */ - private def createAuxiliaryFields( - keyPositions: Array[Int], - comparators: Array[NullAwareComparator[Any]]) - : (Array[Int], Int, Int, Boolean) = { - - val normalizedKeyLengths = new Array[Int](keyPositions.length) - var numLeadingNormalizableKeys = 0 - var normalizableKeyPrefixLen = 0 - var inverted = false - - var i = 0 - while (i < keyPositions.length) { - val k = comparators(i) - // as long as the leading keys support normalized keys, we can build up the composite key - if (k.supportsNormalizedKey()) { - if (i == 0) { - // the first comparator decides whether we need to invert the key direction - inverted = k.invertNormalizedKey() - } - else if (k.invertNormalizedKey() != inverted) { - // if a successor does not agree on the inversion direction, it cannot be part of the - // normalized key - return (normalizedKeyLengths, - numLeadingNormalizableKeys, - normalizableKeyPrefixLen, - inverted) - } - numLeadingNormalizableKeys += 1 - val len = k.getNormalizeKeyLen - if (len < 0) { - throw new RuntimeException("Comparator " + k.getClass.getName + - " specifies an invalid length for the normalized key: " + len) - } - normalizedKeyLengths(i) = len - normalizableKeyPrefixLen += len - if (normalizableKeyPrefixLen < 0) { - // overflow, which means we are out of budget for normalized key space anyways - return (normalizedKeyLengths, - numLeadingNormalizableKeys, - Integer.MAX_VALUE, - inverted) - } - } - else { - return (normalizedKeyLengths, - numLeadingNormalizableKeys, - normalizableKeyPrefixLen, - inverted) - } - i += 1 - } - (normalizedKeyLengths, - numLeadingNormalizableKeys, - normalizableKeyPrefixLen, - inverted) - } -} - diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/typeutils/RowSerializer.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/typeutils/RowSerializer.scala deleted file mode 100644 index 825a99c1385da..0000000000000 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/typeutils/RowSerializer.scala +++ /dev/null @@ -1,209 +0,0 @@ -/* - * 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.table.typeutils - -import org.apache.flink.api.common.typeutils.TypeSerializer -import org.apache.flink.api.table.Row -import org.apache.flink.api.table.typeutils.NullMaskUtils.{writeNullMask, readIntoNullMask, readIntoAndCopyNullMask} -import org.apache.flink.core.memory.{DataInputView, DataOutputView} - -/** - * Serializer for [[Row]]. - */ -class RowSerializer(val fieldSerializers: Array[TypeSerializer[Any]]) - extends TypeSerializer[Row] { - - private val nullMask = new Array[Boolean](fieldSerializers.length) - - override def isImmutableType: Boolean = false - - override def getLength: Int = -1 - - override def duplicate: RowSerializer = { - val duplicateFieldSerializers = fieldSerializers.map(_.duplicate()) - new RowSerializer(duplicateFieldSerializers) - } - - override def createInstance: Row = { - new Row(fieldSerializers.length) - } - - override def copy(from: Row, reuse: Row): Row = { - val len = fieldSerializers.length - - // cannot reuse, do a non-reuse copy - if (reuse == null) { - return copy(from) - } - - if (from.productArity != len || reuse.productArity != len) { - throw new RuntimeException("Row arity of reuse or from is incompatible with this " + - "RowSerializer.") - } - - var i = 0 - while (i < len) { - val fromField = from.productElement(i) - if (fromField != null) { - val reuseField = reuse.productElement(i) - if (reuseField != null) { - val copy = fieldSerializers(i).copy(fromField, reuseField) - reuse.setField(i, copy) - } - else { - val copy = fieldSerializers(i).copy(fromField) - reuse.setField(i, copy) - } - } - else { - reuse.setField(i, null) - } - i += 1 - } - reuse - } - - override def copy(from: Row): Row = { - val len = fieldSerializers.length - - if (from.productArity != len) { - throw new RuntimeException("Row arity of from does not match serializers.") - } - val result = new Row(len) - var i = 0 - while (i < len) { - val fromField = from.productElement(i).asInstanceOf[AnyRef] - if (fromField != null) { - val copy = fieldSerializers(i).copy(fromField) - result.setField(i, copy) - } - else { - result.setField(i, null) - } - i += 1 - } - result - } - - override def serialize(value: Row, target: DataOutputView) { - val len = fieldSerializers.length - - if (value.productArity != len) { - throw new RuntimeException("Row arity of value does not match serializers.") - } - - // write a null mask - writeNullMask(len, value, target) - - // serialize non-null fields - var i = 0 - while (i < len) { - val o = value.productElement(i).asInstanceOf[AnyRef] - if (o != null) { - val serializer = fieldSerializers(i) - serializer.serialize(value.productElement(i), target) - } - i += 1 - } - } - - override def deserialize(reuse: Row, source: DataInputView): Row = { - val len = fieldSerializers.length - - if (reuse.productArity != len) { - throw new RuntimeException("Row arity of reuse does not match serializers.") - } - - // read null mask - readIntoNullMask(len, source, nullMask) - - // read non-null fields - var i = 0 - while (i < len) { - if (nullMask(i)) { - reuse.setField(i, null) - } - else { - val reuseField = reuse.productElement(i).asInstanceOf[AnyRef] - if (reuseField != null) { - reuse.setField(i, fieldSerializers(i).deserialize(reuseField, source)) - } - else { - reuse.setField(i, fieldSerializers(i).deserialize(source)) - } - } - i += 1 - } - reuse - } - - override def deserialize(source: DataInputView): Row = { - val len = fieldSerializers.length - - val result = new Row(len) - - // read null mask - readIntoNullMask(len, source, nullMask) - - // read non-null fields - var i = 0 - while (i < len) { - if (nullMask(i)) { - result.setField(i, null) - } - else { - result.setField(i, fieldSerializers(i).deserialize(source)) - } - i += 1 - } - result - } - - override def copy(source: DataInputView, target: DataOutputView): Unit = { - val len = fieldSerializers.length - - // copy null mask - readIntoAndCopyNullMask(len, source, target, nullMask) - - // read non-null fields - var i = 0 - while (i < len) { - if (!nullMask(i)) { - fieldSerializers(i).copy(source, target) - } - i += 1 - } - } - - override def equals(any: Any): Boolean = { - any match { - case otherRS: RowSerializer => - otherRS.canEqual(this) && - fieldSerializers.sameElements(otherRS.fieldSerializers) - case _ => false - } - } - - override def canEqual(obj: AnyRef): Boolean = { - obj.isInstanceOf[RowSerializer] - } - - override def hashCode(): Int = { - java.util.Arrays.hashCode(fieldSerializers.asInstanceOf[Array[AnyRef]]) - } -} diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/typeutils/RowTypeInfo.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/typeutils/RowTypeInfo.scala deleted file mode 100644 index 711bb492cb357..0000000000000 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/typeutils/RowTypeInfo.scala +++ /dev/null @@ -1,108 +0,0 @@ -/* - * 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.table.typeutils - -import org.apache.flink.api.common.ExecutionConfig -import org.apache.flink.api.common.typeinfo.TypeInformation -import org.apache.flink.api.common.typeutils.CompositeType.TypeComparatorBuilder -import org.apache.flink.api.common.typeutils.TypeComparator -import org.apache.flink.api.scala.typeutils.CaseClassTypeInfo - -import scala.collection.mutable.ArrayBuffer -import org.apache.flink.api.common.typeutils.TypeSerializer -import org.apache.flink.api.table.Row - -/** - * TypeInformation for [[Row]]. - */ -class RowTypeInfo(fieldTypes: Seq[TypeInformation[_]]) - extends CaseClassTypeInfo[Row]( - classOf[Row], - Array(), - fieldTypes, - for (i <- fieldTypes.indices) yield "f" + i) -{ - - def this(fieldTypes: Array[TypeInformation[_]]) = { - this(fieldTypes.toSeq) - } - - /** - * Temporary variable for directly passing orders to comparators. - */ - var comparatorOrders: Option[Array[Boolean]] = None - - override def createSerializer(executionConfig: ExecutionConfig): TypeSerializer[Row] = { - val fieldSerializers: Array[TypeSerializer[Any]] = new Array[TypeSerializer[Any]](getArity) - for (i <- 0 until getArity) { - fieldSerializers(i) = this.types(i).createSerializer(executionConfig) - .asInstanceOf[TypeSerializer[Any]] - } - - new RowSerializer(fieldSerializers) - } - - override def createComparator( - logicalKeyFields: Array[Int], - orders: Array[Boolean], - logicalFieldOffset: Int, - config: ExecutionConfig) - : TypeComparator[Row] = { - // store the order information for the builder - comparatorOrders = Some(orders) - val comparator = super.createComparator(logicalKeyFields, orders, logicalFieldOffset, config) - comparatorOrders = None - comparator - } - - override def createTypeComparatorBuilder(): TypeComparatorBuilder[Row] = { - new RowTypeComparatorBuilder(comparatorOrders.getOrElse( - throw new IllegalStateException("Cannot create comparator builder without orders."))) - } - - private class RowTypeComparatorBuilder( - comparatorOrders: Array[Boolean]) - extends TypeComparatorBuilder[Row] { - - val fieldComparators: ArrayBuffer[TypeComparator[_]] = new ArrayBuffer[TypeComparator[_]]() - val logicalKeyFields: ArrayBuffer[Int] = new ArrayBuffer[Int]() - - override def initializeTypeComparatorBuilder(size: Int): Unit = { - fieldComparators.sizeHint(size) - logicalKeyFields.sizeHint(size) - } - - override def addComparatorField(fieldId: Int, comparator: TypeComparator[_]): Unit = { - fieldComparators += comparator - logicalKeyFields += fieldId - } - - override def createTypeComparator(config: ExecutionConfig): TypeComparator[Row] = { - val maxIndex = logicalKeyFields.max - - new RowComparator( - getArity, - logicalKeyFields.toArray, - fieldComparators.toArray.asInstanceOf[Array[TypeComparator[Any]]], - types.take(maxIndex + 1).map(_.createSerializer(config).asInstanceOf[TypeSerializer[Any]]), - comparatorOrders - ) - } - } -} - diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/runtime/io/RowCsvInputFormatTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/runtime/io/RowCsvInputFormatTest.scala deleted file mode 100644 index 36aaf88f9dde2..0000000000000 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/runtime/io/RowCsvInputFormatTest.scala +++ /dev/null @@ -1,883 +0,0 @@ -/* - * 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.table.runtime.io - -import java.io.{File, FileOutputStream, OutputStreamWriter} -import java.nio.charset.StandardCharsets -import java.sql.{Date, Time, Timestamp} - -import org.apache.flink.api.common.io.ParseException -import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, SqlTimeTypeInfo} -import org.apache.flink.types.Row -import org.apache.flink.api.table.runtime.io.RowCsvInputFormatTest.{PATH, createTempFile, testRemovingTrailingCR} -import org.apache.flink.api.java.typeutils.RowTypeInfo -import org.apache.flink.configuration.Configuration -import org.apache.flink.core.fs.{FileInputSplit, Path} -import org.apache.flink.types.parser.{FieldParser, StringParser} -import org.junit.Assert._ -import org.junit.{Ignore, Test} -import scala.collection.JavaConversions._ - -class RowCsvInputFormatTest { - - @Test - def ignoreInvalidLines() { - val fileContent = - "#description of the data\n" + - "header1|header2|header3|\n" + - "this is|1|2.0|\n" + - "//a comment\n" + - "a test|3|4.0|\n" + - "#next|5|6.0|\n" - - val split = createTempFile(fileContent) - - val typeInfo = new RowTypeInfo(Seq( - BasicTypeInfo.STRING_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.DOUBLE_TYPE_INFO): _*) - - val format = new RowCsvInputFormat(PATH, typeInfo, "\n", "|") - format.setLenient(false) - val parameters = new Configuration - format.configure(parameters) - format.open(split) - - var result = new Row(3) - try { - result = format.nextRecord(result) - fail("Parse Exception was not thrown! (Row too short)") - } - catch { - case ex: ParseException => // ok - } - - try { - result = format.nextRecord(result) - fail("Parse Exception was not thrown! (Invalid int value)") - } - catch { - case ex: ParseException => // ok - } - - result = format.nextRecord(result) - assertNotNull(result) - assertEquals("this is", result.getField(0)) - assertEquals(1, result.getField(1)) - assertEquals(2.0, result.getField(2)) - - try { - result = format.nextRecord(result) - fail("Parse Exception was not thrown! (Row too short)") - } - catch { - case ex: ParseException => // ok - } - - result = format.nextRecord(result) - assertNotNull(result) - assertEquals("a test", result.getField(0)) - assertEquals(3, result.getField(1)) - assertEquals(4.0, result.getField(2)) - - result = format.nextRecord(result) - assertNotNull(result) - assertEquals("#next", result.getField(0)) - assertEquals(5, result.getField(1)) - assertEquals(6.0, result.getField(2)) - - result = format.nextRecord(result) - assertNull(result) - - // re-open with lenient = true - format.setLenient(true) - format.configure(parameters) - format.open(split) - - result = new Row(3) - - result = format.nextRecord(result) - assertNotNull(result) - assertEquals("header1", result.getField(0)) - assertNull(result.getField(1)) - assertNull(result.getField(2)) - - result = format.nextRecord(result) - assertNotNull(result) - assertEquals("this is", result.getField(0)) - assertEquals(1, result.getField(1)) - assertEquals(2.0, result.getField(2)) - - result = format.nextRecord(result) - assertNotNull(result) - assertEquals("a test", result.getField(0)) - assertEquals(3, result.getField(1)) - assertEquals(4.0, result.getField(2)) - - result = format.nextRecord(result) - assertNotNull(result) - assertEquals("#next", result.getField(0)) - assertEquals(5, result.getField(1)) - assertEquals(6.0, result.getField(2)) - result = format.nextRecord(result) - assertNull(result) - } - - @Test - def ignoreSingleCharPrefixComments() { - val fileContent = - "#description of the data\n" + - "#successive commented line\n" + - "this is|1|2.0|\n" + - "a test|3|4.0|\n" + - "#next|5|6.0|\n" - - val split = createTempFile(fileContent) - - val typeInfo = new RowTypeInfo(Seq( - BasicTypeInfo.STRING_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.DOUBLE_TYPE_INFO): _*) - - val format = new RowCsvInputFormat(PATH, typeInfo, "\n", "|") - format.setCommentPrefix("#") - format.configure(new Configuration) - format.open(split) - - var result = new Row(3) - - result = format.nextRecord(result) - assertNotNull(result) - assertEquals("this is", result.getField(0)) - assertEquals(1, result.getField(1)) - assertEquals(2.0, result.getField(2)) - - result = format.nextRecord(result) - assertNotNull(result) - assertEquals("a test", result.getField(0)) - assertEquals(3, result.getField(1)) - assertEquals(4.0, result.getField(2)) - - result = format.nextRecord(result) - assertNull(result) - } - - @Test - def ignoreMultiCharPrefixComments() { - val fileContent = - "//description of the data\n" + - "//successive commented line\n" + - "this is|1|2.0|\n" + - "a test|3|4.0|\n" + - "//next|5|6.0|\n" - - val split = createTempFile(fileContent) - - val typeInfo = new RowTypeInfo(Seq( - BasicTypeInfo.STRING_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.DOUBLE_TYPE_INFO): _*) - - val format = new RowCsvInputFormat(PATH, typeInfo, "\n", "|") - format.setCommentPrefix("//") - format.configure(new Configuration) - format.open(split) - - var result = new Row(3) - - result = format.nextRecord(result) - assertNotNull(result) - assertEquals("this is", result.getField(0)) - assertEquals(1, result.getField(1)) - assertEquals(2.0, result.getField(2)) - - result = format.nextRecord(result) - assertNotNull(result) - assertEquals("a test", result.getField(0)) - assertEquals(3, result.getField(1)) - assertEquals(4.0, result.getField(2)) - result = format.nextRecord(result) - assertNull(result) - } - - @Test - def readStringFields() { - val fileContent = "abc|def|ghijk\nabc||hhg\n|||" - - val split = createTempFile(fileContent) - - val typeInfo = new RowTypeInfo(Seq( - BasicTypeInfo.STRING_TYPE_INFO, - BasicTypeInfo.STRING_TYPE_INFO, - BasicTypeInfo.STRING_TYPE_INFO): _*) - - val format = new RowCsvInputFormat(PATH, typeInfo, "\n", "|") - format.configure(new Configuration) - format.open(split) - - var result = new Row(3) - - result = format.nextRecord(result) - assertNotNull(result) - assertEquals("abc", result.getField(0)) - assertEquals("def", result.getField(1)) - assertEquals("ghijk", result.getField(2)) - - result = format.nextRecord(result) - assertNotNull(result) - assertEquals("abc", result.getField(0)) - assertEquals("", result.getField(1)) - assertEquals("hhg", result.getField(2)) - - result = format.nextRecord(result) - assertNotNull(result) - assertEquals("", result.getField(0)) - assertEquals("", result.getField(1)) - assertEquals("", result.getField(2)) - - result = format.nextRecord(result) - assertNull(result) - assertTrue(format.reachedEnd) - } - - @Test def readMixedQuotedStringFields() { - val fileContent = "@a|b|c@|def|@ghijk@\nabc||@|hhg@\n|||" - - val split = createTempFile(fileContent) - - val typeInfo = new RowTypeInfo(Seq( - BasicTypeInfo.STRING_TYPE_INFO, - BasicTypeInfo.STRING_TYPE_INFO, - BasicTypeInfo.STRING_TYPE_INFO): _*) - - val format = new RowCsvInputFormat(PATH, typeInfo, "\n", "|") - format.configure(new Configuration) - format.enableQuotedStringParsing('@') - format.open(split) - - var result = new Row(3) - - result = format.nextRecord(result) - assertNotNull(result) - assertEquals("a|b|c", result.getField(0)) - assertEquals("def", result.getField(1)) - assertEquals("ghijk", result.getField(2)) - - result = format.nextRecord(result) - assertNotNull(result) - assertEquals("abc", result.getField(0)) - assertEquals("", result.getField(1)) - assertEquals("|hhg", result.getField(2)) - - result = format.nextRecord(result) - assertNotNull(result) - assertEquals("", result.getField(0)) - assertEquals("", result.getField(1)) - assertEquals("", result.getField(2)) - - result = format.nextRecord(result) - assertNull(result) - assertTrue(format.reachedEnd) - } - - @Test def readStringFieldsWithTrailingDelimiters() { - val fileContent = "abc|-def|-ghijk\nabc|-|-hhg\n|-|-|-\n" - - val split = createTempFile(fileContent) - - val typeInfo = new RowTypeInfo(Seq( - BasicTypeInfo.STRING_TYPE_INFO, - BasicTypeInfo.STRING_TYPE_INFO, - BasicTypeInfo.STRING_TYPE_INFO): _*) - - val format = new RowCsvInputFormat(PATH, typeInfo, "\n", "|") - format.setFieldDelimiter("|-") - format.configure(new Configuration) - format.open(split) - - var result = new Row(3) - - result = format.nextRecord(result) - assertNotNull(result) - assertEquals("abc", result.getField(0)) - assertEquals("def", result.getField(1)) - assertEquals("ghijk", result.getField(2)) - - result = format.nextRecord(result) - assertNotNull(result) - assertEquals("abc", result.getField(0)) - assertEquals("", result.getField(1)) - assertEquals("hhg", result.getField(2)) - - result = format.nextRecord(result) - assertNotNull(result) - assertEquals("", result.getField(0)) - assertEquals("", result.getField(1)) - assertEquals("", result.getField(2)) - - result = format.nextRecord(result) - assertNull(result) - assertTrue(format.reachedEnd) - } - - @Test - def testIntegerFields() { - val fileContent = "111|222|333|444|555\n666|777|888|999|000|\n" - - val split = createTempFile(fileContent) - - val typeInfo = new RowTypeInfo(Seq( - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO): _*) - - val format = new RowCsvInputFormat(PATH, typeInfo, "\n", "|") - - format.setFieldDelimiter("|") - format.configure(new Configuration) - format.open(split) - - var result = new Row(5) - - result = format.nextRecord(result) - assertNotNull(result) - assertEquals(111, result.getField(0)) - assertEquals(222, result.getField(1)) - assertEquals(333, result.getField(2)) - assertEquals(444, result.getField(3)) - assertEquals(555, result.getField(4)) - - result = format.nextRecord(result) - assertNotNull(result) - assertEquals(666, result.getField(0)) - assertEquals(777, result.getField(1)) - assertEquals(888, result.getField(2)) - assertEquals(999, result.getField(3)) - assertEquals(0, result.getField(4)) - - result = format.nextRecord(result) - assertNull(result) - assertTrue(format.reachedEnd) - } - - @Test - def testEmptyFields() { - val fileContent = - ",,,,,,,,\n" + - ",,,,,,,,\n" + - ",,,,,,,,\n" + - ",,,,,,,,\n" + - ",,,,,,,,\n" + - ",,,,,,,,\n" + - ",,,,,,,,\n" + - ",,,,,,,,\n" - - val split = createTempFile(fileContent) - - val typeInfo = new RowTypeInfo(Seq( - BasicTypeInfo.BOOLEAN_TYPE_INFO, - BasicTypeInfo.BYTE_TYPE_INFO, - BasicTypeInfo.DOUBLE_TYPE_INFO, - BasicTypeInfo.FLOAT_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.LONG_TYPE_INFO, - BasicTypeInfo.SHORT_TYPE_INFO, - BasicTypeInfo.STRING_TYPE_INFO): _*) - - val format = new RowCsvInputFormat(PATH, rowTypeInfo = typeInfo, emptyColumnAsNull = true) - format.setFieldDelimiter(",") - format.configure(new Configuration) - format.open(split) - - var result = new Row(8) - val linesCnt = fileContent.split("\n").length - - for (i <- 0 until linesCnt) yield { - result = format.nextRecord(result) - assertNull(result.getField(i)) - } - - // ensure no more rows - assertNull(format.nextRecord(result)) - assertTrue(format.reachedEnd) - } - - @Test - def testDoubleFields() { - val fileContent = "11.1|22.2|33.3|44.4|55.5\n66.6|77.7|88.8|99.9|00.0|\n" - - val split = createTempFile(fileContent) - - val typeInfo = new RowTypeInfo(Seq( - BasicTypeInfo.DOUBLE_TYPE_INFO, - BasicTypeInfo.DOUBLE_TYPE_INFO, - BasicTypeInfo.DOUBLE_TYPE_INFO, - BasicTypeInfo.DOUBLE_TYPE_INFO, - BasicTypeInfo.DOUBLE_TYPE_INFO): _*) - - val format = new RowCsvInputFormat(PATH, rowTypeInfo = typeInfo) - format.setFieldDelimiter("|") - format.configure(new Configuration) - format.open(split) - - var result = new Row(5) - - result = format.nextRecord(result) - assertNotNull(result) - assertEquals(11.1, result.getField(0)) - assertEquals(22.2, result.getField(1)) - assertEquals(33.3, result.getField(2)) - assertEquals(44.4, result.getField(3)) - assertEquals(55.5, result.getField(4)) - - result = format.nextRecord(result) - assertNotNull(result) - assertEquals(66.6, result.getField(0)) - assertEquals(77.7, result.getField(1)) - assertEquals(88.8, result.getField(2)) - assertEquals(99.9, result.getField(3)) - assertEquals(0.0, result.getField(4)) - - result = format.nextRecord(result) - assertNull(result) - assertTrue(format.reachedEnd) - } - - @Test - def testReadFirstN() { - val fileContent = "111|222|333|444|555|\n666|777|888|999|000|\n" - - val split = createTempFile(fileContent) - - val typeInfo = new RowTypeInfo(Seq( - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO): _*) - - val format = new RowCsvInputFormat(PATH, rowTypeInfo = typeInfo) - format.setFieldDelimiter("|") - format.configure(new Configuration) - format.open(split) - - var result = new Row(2) - - result = format.nextRecord(result) - assertNotNull(result) - assertEquals(111, result.getField(0)) - assertEquals(222, result.getField(1)) - - result = format.nextRecord(result) - assertNotNull(result) - assertEquals(666, result.getField(0)) - assertEquals(777, result.getField(1)) - - result = format.nextRecord(result) - assertNull(result) - assertTrue(format.reachedEnd) - } - - @Test - def testReadSparseWithNullFieldsForTypes() { - val fileContent = "111|x|222|x|333|x|444|x|555|x|666|x|777|x|888|x|999|x|000|x|\n" + - "000|x|999|x|888|x|777|x|666|x|555|x|444|x|333|x|222|x|111|x|" - - val split = createTempFile(fileContent) - - val typeInfo: RowTypeInfo = new RowTypeInfo(Seq( - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO): _*) - - val format = new RowCsvInputFormat( - PATH, - rowTypeInfo = typeInfo, - includedFieldsMask = Array(true, false, false, true, false, false, false, true)) - format.setFieldDelimiter("|x|") - format.configure(new Configuration) - format.open(split) - - var result = new Row(3) - - result = format.nextRecord(result) - assertNotNull(result) - assertEquals(111, result.getField(0)) - assertEquals(444, result.getField(1)) - assertEquals(888, result.getField(2)) - - result = format.nextRecord(result) - assertNotNull(result) - assertEquals(0, result.getField(0)) - assertEquals(777, result.getField(1)) - assertEquals(333, result.getField(2)) - - result = format.nextRecord(result) - assertNull(result) - assertTrue(format.reachedEnd) - } - - @Test - def testReadSparseWithPositionSetter() { - val fileContent = "111|222|333|444|555|666|777|888|999|000|\n" + - "000|999|888|777|666|555|444|333|222|111|" - - val split = createTempFile(fileContent) - - val typeInfo = new RowTypeInfo(Seq( - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO): _*) - - val format = new RowCsvInputFormat( - PATH, - typeInfo, - Array(0, 3, 7)) - format.setFieldDelimiter("|") - format.configure(new Configuration) - format.open(split) - - var result = new Row(3) - result = format.nextRecord(result) - - assertNotNull(result) - assertEquals(111, result.getField(0)) - assertEquals(444, result.getField(1)) - assertEquals(888, result.getField(2)) - - result = format.nextRecord(result) - assertNotNull(result) - assertEquals(0, result.getField(0)) - assertEquals(777, result.getField(1)) - assertEquals(333, result.getField(2)) - - result = format.nextRecord(result) - assertNull(result) - assertTrue(format.reachedEnd) - } - - @Test - def testReadSparseWithMask() { - val fileContent = "111&&222&&333&&444&&555&&666&&777&&888&&999&&000&&\n" + - "000&&999&&888&&777&&666&&555&&444&&333&&222&&111&&" - - val split = RowCsvInputFormatTest.createTempFile(fileContent) - - val typeInfo = new RowTypeInfo(Seq( - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO): _*) - - val format = new RowCsvInputFormat( - PATH, - rowTypeInfo = typeInfo, - includedFieldsMask = Array(true, false, false, true, false, false, false, true)) - format.setFieldDelimiter("&&") - format.configure(new Configuration) - format.open(split) - - var result = new Row(3) - - result = format.nextRecord(result) - assertNotNull(result) - assertEquals(111, result.getField(0)) - assertEquals(444, result.getField(1)) - assertEquals(888, result.getField(2)) - - result = format.nextRecord(result) - assertNotNull(result) - assertEquals(0, result.getField(0)) - assertEquals(777, result.getField(1)) - assertEquals(333, result.getField(2)) - - result = format.nextRecord(result) - assertNull(result) - assertTrue(format.reachedEnd) - } - - @Test - def testParseStringErrors() { - val stringParser = new StringParser - stringParser.enableQuotedStringParsing('"'.toByte) - - val failures = Seq( - ("\"string\" trailing", FieldParser.ParseErrorState.UNQUOTED_CHARS_AFTER_QUOTED_STRING), - ("\"unterminated ", FieldParser.ParseErrorState.UNTERMINATED_QUOTED_STRING) - ) - - for (failure <- failures) { - val result = stringParser.parseField( - failure._1.getBytes, - 0, - failure._1.length, - Array[Byte]('|'), - null) - - assertEquals(-1, result) - assertEquals(failure._2, stringParser.getErrorState) - } - } - - // Test disabled because we do not support double-quote escaped quotes right now. - @Test - @Ignore - def testParserCorrectness() { - // RFC 4180 Compliance Test content - // Taken from http://en.wikipedia.org/wiki/Comma-separated_values#Example - val fileContent = "Year,Make,Model,Description,Price\n" + - "1997,Ford,E350,\"ac, abs, moon\",3000.00\n" + - "1999,Chevy,\"Venture \"\"Extended Edition\"\"\",\"\",4900.00\n" + - "1996,Jeep,Grand Cherokee,\"MUST SELL! air, moon roof, loaded\",4799.00\n" + - "1999,Chevy,\"Venture \"\"Extended Edition, Very Large\"\"\",,5000.00\n" + - ",,\"Venture \"\"Extended Edition\"\"\",\"\",4900.00" - - val split = createTempFile(fileContent) - - val typeInfo = new RowTypeInfo(Seq( - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.STRING_TYPE_INFO, - BasicTypeInfo.STRING_TYPE_INFO, - BasicTypeInfo.STRING_TYPE_INFO, - BasicTypeInfo.DOUBLE_TYPE_INFO): _*) - - val format = new RowCsvInputFormat(PATH, typeInfo) - format.setSkipFirstLineAsHeader(true) - format.setFieldDelimiter(",") - format.configure(new Configuration) - format.open(split) - - var result = new Row(5) - val r1: Row = new Row(5) - r1.setField(0, 1997) - r1.setField(1, "Ford") - r1.setField(2, "E350") - r1.setField(3, "ac, abs, moon") - r1.setField(4, 3000.0) - - val r2: Row = new Row(5) - r2.setField(0, 1999) - r2.setField(1, "Chevy") - r2.setField(2, "Venture \"Extended Edition\"") - r2.setField(3, "") - r2.setField(4, 4900.0) - - val r3: Row = new Row(5) - r3.setField(0, 1996) - r3.setField(1, "Jeep") - r3.setField(2, "Grand Cherokee") - r3.setField(3, "MUST SELL! air, moon roof, loaded") - r3.setField(4, 4799.0) - - val r4: Row = new Row(5) - r4.setField(0, 1999) - r4.setField(1, "Chevy") - r4.setField(2, "Venture \"Extended Edition, Very Large\"") - r4.setField(3, "") - r4.setField(4, 5000.0) - - val r5: Row = new Row(5) - r5.setField(0, 0) - r5.setField(1, "") - r5.setField(2, "Venture \"Extended Edition\"") - r5.setField(3, "") - r5.setField(4, 4900.0) - - val expectedLines = Array(r1, r2, r3, r4, r5) - for (expected <- expectedLines) { - result = format.nextRecord(result) - assertEquals(expected, result) - } - assertNull(format.nextRecord(result)) - assertTrue(format.reachedEnd) - } - - @Test - def testWindowsLineEndRemoval() { - - // check typical use case -- linux file is correct and it is set up to linux(\n) - testRemovingTrailingCR("\n", "\n") - - // check typical windows case -- windows file endings and file has windows file endings set up - testRemovingTrailingCR("\r\n", "\r\n") - - // check problematic case windows file -- windows file endings(\r\n) - // but linux line endings (\n) set up - testRemovingTrailingCR("\r\n", "\n") - - // check problematic case linux file -- linux file endings (\n) - // but windows file endings set up (\r\n) - // specific setup for windows line endings will expect \r\n because - // it has to be set up and is not standard. - } - - @Test - def testQuotedStringParsingWithIncludeFields() { - val fileContent = "\"20:41:52-1-3-2015\"|\"Re: Taskmanager memory error in Eclipse\"|" + - "\"Blahblah \"|\"blaaa|\"blubb\"" - val tempFile = File.createTempFile("CsvReaderQuotedString", "tmp") - tempFile.deleteOnExit() - tempFile.setWritable(true) - - val writer = new OutputStreamWriter(new FileOutputStream(tempFile)) - writer.write(fileContent) - writer.close() - - val typeInfo = new RowTypeInfo(Seq( - BasicTypeInfo.STRING_TYPE_INFO, - BasicTypeInfo.STRING_TYPE_INFO): _*) - - val inputFormat = new RowCsvInputFormat( - new Path(tempFile.toURI.toString), - rowTypeInfo = typeInfo, - includedFieldsMask = Array(true, false, true)) - inputFormat.enableQuotedStringParsing('"') - inputFormat.setFieldDelimiter("|") - inputFormat.setDelimiter('\n') - inputFormat.configure(new Configuration) - - val splits = inputFormat.createInputSplits(1) - inputFormat.open(splits(0)) - - val record = inputFormat.nextRecord(new Row(2)) - assertEquals("20:41:52-1-3-2015", record.getField(0)) - assertEquals("Blahblah ", record.getField(1)) - } - - @Test - def testQuotedStringParsingWithEscapedQuotes() { - val fileContent = "\"\\\"Hello\\\" World\"|\"We are\\\" young\"" - val tempFile = File.createTempFile("CsvReaderQuotedString", "tmp") - tempFile.deleteOnExit() - tempFile.setWritable(true) - - val writer = new OutputStreamWriter(new FileOutputStream(tempFile)) - writer.write(fileContent) - writer.close() - - val typeInfo = new RowTypeInfo(Seq( - BasicTypeInfo.STRING_TYPE_INFO, - BasicTypeInfo.STRING_TYPE_INFO): _*) - - val inputFormat = new RowCsvInputFormat( - new Path(tempFile.toURI.toString), - rowTypeInfo = typeInfo) - inputFormat.enableQuotedStringParsing('"') - inputFormat.setFieldDelimiter("|") - inputFormat.setDelimiter('\n') - inputFormat.configure(new Configuration) - - val splits = inputFormat.createInputSplits(1) - inputFormat.open(splits(0)) - - val record = inputFormat.nextRecord(new Row(2)) - assertEquals("\\\"Hello\\\" World", record.getField(0)) - assertEquals("We are\\\" young", record.getField(1)) - } - - @Test - def testSqlTimeFields() { - val fileContent = "1990-10-14|02:42:25|1990-10-14 02:42:25.123|1990-1-4 2:2:5\n" + - "1990-10-14|02:42:25|1990-10-14 02:42:25.123|1990-1-4 2:2:5.3\n" - - val split = createTempFile(fileContent) - - val typeInfo = new RowTypeInfo(Seq( - SqlTimeTypeInfo.DATE, - SqlTimeTypeInfo.TIME, - SqlTimeTypeInfo.TIMESTAMP, - SqlTimeTypeInfo.TIMESTAMP): _*) - - val format = new RowCsvInputFormat(PATH, rowTypeInfo = typeInfo) - format.setFieldDelimiter("|") - format.configure(new Configuration) - format.open(split) - - var result = new Row(4) - - result = format.nextRecord(result) - assertNotNull(result) - assertEquals(Date.valueOf("1990-10-14"), result.getField(0)) - assertEquals(Time.valueOf("02:42:25"), result.getField(1)) - assertEquals(Timestamp.valueOf("1990-10-14 02:42:25.123"), result.getField(2)) - assertEquals(Timestamp.valueOf("1990-01-04 02:02:05"), result.getField(3)) - - result = format.nextRecord(result) - assertNotNull(result) - assertEquals(Date.valueOf("1990-10-14"), result.getField(0)) - assertEquals(Time.valueOf("02:42:25"), result.getField(1)) - assertEquals(Timestamp.valueOf("1990-10-14 02:42:25.123"), result.getField(2)) - assertEquals(Timestamp.valueOf("1990-01-04 02:02:05.3"), result.getField(3)) - - result = format.nextRecord(result) - assertNull(result) - assertTrue(format.reachedEnd) - } -} - -object RowCsvInputFormatTest { - - private val PATH = new Path("an/ignored/file/") - - // static variables for testing the removal of \r\n to \n - private val FIRST_PART = "That is the first part" - private val SECOND_PART = "That is the second part" - - private def createTempFile(content: String): FileInputSplit = { - val tempFile = File.createTempFile("test_contents", "tmp") - tempFile.deleteOnExit() - val wrt = new OutputStreamWriter(new FileOutputStream(tempFile), StandardCharsets.UTF_8) - wrt.write(content) - wrt.close() - new FileInputSplit( - 0, - new Path(tempFile.toURI.toString), - 0, - tempFile.length, - Array("localhost")) - } - - private def testRemovingTrailingCR(lineBreakerInFile: String, lineBreakerSetup: String) { - val fileContent = FIRST_PART + lineBreakerInFile + SECOND_PART + lineBreakerInFile - - // create input file - val tempFile = File.createTempFile("CsvInputFormatTest", "tmp") - tempFile.deleteOnExit() - tempFile.setWritable(true) - - val wrt = new OutputStreamWriter(new FileOutputStream(tempFile)) - wrt.write(fileContent) - wrt.close() - - val typeInfo = new RowTypeInfo(Seq(BasicTypeInfo.STRING_TYPE_INFO): _*) - - val inputFormat = new RowCsvInputFormat(new Path(tempFile.toURI.toString), typeInfo) - inputFormat.configure(new Configuration) - inputFormat.setDelimiter(lineBreakerSetup) - - val splits = inputFormat.createInputSplits(1) - inputFormat.open(splits(0)) - - var result = inputFormat.nextRecord(new Row(1)) - assertNotNull("Expecting to not return null", result) - assertEquals(FIRST_PART, result.getField(0)) - - result = inputFormat.nextRecord(result) - assertNotNull("Expecting to not return null", result) - assertEquals(SECOND_PART, result.getField(0)) - } -} diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/typeutils/RowComparatorTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/typeutils/RowComparatorTest.scala deleted file mode 100644 index c441a4f8c0795..0000000000000 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/typeutils/RowComparatorTest.scala +++ /dev/null @@ -1,136 +0,0 @@ -/* - * 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.table.typeutils - -import org.apache.flink.api.common.ExecutionConfig -import org.apache.flink.api.common.typeinfo.BasicTypeInfo -import org.apache.flink.api.common.typeutils.{ComparatorTestBase, TypeComparator, TypeSerializer} -import org.apache.flink.api.java.tuple -import org.apache.flink.api.java.typeutils.{TupleTypeInfo, TypeExtractor} -import org.apache.flink.types.Row -import org.apache.flink.api.table.typeutils.RowComparatorTest.MyPojo -import org.apache.flink.api.java.typeutils.{RowTypeInfo => RowTypeInfoNew} -import org.junit.Assert._ - -class RowComparatorTest extends ComparatorTestBase[Row] { - - val typeInfo = new RowTypeInfoNew( - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.DOUBLE_TYPE_INFO, - BasicTypeInfo.STRING_TYPE_INFO, - new TupleTypeInfo[tuple.Tuple2[Int, Boolean]]( - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.BOOLEAN_TYPE_INFO, - BasicTypeInfo.SHORT_TYPE_INFO), - TypeExtractor.createTypeInfo(classOf[MyPojo])) - - val testPojo1 = new MyPojo() - // TODO we cannot test null here as PojoComparator has no support for null keys - testPojo1.name = "" - val testPojo2 = new MyPojo() - testPojo2.name = "Test1" - val testPojo3 = new MyPojo() - testPojo3.name = "Test2" - - val data: Array[Row] = Array( - createRow(null, null, null, null, null), - createRow(0, null, null, null, null), - createRow(0, 0.0, null, null, null), - createRow(0, 0.0, "a", null, null), - createRow(1, 0.0, "a", null, null), - createRow(1, 1.0, "a", null, null), - createRow(1, 1.0, "b", null, null), - createRow(1, 1.0, "b", new tuple.Tuple3[Int, Boolean, Short](1, false, 2), null), - createRow(1, 1.0, "b", new tuple.Tuple3[Int, Boolean, Short](2, false, 2), null), - createRow(1, 1.0, "b", new tuple.Tuple3[Int, Boolean, Short](2, true, 2), null), - createRow(1, 1.0, "b", new tuple.Tuple3[Int, Boolean, Short](2, true, 3), null), - createRow(1, 1.0, "b", new tuple.Tuple3[Int, Boolean, Short](2, true, 3), testPojo1), - createRow(1, 1.0, "b", new tuple.Tuple3[Int, Boolean, Short](2, true, 3), testPojo2), - createRow(1, 1.0, "b", new tuple.Tuple3[Int, Boolean, Short](2, true, 3), testPojo3) - ) - - override protected def deepEquals(message: String, should: Row, is: Row): Unit = { - val arity = should.getArity - assertEquals(message, arity, is.getArity) - var index = 0 - while (index < arity) { - val copiedValue: Any = should.getField(index) - val element: Any = is.getField(index) - assertEquals(message, element, copiedValue) - index += 1 - } - } - - override protected def createComparator(ascending: Boolean): TypeComparator[Row] = { - typeInfo.createComparator( - Array(0, 1, 2, 3, 4, 5, 6), - Array(ascending, ascending, ascending, ascending, ascending, ascending, ascending), - 0, - new ExecutionConfig()) - } - - override protected def createSerializer(): TypeSerializer[Row] = { - typeInfo.createSerializer(new ExecutionConfig()) - } - - override protected def getSortedTestData: Array[Row] = { - data - } - - override protected def supportsNullKeys: Boolean = true - - def createRow(f0: Any, f1: Any, f2: Any, f3: Any, f4: Any): Row = { - val r: Row = new Row(5) - r.setField(0, f0) - r.setField(1, f1) - r.setField(2, f2) - r.setField(3, f3) - r.setField(4, f4) - r - } -} - -object RowComparatorTest { - - class MyPojo() extends Serializable with Comparable[MyPojo] { - // we cannot use null because the PojoComparator does not support null properly - var name: String = "" - - override def compareTo(o: MyPojo): Int = { - if (name == null && o.name == null) { - 0 - } - else if (name == null) { - -1 - } - else if (o.name == null) { - 1 - } - else { - name.compareTo(o.name) - } - } - - override def equals(other: Any): Boolean = other match { - case that: MyPojo => compareTo(that) == 0 - case _ => false - } - } -} - diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/typeutils/RowComparatorWithManyFieldsTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/typeutils/RowComparatorWithManyFieldsTest.scala deleted file mode 100644 index 645e18d43ac88..0000000000000 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/typeutils/RowComparatorWithManyFieldsTest.scala +++ /dev/null @@ -1,83 +0,0 @@ -/* - * 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.table.typeutils - -import org.apache.flink.api.common.ExecutionConfig -import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, TypeInformation} -import org.apache.flink.api.common.typeutils.{ComparatorTestBase, TypeComparator, TypeSerializer} -import org.apache.flink.types.Row -import org.apache.flink.api.java.typeutils.{RowTypeInfo => RowTypeInfoNew} -import org.apache.flink.util.Preconditions -import org.junit.Assert._ - -/** - * Tests [[RowComparator]] for wide rows. - */ -class RowComparatorWithManyFieldsTest extends ComparatorTestBase[Row] { - val numberOfFields = 10 - val fieldTypes = new Array[TypeInformation[_]](numberOfFields) - for (i <- 0 until numberOfFields) { - fieldTypes(i) = BasicTypeInfo.STRING_TYPE_INFO - } - val typeInfo = new RowTypeInfoNew(fieldTypes: _*) - - val data: Array[Row] = Array( - createRow(Array(null, "b0", "c0", "d0", "e0", "f0", "g0", "h0", "i0", "j0")), - createRow(Array("a1", "b1", "c1", "d1", "e1", "f1", "g1", "h1", "i1", "j1")), - createRow(Array("a2", "b2", "c2", "d2", "e2", "f2", "g2", "h2", "i2", "j2")), - createRow(Array("a3", "b3", "c3", "d3", "e3", "f3", "g3", "h3", "i3", "j3")) - ) - - override protected def deepEquals(message: String, should: Row, is: Row): Unit = { - val arity = should.getArity - assertEquals(message, arity, is.getArity) - var index = 0 - while (index < arity) { - val copiedValue: Any = should.getField(index) - val element: Any = is.getField(index) - assertEquals(message, element, copiedValue) - index += 1 - } - } - - override protected def createComparator(ascending: Boolean): TypeComparator[Row] = { - typeInfo.createComparator( - Array(0), - Array(ascending), - 0, - new ExecutionConfig()) - } - - override protected def createSerializer(): TypeSerializer[Row] = { - typeInfo.createSerializer(new ExecutionConfig()) - } - - override protected def getSortedTestData: Array[Row] = { - data - } - - override protected def supportsNullKeys: Boolean = true - - private def createRow(values: Array[_]): Row = { - Preconditions.checkArgument(values.length == numberOfFields) - val r: Row = new Row(numberOfFields) - values.zipWithIndex.foreach { case (e, i) => r.setField(i, e) } - r - } -} diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/typeutils/RowSerializerTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/typeutils/RowSerializerTest.scala deleted file mode 100644 index 28dfbc03248c3..0000000000000 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/typeutils/RowSerializerTest.scala +++ /dev/null @@ -1,193 +0,0 @@ -/* - * 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.table.typeutils - -import org.apache.flink.api.common.ExecutionConfig -import org.apache.flink.api.common.typeinfo.BasicTypeInfo -import org.apache.flink.api.common.typeutils.{SerializerTestInstance, TypeSerializer} -import org.apache.flink.api.java.tuple -import org.apache.flink.api.java.typeutils.{TupleTypeInfo, TypeExtractor} -import org.apache.flink.types.Row -import org.apache.flink.api.table.typeutils.RowSerializerTest.MyPojo -import org.junit.Assert._ -import org.junit.Test - -class RowSerializerTest { - - class RowSerializerTestInstance( - serializer: TypeSerializer[Row], - testData: Array[Row]) - extends SerializerTestInstance[Row](serializer, classOf[Row], -1, testData: _*) { - - override protected def deepEquals(message: String, should: Row, is: Row): Unit = { - val arity = should.getArity - assertEquals(message, arity, is.getArity) - var index = 0 - while (index < arity) { - val copiedValue: Any = should.getField(index) - val element: Any = is.getField(index) - assertEquals(message, element, copiedValue) - index += 1 - } - } - } - - @Test - def testRowSerializer(): Unit = { - val rowInfo = new org.apache.flink.api.java.typeutils.RowTypeInfo( - BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO) - - val row1 = new Row(2) - row1.setField(0, 1) - row1.setField(1, "a") - - val row2 = new Row(2) - row2.setField(0, 2) - row2.setField(1, null) - - val testData: Array[Row] = Array(row1, row2) - - val rowSerializer: TypeSerializer[Row] = rowInfo.createSerializer(new ExecutionConfig) - - val testInstance = new RowSerializerTestInstance(rowSerializer, testData) - - testInstance.testAll() - } - - @Test - def testLargeRowSerializer(): Unit = { - val rowInfo = new org.apache.flink.api.java.typeutils.RowTypeInfo( - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.STRING_TYPE_INFO) - - val row = new Row(13) - row.setField(0, 2) - row.setField(1, null) - row.setField(3, null) - row.setField(4, null) - row.setField(5, null) - row.setField(6, null) - row.setField(7, null) - row.setField(8, null) - row.setField(9, null) - row.setField(10, null) - row.setField(11, null) - row.setField(12, "Test") - - val testData: Array[Row] = Array(row) - - val rowSerializer: TypeSerializer[Row] = rowInfo.createSerializer(new ExecutionConfig) - - val testInstance = new RowSerializerTestInstance(rowSerializer, testData) - - testInstance.testAll() - } - - @Test - def testRowSerializerWithComplexTypes(): Unit = { - val rowInfo = new org.apache.flink.api.java.typeutils.RowTypeInfo( - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.DOUBLE_TYPE_INFO, - BasicTypeInfo.STRING_TYPE_INFO, - new TupleTypeInfo[tuple.Tuple2[Int, Boolean]]( - BasicTypeInfo.INT_TYPE_INFO, - BasicTypeInfo.BOOLEAN_TYPE_INFO, - BasicTypeInfo.SHORT_TYPE_INFO), - TypeExtractor.createTypeInfo(classOf[MyPojo])) - - val testPojo1 = new MyPojo() - testPojo1.name = null - val testPojo2 = new MyPojo() - testPojo2.name = "Test1" - val testPojo3 = new MyPojo() - testPojo3.name = "Test2" - - val testData: Array[Row] = Array( - createRow(null, null, null, null, null), - createRow(0, null, null, null, null), - createRow(0, 0.0, null, null, null), - createRow(0, 0.0, "a", null, null), - createRow(1, 0.0, "a", null, null), - createRow(1, 1.0, "a", null, null), - createRow(1, 1.0, "b", null, null), - createRow(1, 1.0, "b", new tuple.Tuple3[Int, Boolean, Short](1, false, 2), null), - createRow(1, 1.0, "b", new tuple.Tuple3[Int, Boolean, Short](2, false, 2), null), - createRow(1, 1.0, "b", new tuple.Tuple3[Int, Boolean, Short](2, true, 2), null), - createRow(1, 1.0, "b", new tuple.Tuple3[Int, Boolean, Short](2, true, 3), null), - createRow(1, 1.0, "b", new tuple.Tuple3[Int, Boolean, Short](2, true, 3), testPojo1), - createRow(1, 1.0, "b", new tuple.Tuple3[Int, Boolean, Short](2, true, 3), testPojo2), - createRow(1, 1.0, "b", new tuple.Tuple3[Int, Boolean, Short](2, true, 3), testPojo3) - ) - - val rowSerializer: TypeSerializer[Row] = rowInfo.createSerializer(new ExecutionConfig) - - val testInstance = new RowSerializerTestInstance(rowSerializer, testData) - - testInstance.testAll() - } - - // ---------------------------------------------------------------------------------------------- - - def createRow(f0: Any, f1: Any, f2: Any, f3: Any, f4: Any): Row = { - val r: Row = new Row(5) - r.setField(0, f0) - r.setField(1, f1) - r.setField(2, f2) - r.setField(3, f3) - r.setField(4, f4) - r - } -} - -object RowSerializerTest { - class MyPojo() extends Serializable with Comparable[MyPojo] { - var name: String = null - - override def compareTo(o: MyPojo): Int = { - if (name == null && o.name == null) { - 0 - } - else if (name == null) { - -1 - } - else if (o.name == null) { - 1 - } - else { - name.compareTo(o.name) - } - } - - override def equals(other: Any): Boolean = other match { - case that: MyPojo => compareTo(that) == 0 - case _ => false - } - } -}