提交 a07dba6f 编写于 作者: T twalthr 提交者: Fabian Hueske

Introduction of a Writable type and tests

上级 cf18cde4
......@@ -277,7 +277,7 @@ public abstract class ExecutionEnvironment {
@SuppressWarnings("unchecked")
TypeInformation<X> producedType = (inputFormat instanceof ResultTypeQueryable) ?
((ResultTypeQueryable<X>) inputFormat).getProducedType() :
TypeExtractor.extractInputFormatTypes(inputFormat);
TypeExtractor.getInputFormatTypes(inputFormat);
return createInput(inputFormat, producedType);
}
......
......@@ -120,7 +120,7 @@ public abstract class Keys<T> {
public SelectorFunctionKeys(KeySelector<T, K> keyExtractor, TypeInformation<T> type) {
this.keyExtractor = keyExtractor;
this.keyType = TypeExtractor.getKeyExtractorType(keyExtractor, type);
this.keyType = TypeExtractor.getKeySelectorTypes(keyExtractor, type);
}
public TypeInformation<K> getKeyType() {
......
......@@ -22,6 +22,7 @@ import java.lang.reflect.TypeVariable;
import java.util.ArrayList;
import org.apache.commons.lang3.Validate;
import org.apache.hadoop.io.Writable;
import eu.stratosphere.api.common.io.InputFormat;
import eu.stratosphere.api.java.functions.CoGroupFunction;
......@@ -74,13 +75,13 @@ public class TypeExtractor {
return createTypeInfo(CrossFunction.class, crossFunction.getClass(), 2, in1Type, in2Type);
}
public static <IN, OUT> TypeInformation<OUT> getKeyExtractorType(KeySelector<IN, OUT> selector, TypeInformation<IN> inType) {
public static <IN, OUT> TypeInformation<OUT> getKeySelectorTypes(KeySelector<IN, OUT> selector, TypeInformation<IN> inType) {
validateInputType(KeySelector.class, selector.getClass(), 0, inType);
return createTypeInfo(KeySelector.class, selector.getClass(), 1, inType, null);
}
public static <IN> TypeInformation<IN> extractInputFormatTypes(InputFormat<IN, ?> format) {
throw new UnsupportedOperationException("not implemented yet");
public static <IN> TypeInformation<IN> getInputFormatTypes(InputFormat<IN, ?> inputFormat) {
return createTypeInfo(InputFormat.class, inputFormat.getClass(), 0, null, null);
}
// --------------------------------------------------------------------------------------------
......@@ -203,6 +204,21 @@ public class TypeExtractor {
validateInfo(new ArrayList<Type>(typeHierarchy), subTypes[i], ((TupleTypeInfo<?>) typeInfo).getTypeAt(i));
}
}
// check for Writable
else if (typeInfo instanceof WritableTypeInfo<?>) {
// check if writable at all
if (!(type instanceof Class<?> && Writable.class.isAssignableFrom((Class<?>) type))) {
throw new InvalidTypesException("Writable type expected.");
}
// check writable type contents
Class<?> clazz = null;
if (((WritableTypeInfo<?>) typeInfo).getTypeClass() != (clazz = (Class<?>) type)) {
throw new InvalidTypesException("Writable type '"
+ ((WritableTypeInfo<?>) typeInfo).getTypeClass().getCanonicalName() + "' expected but was '"
+ clazz.getCanonicalName() + "'.");
}
}
// check for basic array
else if (typeInfo instanceof BasicArrayTypeInfo<?, ?>) {
Type component = null;
......@@ -566,6 +582,11 @@ public class TypeExtractor {
}
}
// check for writable types
if(Writable.class.isAssignableFrom(clazz)) {
return (TypeInformation<X>) WritableTypeInfo.getWritableTypeInfo((Class<? extends Writable>) clazz);
}
// check for basic types
TypeInformation<X> basicTypeInfo = BasicTypeInfo.getInfoFor(clazz);
if (basicTypeInfo != null) {
......
......@@ -38,8 +38,10 @@ public abstract class TypeInformation<T> {
private static final String TUPLE_PACKAGE = "eu.stratosphere.api.java.tuple";
private static final String VALUE_PACKAGE = "eu.stratosphere.types";
private static final String WRITABLE_PACKAGE = "org.apache.hadoop.io";
private static final Pattern tuplePattern = Pattern.compile("^((" + TUPLE_PACKAGE.replaceAll("\\.", "\\\\.") + "\\.)?Tuple[0-9]+)<");
private static final Pattern writablePattern = Pattern.compile("^((" + WRITABLE_PACKAGE.replaceAll("\\.", "\\\\.") + "\\.)?Writable)<([^\\s,>]*)(,|>|$)");
private static final Pattern basicTypePattern = Pattern
.compile("^((java\\.lang\\.)?(String|Integer|Byte|Short|Character|Double|Float|Long|Boolean))(,|>|$)");
private static final Pattern basicType2Pattern = Pattern.compile("^(int|byte|short|char|double|float|long|boolean)(,|>|$)");
......@@ -61,13 +63,14 @@ public abstract class TypeInformation<T> {
* <code>String[]</code>, etc.
* <li>Tuple types such as <code>Tuple1&lt;TYPE0&gt;</code>,
* <code>Tuple2&lt;TYPE0, TYPE1&gt;</code>, etc.</li>
* <li>Custom types such as <code>org.my.CustomObject</code>,
* <code>org.my.CustomObject$StaticInnerClass</code>, etc.
* <li>Custom type arrays such as <code>org.my.CustomObject[]</code>,
* <code>org.my.CustomObject$InnerClass[]</code>, etc.
* <li>Custom types such as <code>org.my.CustomClass</code>,
* <code>org.my.CustomClass$StaticInnerClass</code>, etc.
* <li>Custom type arrays such as <code>org.my.CustomClass[]</code>,
* <code>org.my.CustomClass$StaticInnerClass[]</code>, etc.
* <li>Value types such as <code>DoubleValue</code>,
* <code>StringValue</code>, <code>IntegerValue</code>, etc.</li>
* <li>Tuple arrays such as <code>Tuple2<TYPE0,TYPE1>[], etc.</code></li>
* <li>Tuple array types such as <code>Tuple2<TYPE0,TYPE1>[], etc.</code></li>
* <li>Writable types such as <code>Writable&lt;org.my.CustomWritable&gt;</code></li>
* </ul>
*
* Example:
......@@ -99,6 +102,8 @@ public abstract class TypeInformation<T> {
String infoString = sb.toString();
final Matcher tupleMatcher = tuplePattern.matcher(infoString);
final Matcher writableMatcher = writablePattern.matcher(infoString);
final Matcher basicTypeMatcher = basicTypePattern.matcher(infoString);
final Matcher basicType2Matcher = basicType2Pattern.matcher(infoString);
......@@ -162,6 +167,20 @@ public abstract class TypeInformation<T> {
returnType = new TupleTypeInfo(clazz, types);
}
}
// writable types
else if (writableMatcher.find()) {
String className = writableMatcher.group(1);
String fullyQualifiedName = writableMatcher.group(3);
sb.delete(0, className.length() + 1 + fullyQualifiedName.length());
try {
Class<?> clazz = Class.forName(fullyQualifiedName);
returnType = WritableTypeInfo.getWritableTypeInfo((Class) clazz);
} catch (ClassNotFoundException e) {
throw new IllegalArgumentException("Class '" + fullyQualifiedName
+ "' could not be found for use as writable type. Please note that inner classes must be declared static.");
}
}
// basic types of classes
else if (basicTypeMatcher.find()) {
String className = basicTypeMatcher.group(1);
......
......@@ -34,7 +34,7 @@ public class ValueTypeInfo<T extends Value> extends TypeInformation<T> implement
if (type == null) {
throw new NullPointerException();
}
if (!Value.class.isAssignableFrom(type)) {
if (!Value.class.isAssignableFrom(type) && !type.equals(Value.class)) {
throw new IllegalArgumentException("ValueTypeInfo can only be used for subclasses of " + Value.class.getName());
}
......
/***********************************************************************************************************************
*
* Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu)
*
* Licensed 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 eu.stratosphere.api.java.typeutils;
import org.apache.hadoop.io.Writable;
import eu.stratosphere.api.common.typeutils.TypeComparator;
import eu.stratosphere.api.common.typeutils.TypeSerializer;
import eu.stratosphere.api.java.functions.InvalidTypesException;
import eu.stratosphere.api.java.typeutils.runtime.WritableComparator;
import eu.stratosphere.api.java.typeutils.runtime.WritableSerializer;
public class WritableTypeInfo<T extends Writable> extends TypeInformation<T> implements AtomicType<T> {
private final Class<T> typeClass;
public WritableTypeInfo(Class<T> typeClass) {
if (typeClass == null) {
throw new NullPointerException();
}
if (!Writable.class.isAssignableFrom(typeClass) || typeClass == Writable.class) {
throw new IllegalArgumentException("WritableTypeInfo can only be used for subclasses of " + Writable.class.getName());
}
this.typeClass = typeClass;
}
@SuppressWarnings({ "rawtypes", "unchecked" })
@Override
public TypeComparator<T> createComparator(boolean sortOrderAscending) {
if(Comparable.class.isAssignableFrom(typeClass)) {
return new WritableComparator(sortOrderAscending, typeClass);
}
else {
throw new UnsupportedOperationException("Writable does not implement Comparable interface.");
}
}
@Override
public boolean isBasicType() {
return false;
}
@Override
public boolean isTupleType() {
return false;
}
@Override
public int getArity() {
return 1;
}
@Override
public Class<T> getTypeClass() {
return this.typeClass;
}
@Override
public boolean isKeyType() {
return Comparable.class.isAssignableFrom(typeClass);
}
@Override
public TypeSerializer<T> createSerializer() {
return new WritableSerializer<T>(typeClass);
}
@Override
public String toString() {
return "WritableType<" + typeClass.getName() + ">";
}
// --------------------------------------------------------------------------------------------
static final <T extends Writable> TypeInformation<T> getWritableTypeInfo(Class<T> typeClass) {
if (Writable.class.isAssignableFrom(typeClass) && !typeClass.equals(Writable.class)) {
return new WritableTypeInfo<T>(typeClass);
}
else {
throw new InvalidTypesException("The given class is no subclass of " + Writable.class.getName());
}
}
}
/***********************************************************************************************************************
*
* Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu)
*
* Licensed 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 eu.stratosphere.api.java.typeutils.runtime;
import java.io.IOException;
import org.apache.hadoop.io.Writable;
import com.esotericsoftware.kryo.Kryo;
import eu.stratosphere.api.common.typeutils.TypeComparator;
import eu.stratosphere.core.memory.DataInputView;
import eu.stratosphere.core.memory.DataOutputView;
import eu.stratosphere.core.memory.MemorySegment;
import eu.stratosphere.types.NormalizableKey;
import eu.stratosphere.util.InstantiationUtil;
public class WritableComparator<T extends Writable & Comparable<T>> extends TypeComparator<T> {
private static final long serialVersionUID = 1L;
private Class<T> type;
private final boolean ascendingComparison;
private transient T reference;
private transient T tempReference;
private transient Kryo kryo;
public WritableComparator(boolean ascending, Class<T> type) {
this.type = type;
this.ascendingComparison = ascending;
}
@Override
public int hash(T record) {
return record.hashCode();
}
@Override
public void setReference(T toCompare) {
checkKryoInitialized();
reference = this.kryo.copy(toCompare);
}
@Override
public boolean equalToReference(T candidate) {
return candidate.equals(reference);
}
@Override
public int compareToReference(TypeComparator<T> referencedComparator) {
T otherRef = ((WritableComparator<T>) referencedComparator).reference;
int comp = otherRef.compareTo(reference);
return ascendingComparison ? comp : -comp;
}
@Override
public int compare(T first, T second) {
int comp = first.compareTo(second);
return ascendingComparison ? comp : -comp;
}
@Override
public int compare(DataInputView firstSource, DataInputView secondSource) throws IOException {
ensureReferenceInstantiated();
ensureTempReferenceInstantiated();
reference.readFields(firstSource);
tempReference.readFields(secondSource);
int comp = reference.compareTo(tempReference);
return ascendingComparison ? comp : -comp;
}
@Override
public boolean supportsNormalizedKey() {
return NormalizableKey.class.isAssignableFrom(type);
}
@Override
public int getNormalizeKeyLen() {
ensureReferenceInstantiated();
NormalizableKey<?> key = (NormalizableKey<?>) reference;
return key.getMaxNormalizedKeyLen();
}
@Override
public boolean isNormalizedKeyPrefixOnly(int keyBytes) {
return keyBytes < getNormalizeKeyLen();
}
@Override
public void putNormalizedKey(T record, MemorySegment target, int offset, int numBytes) {
NormalizableKey<?> key = (NormalizableKey<?>) record;
key.copyNormalizedKey(target, offset, numBytes);
}
@Override
public boolean invertNormalizedKey() {
return !ascendingComparison;
}
@Override
public TypeComparator<T> duplicate() {
return new WritableComparator<T>(ascendingComparison, type);
}
// --------------------------------------------------------------------------------------------
// unsupported normalization
// --------------------------------------------------------------------------------------------
@Override
public boolean supportsSerializationWithKeyNormalization() {
return false;
}
@Override
public void writeWithKeyNormalization(T record, DataOutputView target) throws IOException {
throw new UnsupportedOperationException();
}
@Override
public T readWithKeyDenormalization(T reuse, DataInputView source) throws IOException {
throw new UnsupportedOperationException();
}
// --------------------------------------------------------------------------------------------
private final void checkKryoInitialized() {
if (this.kryo == null) {
this.kryo = new Kryo();
this.kryo.setAsmEnabled(true);
this.kryo.register(type);
}
}
private final void ensureReferenceInstantiated() {
if (reference == null) {
reference = InstantiationUtil.instantiate(type, Writable.class);
}
}
private final void ensureTempReferenceInstantiated() {
if (tempReference == null) {
tempReference = InstantiationUtil.instantiate(type, Writable.class);
}
}
}
/***********************************************************************************************************************
*
* Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu)
*
* Licensed 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 eu.stratosphere.api.java.typeutils.runtime;
import java.io.IOException;
import org.apache.hadoop.io.Writable;
import com.esotericsoftware.kryo.Kryo;
import eu.stratosphere.api.common.typeutils.TypeSerializer;
import eu.stratosphere.core.memory.DataInputView;
import eu.stratosphere.core.memory.DataOutputView;
import eu.stratosphere.util.InstantiationUtil;
public class WritableSerializer<T extends Writable> extends TypeSerializer<T> {
private static final long serialVersionUID = 1L;
private final Class<T> typeClass;
private transient Kryo kryo;
private transient T copyInstance;
public WritableSerializer(Class<T> typeClass) {
this.typeClass = typeClass;
}
@Override
public T createInstance() {
return InstantiationUtil.instantiate(typeClass);
}
@Override
public T copy(T from, T reuse) {
checkKryoInitialized();
reuse = this.kryo.copy(from);
return reuse;
}
@Override
public int getLength() {
return -1;
}
@Override
public void serialize(T record, DataOutputView target) throws IOException {
record.write(target);
}
@Override
public T deserialize(T reuse, DataInputView source) throws IOException {
reuse.readFields(source);
return reuse;
}
@Override
public void copy(DataInputView source, DataOutputView target) throws IOException {
ensureInstanceInstantiated();
copyInstance.readFields(source);
copyInstance.write(target);
}
@Override
public boolean isImmutableType() {
return false;
}
@Override
public boolean isStateful() {
return true;
}
// --------------------------------------------------------------------------------------------
private final void ensureInstanceInstantiated() {
if (copyInstance == null) {
copyInstance = createInstance();
}
}
private final void checkKryoInitialized() {
if (this.kryo == null) {
this.kryo = new Kryo();
this.kryo.setAsmEnabled(true);
this.kryo.register(typeClass);
}
}
}
......@@ -14,8 +14,12 @@
**********************************************************************************************************************/
package eu.stratosphere.api.java.type.extractor;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import java.util.Iterator;
import org.apache.hadoop.io.Writable;
import org.junit.Assert;
import org.junit.Test;
......@@ -40,6 +44,7 @@ import eu.stratosphere.api.java.typeutils.TupleTypeInfo;
import eu.stratosphere.api.java.typeutils.TypeExtractor;
import eu.stratosphere.api.java.typeutils.TypeInformation;
import eu.stratosphere.api.java.typeutils.ValueTypeInfo;
import eu.stratosphere.api.java.typeutils.WritableTypeInfo;
import eu.stratosphere.types.DoubleValue;
import eu.stratosphere.types.IntValue;
import eu.stratosphere.types.StringValue;
......@@ -75,6 +80,38 @@ public class TypeExtractorTest {
// use getForObject()
Assert.assertEquals(BasicTypeInfo.BOOLEAN_TYPE_INFO, TypeExtractor.getForObject(Boolean.valueOf(true)));
}
public static class MyWritable implements Writable {
@Override
public void write(DataOutput out) throws IOException {
}
@Override
public void readFields(DataInput in) throws IOException {
}
}
@SuppressWarnings({ "unchecked", "rawtypes" })
@Test
public void testWritableType() {
MapFunction<?, ?> function = new MapFunction<MyWritable, MyWritable>() {
private static final long serialVersionUID = 1L;
@Override
public MyWritable map(MyWritable value) throws Exception {
return null;
}
};
TypeInformation<?> ti = TypeExtractor.getMapReturnTypes(function, (TypeInformation) new WritableTypeInfo<MyWritable>(MyWritable.class));
Assert.assertTrue(ti instanceof WritableTypeInfo<?>);
Assert.assertEquals(MyWritable.class, ((WritableTypeInfo<?>) ti).getTypeClass());
}
@SuppressWarnings({ "unchecked", "rawtypes" })
@Test
......@@ -344,7 +381,7 @@ public class TypeExtractorTest {
}
};
TypeInformation<?> ti = TypeExtractor.getKeyExtractorType(function, (TypeInformation) TypeInformation.parse("StringValue"));
TypeInformation<?> ti = TypeExtractor.getKeySelectorTypes(function, (TypeInformation) TypeInformation.parse("StringValue"));
Assert.assertFalse(ti.isBasicType());
Assert.assertFalse(ti.isTupleType());
......@@ -1249,5 +1286,21 @@ public class TypeExtractorTest {
} catch (InvalidTypesException e) {
// right
}
MapFunction<?, ?> function4 = new MapFunction<Writable, String>() {
private static final long serialVersionUID = 1L;
@Override
public String map(Writable value) throws Exception {
return null;
}
};
try {
TypeExtractor.getMapReturnTypes(function4, (TypeInformation) new WritableTypeInfo<MyWritable>(MyWritable.class));
Assert.fail("exception expected");
} catch (InvalidTypesException e) {
// right
}
}
}
......@@ -14,8 +14,13 @@
**********************************************************************************************************************/
package eu.stratosphere.api.java.typeutils;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import junit.framework.Assert;
import org.apache.hadoop.io.Writable;
import org.junit.Test;
import eu.stratosphere.api.java.typeutils.BasicArrayTypeInfo;
......@@ -145,6 +150,27 @@ public class TypeInfoParserTest {
Assert.assertEquals(Class.class, ((GenericTypeInfo<?>) ti).getTypeClass());
}
public static class MyWritable implements Writable {
@Override
public void write(DataOutput out) throws IOException {
}
@Override
public void readFields(DataInput in) throws IOException {
}
}
@Test
public void testWritableType() {
TypeInformation<?> ti = TypeInformation.parse("Writable<eu.stratosphere.api.java.typeutils.TypeInfoParserTest$MyWritable>");
Assert.assertTrue(ti instanceof WritableTypeInfo<?>);
Assert.assertEquals(MyWritable.class, ((WritableTypeInfo<?>) ti).getTypeClass());
}
@Test
public void testObjectArrays() {
TypeInformation<?> ti = TypeInformation.parse("java.lang.Class[]");
......
/***********************************************************************************************************************
*
* Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu)
*
* Licensed 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 eu.stratosphere.api.java.typeutils.runtime;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import org.apache.hadoop.io.Writable;
public class StringArrayWritable implements Writable, Comparable<StringArrayWritable> {
private String[] array = new String[0];
public StringArrayWritable() {
super();
}
public StringArrayWritable(String[] array) {
this.array = array;
}
@Override
public void write(DataOutput out) throws IOException {
out.writeInt(this.array.length);
for(String str : this.array) {
byte[] b = str.getBytes();
out.writeInt(b.length);
out.write(b);
}
}
@Override
public void readFields(DataInput in) throws IOException {
this.array = new String[in.readInt()];
for(int i = 0; i < this.array.length; i++) {
byte[] b = new byte[in.readInt()];
in.readFully(b);
this.array[i] = new String(b);
}
}
@Override
public int compareTo(StringArrayWritable o) {
if(this.array.length != o.array.length) {
return this.array.length - o.array.length;
}
for(int i = 0; i < this.array.length; i++) {
int comp = this.array[i].compareTo(o.array[i]);
if(comp != 0) {
return comp;
}
}
return 0;
}
@Override
public boolean equals(Object obj) {
if(!(obj instanceof StringArrayWritable)) {
return false;
}
return this.compareTo((StringArrayWritable) obj) == 0;
}
}
\ No newline at end of file
/***********************************************************************************************************************
*
* Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu)
*
* Licensed 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 eu.stratosphere.api.java.typeutils.runtime;
import eu.stratosphere.api.common.typeutils.ComparatorTestBase;
import eu.stratosphere.api.common.typeutils.TypeComparator;
import eu.stratosphere.api.common.typeutils.TypeSerializer;
public class WritableComparatorTest extends ComparatorTestBase<StringArrayWritable> {
StringArrayWritable[] data = new StringArrayWritable[]{
new StringArrayWritable(new String[]{}),
new StringArrayWritable(new String[]{""}),
new StringArrayWritable(new String[]{"a","a"}),
new StringArrayWritable(new String[]{"a","b"}),
new StringArrayWritable(new String[]{"c","c"}),
new StringArrayWritable(new String[]{"d","f"}),
new StringArrayWritable(new String[]{"d","m"}),
new StringArrayWritable(new String[]{"z","x"}),
new StringArrayWritable(new String[]{"a","a", "a"})
};
@Override
protected TypeComparator<StringArrayWritable> createComparator(boolean ascending) {
return new WritableComparator<StringArrayWritable>(ascending, StringArrayWritable.class);
}
@Override
protected TypeSerializer<StringArrayWritable> createSerializer() {
return new WritableSerializer<StringArrayWritable>(StringArrayWritable.class);
}
@Override
protected StringArrayWritable[] getSortedTestData() {
return data;
}
}
/***********************************************************************************************************************
*
* Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu)
*
* Licensed 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 eu.stratosphere.api.java.typeutils.runtime;
import org.junit.Test;
import eu.stratosphere.api.common.typeutils.SerializerTestInstance;
import eu.stratosphere.api.java.typeutils.TypeExtractor;
import eu.stratosphere.api.java.typeutils.WritableTypeInfo;
public class WritableSerializerTest {
@Test
public void testStringArrayWritable() {
StringArrayWritable[] data = new StringArrayWritable[]{
new StringArrayWritable(new String[]{}),
new StringArrayWritable(new String[]{""}),
new StringArrayWritable(new String[]{"a","a"}),
new StringArrayWritable(new String[]{"a","b"}),
new StringArrayWritable(new String[]{"c","c"}),
new StringArrayWritable(new String[]{"d","f"}),
new StringArrayWritable(new String[]{"d","m"}),
new StringArrayWritable(new String[]{"z","x"}),
new StringArrayWritable(new String[]{"a","a", "a"})
};
WritableTypeInfo<StringArrayWritable> writableTypeInfo = (WritableTypeInfo<StringArrayWritable>) TypeExtractor.getForObject(data[0]);
WritableSerializer<StringArrayWritable> writableSerializer = (WritableSerializer<StringArrayWritable>) writableTypeInfo.createSerializer();
SerializerTestInstance<StringArrayWritable> testInstance = new SerializerTestInstance<StringArrayWritable>(writableSerializer,writableTypeInfo.getTypeClass(), -1, data);
testInstance.testAll();
}
}
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册