org.apache.avro
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/MultisetSerializer.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/MultisetSerializer.java
deleted file mode 100644
index ef09d46568ef50..00000000000000
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/MultisetSerializer.java
+++ /dev/null
@@ -1,200 +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.common.typeutils.base;
-
-import org.apache.commons.collections4.multiset.AbstractMultiSet;
-import org.apache.commons.collections4.multiset.HashMultiSet;
-import org.apache.flink.annotation.Internal;
-import org.apache.flink.api.common.typeutils.CompatibilityResult;
-import org.apache.flink.api.common.typeutils.CompatibilityUtil;
-import org.apache.flink.api.common.typeutils.TypeDeserializerAdapter;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot;
-import org.apache.flink.api.common.typeutils.UnloadableDummyTypeSerializer;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.core.memory.DataInputView;
-import org.apache.flink.core.memory.DataOutputView;
-
-import java.io.IOException;
-
-import static org.apache.flink.util.Preconditions.checkNotNull;
-
-/**
- * A serializer for Multisets. The serializer relies on an element serializer
- * for the serialization of the Multiset's elements.
- *
- * The serialization format for the Multiset is as follows: four bytes for the length of the lost,
- * followed by the serialized representation of each element.
- *
- * @param The type of element in the Multiset.
- */
-@Internal
-public final class MultisetSerializer extends TypeSerializer> {
-
- private static final long serialVersionUID = 12L;
-
- /** The serializer for the elements of the Multiset */
- private final TypeSerializer elementSerializer;
-
- /**
- * Creates a Multiset serializer that uses the given serializer to serialize the Multiset's elements.
- *
- * @param elementSerializer The serializer for the elements of the Multiset
- */
- public MultisetSerializer(TypeSerializer elementSerializer) {
- this.elementSerializer = checkNotNull(elementSerializer);
- }
-
- // ------------------------------------------------------------------------
- // MultisetSerializer specific properties
- // ------------------------------------------------------------------------
-
- /**
- * Gets the serializer for the elements of the Multiset.
- * @return The serializer for the elements of the Multiset
- */
- public TypeSerializer getElementSerializer() {
- return elementSerializer;
- }
-
- // ------------------------------------------------------------------------
- // Type Serializer implementation
- // ------------------------------------------------------------------------
-
- @Override
- public boolean isImmutableType() {
- return false;
- }
-
- @Override
- public TypeSerializer> duplicate() {
- TypeSerializer duplicateElement = elementSerializer.duplicate();
- return duplicateElement == elementSerializer ?
- this : new MultisetSerializer<>(duplicateElement);
- }
-
- @Override
- public AbstractMultiSet createInstance() {
- return new HashMultiSet<>();
- }
-
- @Override
- public AbstractMultiSet copy(AbstractMultiSet from) {
- return new HashMultiSet<>(from);
- }
-
- @Override
- public AbstractMultiSet copy(AbstractMultiSet from, AbstractMultiSet reuse) {
- return copy(from);
- }
-
- @Override
- public int getLength() {
- return -1; // var length
- }
-
- @Override
- public void serialize(AbstractMultiSet multiSet, DataOutputView target) throws IOException {
- final int size = multiSet.size();
- target.writeInt(size);
-
- for (T element : multiSet) {
- elementSerializer.serialize(element, target);
- }
- }
-
- @Override
- public AbstractMultiSet deserialize(DataInputView source) throws IOException {
- final int size = source.readInt();
- final AbstractMultiSet multiSet = new HashMultiSet<>();
- for (int i = 0; i < size; i++) {
- multiSet.add(elementSerializer.deserialize(source));
- }
- return multiSet;
- }
-
- @Override
- public AbstractMultiSet deserialize(AbstractMultiSet reuse, DataInputView source)
- throws IOException {
- return deserialize(source);
- }
-
- @Override
- public void copy(DataInputView source, DataOutputView target) throws IOException {
- // copy number of elements
- final int num = source.readInt();
- target.writeInt(num);
- for (int i = 0; i < num; i++) {
- elementSerializer.copy(source, target);
- }
- }
-
- // --------------------------------------------------------------------
-
- @Override
- public boolean equals(Object obj) {
- return obj == this ||
- (obj != null && obj.getClass() == getClass() &&
- elementSerializer.equals(((MultisetSerializer>) obj).elementSerializer));
- }
-
- @Override
- public boolean canEqual(Object obj) {
- return true;
- }
-
- @Override
- public int hashCode() {
- return elementSerializer.hashCode();
- }
-
- // --------------------------------------------------------------------------------------------
- // Serializer configuration snapshotting & compatibility
- // --------------------------------------------------------------------------------------------
-
- @Override
- public CollectionSerializerConfigSnapshot snapshotConfiguration() {
- return new CollectionSerializerConfigSnapshot<>(elementSerializer);
- }
-
- @Override
- public CompatibilityResult> ensureCompatibility(
- TypeSerializerConfigSnapshot configSnapshot) {
- if (configSnapshot instanceof CollectionSerializerConfigSnapshot) {
- Tuple2, TypeSerializerConfigSnapshot> previousElemSerializerAndConfig =
- ((CollectionSerializerConfigSnapshot) configSnapshot).getSingleNestedSerializerAndConfig();
-
- CompatibilityResult compatResult = CompatibilityUtil.resolveCompatibilityResult(
- previousElemSerializerAndConfig.f0,
- UnloadableDummyTypeSerializer.class,
- previousElemSerializerAndConfig.f1,
- elementSerializer);
-
- if (!compatResult.isRequiresMigration()) {
- return CompatibilityResult.compatible();
- } else if (compatResult.getConvertDeserializer() != null) {
- return CompatibilityResult.requiresMigration(
- new MultisetSerializer<>(
- new TypeDeserializerAdapter<>(compatResult.getConvertDeserializer())));
- }
- }
-
- return CompatibilityResult.requiresMigration();
- }
-}
diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/MultisetTypeInfo.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/MultisetTypeInfo.java
index f861372b520cda..5aadc006bca5f8 100644
--- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/MultisetTypeInfo.java
+++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/MultisetTypeInfo.java
@@ -18,12 +18,11 @@
package org.apache.flink.api.java.typeutils;
-import org.apache.commons.collections4.multiset.AbstractMultiSet;
import org.apache.flink.annotation.PublicEvolving;
-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.TypeSerializer;
-import org.apache.flink.api.common.typeutils.base.MultisetSerializer;
+
+import java.util.Map;
import static org.apache.flink.util.Preconditions.checkNotNull;
@@ -33,19 +32,17 @@
* @param The type of the elements in the Multiset.
*/
@PublicEvolving
-public final class MultisetTypeInfo extends TypeInformation> {
+public final class MultisetTypeInfo extends MapTypeInfo {
private static final long serialVersionUID = 1L;
- private final TypeInformation elementTypeInfo;
-
public MultisetTypeInfo(Class elementTypeClass) {
- this.elementTypeInfo = of(checkNotNull(elementTypeClass, "elementTypeClass"));
+ super(elementTypeClass, Integer.class);
}
public MultisetTypeInfo(TypeInformation elementTypeInfo) {
- this.elementTypeInfo = checkNotNull(elementTypeInfo, "elementTypeInfo");
+ super(elementTypeInfo, BasicTypeInfo.INT_TYPE_INFO);
}
// ------------------------------------------------------------------------
@@ -56,7 +53,7 @@ public MultisetTypeInfo(TypeInformation elementTypeInfo) {
* Gets the type information for the elements contained in the Multiset
*/
public TypeInformation getElementTypeInfo() {
- return elementTypeInfo;
+ return getKeyTypeInfo();
}
// ------------------------------------------------------------------------
@@ -87,8 +84,8 @@ public int getTotalFields() {
@SuppressWarnings("unchecked")
@Override
- public Class> getTypeClass() {
- return (Class>)(Class>)AbstractMultiSet.class;
+ public Class