You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@reef.apache.org by yu...@apache.org on 2015/06/22 17:54:12 UTC

[05/17] incubator-reef git commit: Add Serializer which serialize/deserialize the ClassHierarchy from/to many type of formats

Add Serializer which serialize/deserialize the ClassHierarchy from/to many type of formats


Project: http://git-wip-us.apache.org/repos/asf/incubator-reef/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-reef/commit/7a095daf
Tree: http://git-wip-us.apache.org/repos/asf/incubator-reef/tree/7a095daf
Diff: http://git-wip-us.apache.org/repos/asf/incubator-reef/diff/7a095daf

Branch: refs/heads/REEF-395
Commit: 7a095daf13dd8d690c16ad31ce3abba3ec3b0120
Parents: 1d37095
Author: Yunseong Lee <yu...@apache.org>
Authored: Sat Jun 20 02:16:56 2015 +0900
Committer: Yunseong Lee <yu...@apache.org>
Committed: Sat Jun 20 02:16:56 2015 +0900

----------------------------------------------------------------------
 .../formats/AvroClassHierarchySerializer.java   | 254 +++++++++++++++++++
 .../tang/formats/ClassHierarchySerializer.java  |  43 ++++
 2 files changed, 297 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/7a095daf/lang/java/reef-tang/tang/src/main/java/org/apache/reef/tang/formats/AvroClassHierarchySerializer.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-tang/tang/src/main/java/org/apache/reef/tang/formats/AvroClassHierarchySerializer.java b/lang/java/reef-tang/tang/src/main/java/org/apache/reef/tang/formats/AvroClassHierarchySerializer.java
new file mode 100644
index 0000000..e4f52cf
--- /dev/null
+++ b/lang/java/reef-tang/tang/src/main/java/org/apache/reef/tang/formats/AvroClassHierarchySerializer.java
@@ -0,0 +1,254 @@
+package org.apache.reef.tang.formats;
+
+import org.apache.avro.file.DataFileReader;
+import org.apache.avro.file.DataFileWriter;
+import org.apache.avro.io.*;
+import org.apache.avro.specific.SpecificDatumReader;
+import org.apache.avro.specific.SpecificDatumWriter;
+import org.apache.reef.tang.ClassHierarchy;
+import org.apache.reef.tang.formats.avro.*;
+import org.apache.reef.tang.implementation.java.AvroClassHierarchy;
+import org.apache.reef.tang.types.*;
+
+import javax.inject.Inject;
+import java.io.*;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+/**
+ * (De-)Serializing ClassHierarchy to and from AvroClassHierarchy.
+ * <p/>
+ * This class is stateless and is therefore safe to reuse.
+ */
+public class AvroClassHierarchySerializer implements ClassHierarchySerializer {
+
+  /**
+   * The Charset used for the JSON encoding.
+   * <p/>
+   * Copied from <code>org.apache.avro.io.JsonDecoder.CHARSET</code>
+   */
+  private static final String JSON_CHARSET = "ISO-8859-1";
+
+  @Inject
+  public AvroClassHierarchySerializer() {
+  }
+
+  private static AvroNode newClassNode(final String name,
+                                       final String fullName,
+                                       final boolean isInjectionCandidate,
+                                       final boolean isExternalConstructor,
+                                       final boolean isUnit,
+                                       final List<AvroConstructorDef> injectableConstructors,
+                                       final List<AvroConstructorDef> otherConstructors,
+                                       final List<String> implFullNames,
+                                       final String defaultImplementation,
+                                       final List<AvroNode> children) {
+    return AvroNode.newBuilder()
+            .setName(name)
+            .setFullName(fullName)
+            .setClassNode(AvroClassNode.newBuilder()
+                    .setIsInjectionCandidate(isInjectionCandidate)
+                    .setIsExternalConstructor(isExternalConstructor)
+                    .setIsUnit(isUnit)
+                    .setInjectableConstructors(injectableConstructors)
+                    .setOtherConstructors(otherConstructors)
+                    .setImplFullNames(implFullNames)
+                    .setDefaultImplementation(defaultImplementation)
+                    .build())
+            .setChildren(children).build();
+  }
+
+  private static AvroNode newNamedParameterNode(final String name,
+                                                final String fullName,
+                                                final String simpleArgClassName,
+                                                final String fullArgClassName,
+                                                final boolean isSet,
+                                                final boolean isList,
+                                                final String documentation, // can be null
+                                                final String shortName, // can be null
+                                                final List<String> instanceDefault, // can be null
+                                                final List<AvroNode> children) {
+    final AvroNamedParameterNode.Builder namedParameterNodeBuilder = AvroNamedParameterNode.newBuilder()
+            .setSimpleArgClassName(simpleArgClassName)
+            .setFullArgClassName(fullArgClassName)
+            .setIsSet(isSet)
+            .setIsList(isList);
+      namedParameterNodeBuilder.setDocumentation(documentation);
+      namedParameterNodeBuilder.setShortName(shortName);
+      namedParameterNodeBuilder.setInstanceDefault(instanceDefault);
+
+    return AvroNode.newBuilder()
+            .setName(name)
+            .setFullName(fullName)
+            .setNamedParameterNode(namedParameterNodeBuilder.build())
+            .setChildren(children).build();
+  }
+
+  private static AvroNode newPackageNode(final String name,
+                                         final String fullName,
+                                         final List<AvroNode> children) {
+    return AvroNode.newBuilder()
+            .setPackageNode(AvroPackageNode.newBuilder().build())
+            .setName(name).setFullName(fullName).setChildren(children).build();
+  }
+
+  private static AvroConstructorDef newConstructorDef(
+          String fullClassName, List<AvroConstructorArg> args) {
+    return AvroConstructorDef.newBuilder()
+            .setFullArgClassName(fullClassName).setConstructorArg(args).build();
+  }
+
+  private static AvroConstructorArg newConstructorArg(final String fullArgClassName,
+                                                      final String namedParameterName,
+                                                      final boolean isFuture) {
+    AvroConstructorArg.Builder builder = AvroConstructorArg.newBuilder()
+            .setFullArgClassName(fullArgClassName)
+            .setIsInjectionFuture(isFuture);
+    if (namedParameterName != null) {
+      builder.setNamedParameterName(namedParameterName).build();
+    }
+    return builder.build();
+  }
+
+  private static AvroConstructorDef serializeConstructorDef(final ConstructorDef<?> def) {
+    final List<AvroConstructorArg> args = new ArrayList<>();
+    for (ConstructorArg arg : def.getArgs()) {
+      args.add(newConstructorArg(arg.getType(), arg.getNamedParameterName(), arg.isInjectionFuture()));
+    }
+    return newConstructorDef(def.getClassName(), args);
+  }
+
+  private static AvroNode toAvroNode(final Node n) {
+    final List<AvroNode> children = new ArrayList<>();
+    for (final Node child : n.getChildren()) {
+      children.add(toAvroNode(child));
+    }
+    if (n instanceof ClassNode) {
+      final ClassNode<?> cn = (ClassNode<?>) n;
+      final ConstructorDef<?>[] injectable = cn.getInjectableConstructors();
+      final ConstructorDef<?>[] all = cn.getAllConstructors();
+      final List<ConstructorDef<?>> others = new ArrayList<>(Arrays.asList(all));
+      others.removeAll(Arrays.asList(injectable));
+
+      final List<AvroConstructorDef> injectableConstructors = new ArrayList<>();
+      for (final ConstructorDef<?> inj : injectable) {
+        injectableConstructors.add(serializeConstructorDef(inj));
+      }
+      final List<AvroConstructorDef> otherConstructors = new ArrayList<>();
+      for (final ConstructorDef<?> other : others) {
+        otherConstructors.add(serializeConstructorDef(other));
+      }
+      final List<String> implFullNames = new ArrayList<>();
+      for (final ClassNode<?> impl : cn.getKnownImplementations()) {
+        implFullNames.add(impl.getFullName());
+      }
+      return newClassNode(n.getName(), n.getFullName(), cn.isInjectionCandidate(), cn.isExternalConstructor(),
+              cn.isUnit(), injectableConstructors, otherConstructors, implFullNames, cn.getDefaultImplementation(), children);
+    } else if (n instanceof NamedParameterNode) {
+      final NamedParameterNode<?> np = (NamedParameterNode<?>) n;
+      return newNamedParameterNode(np.getName(), np.getFullName(), np.getSimpleArgName(), np.getFullArgName(),
+              np.isSet(), np.isList(), np.getDocumentation(), np.getShortName(),
+              Arrays.asList(np.getDefaultInstanceAsStrings()), children);
+    } else if (n instanceof PackageNode) {
+      return newPackageNode(n.getName(), n.getFullName(), children);
+    } else {
+      throw new IllegalStateException("Encountered unknown type of Node: " + n);
+    }
+  }
+
+  public AvroClassHierarchy toAvro(final ClassHierarchy classHierarchy) {
+    return new AvroClassHierarchy(toAvroNode(classHierarchy.getNamespace()));
+  }
+
+  @Override
+  public void toFile(final ClassHierarchy classHierarchy, final File file) throws IOException {
+    final AvroNode avroNode = toAvroNode(classHierarchy.getNamespace());
+    final DatumWriter<AvroNode> avroNodeWriter = new SpecificDatumWriter<>(AvroNode.class);
+    try (DataFileWriter<AvroNode> dataFileWriter = new DataFileWriter<>(avroNodeWriter)) {
+      dataFileWriter.create(avroNode.getSchema(), file);
+      dataFileWriter.append(avroNode);
+    }
+  }
+
+  @Override
+  public void toTextFile(ClassHierarchy classHierarchy, File file) throws IOException {
+    try (final Writer w = new FileWriter(file)) {
+      w.write(this.toString(classHierarchy));
+    }
+  }
+
+  @Override
+  public byte[] toByteArray(final ClassHierarchy classHierarchy) throws IOException {
+    final DatumWriter<AvroNode> requestWriter = new SpecificDatumWriter<>(AvroNode.class);
+    try (final ByteArrayOutputStream out = new ByteArrayOutputStream()) {
+      final BinaryEncoder encoder = EncoderFactory.get().binaryEncoder(out, null);
+      // TODO Should be a better way
+      requestWriter.write(toAvroNode(classHierarchy.getNamespace()), encoder);
+      encoder.flush();
+      return out.toByteArray();
+    } catch (final IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  // TODO Improve the interface?
+  @Override
+  public String toString(ClassHierarchy classHierarchy) throws IOException {
+    final DatumWriter<AvroNode> configurationWriter = new SpecificDatumWriter<>(AvroNode.class);
+    try (final ByteArrayOutputStream out = new ByteArrayOutputStream()) {
+      final JsonEncoder encoder = EncoderFactory.get().jsonEncoder(AvroConfiguration.SCHEMA$, out);
+      configurationWriter.write(toAvroNode(classHierarchy.getNamespace()), encoder);
+      encoder.flush();
+      out.flush();
+      return out.toString(JSON_CHARSET);
+    } catch (final IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @Override
+  public ClassHierarchy fromFile(final File file) throws IOException {
+    final AvroNode avroNode;
+    try (final DataFileReader<AvroNode> dataFileReader =
+                 new DataFileReader<>(file, new SpecificDatumReader<>(AvroNode.class))) {
+      avroNode = dataFileReader.next();
+    }
+    return new AvroClassHierarchy(avroNode);
+  }
+
+  @Override
+  public ClassHierarchy fromTextFile(final File file) throws IOException {
+    final StringBuilder stringBuilder = new StringBuilder();
+    try (final BufferedReader reader = new BufferedReader(new FileReader(file))) {
+      String line = reader.readLine();
+      while (line != null) {
+        stringBuilder.append(line);
+        line = reader.readLine();
+      }
+    }
+    return fromString(stringBuilder.toString());
+  }
+
+  @Override
+  public ClassHierarchy fromByteArray(final byte[] theBytes) throws IOException {
+    try {
+      final BinaryDecoder decoder = DecoderFactory.get().binaryDecoder(theBytes, null);
+      final SpecificDatumReader<AvroNode> reader = new SpecificDatumReader<>(AvroNode.class);
+      return new AvroClassHierarchy(reader.read(null, decoder));
+    } catch (final IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @Override
+  public ClassHierarchy fromString(final String theString) throws IOException {
+    try {
+      final JsonDecoder decoder = DecoderFactory.get().jsonDecoder(AvroNode.getClassSchema(), theString);
+      final SpecificDatumReader<AvroNode> reader = new SpecificDatumReader<>(AvroNode.class);
+      return new AvroClassHierarchy(reader.read(null, decoder));
+    } catch (final IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/7a095daf/lang/java/reef-tang/tang/src/main/java/org/apache/reef/tang/formats/ClassHierarchySerializer.java
----------------------------------------------------------------------
diff --git a/lang/java/reef-tang/tang/src/main/java/org/apache/reef/tang/formats/ClassHierarchySerializer.java b/lang/java/reef-tang/tang/src/main/java/org/apache/reef/tang/formats/ClassHierarchySerializer.java
new file mode 100644
index 0000000..bf3918b
--- /dev/null
+++ b/lang/java/reef-tang/tang/src/main/java/org/apache/reef/tang/formats/ClassHierarchySerializer.java
@@ -0,0 +1,43 @@
+package org.apache.reef.tang.formats;
+
+import org.apache.reef.tang.ClassHierarchy;
+import org.apache.reef.tang.annotations.DefaultImplementation;
+import org.apache.reef.tang.formats.avro.AvroNode;
+
+import java.io.*;
+
+/**
+ *
+ */
+@DefaultImplementation(org.apache.reef.tang.formats.AvroClassHierarchySerializer.class)
+public interface ClassHierarchySerializer {
+  /**
+   * serialize a class hierarchy into a file.
+   *
+   * @param file
+   * @param classHierarchy
+   * @throws IOException
+   */
+  void toFile(final ClassHierarchy classHierarchy, final File file) throws IOException;
+
+  void toTextFile(final ClassHierarchy classHierarchy, final File file) throws IOException;
+
+  byte[] toByteArray(final ClassHierarchy classHierarchy) throws IOException;
+
+  String toString(final ClassHierarchy classHierarchy) throws IOException;
+
+  /**
+   * Deserialize a class hierarchy from a file. The file can be generated from either Java or C#
+   *
+   * @param file
+   * @return
+   * @throws IOException
+   */
+  ClassHierarchy fromFile(final File file) throws IOException;
+
+  ClassHierarchy fromTextFile(final File file) throws IOException;
+
+  ClassHierarchy fromByteArray(final byte[] theBytes) throws IOException;
+
+  ClassHierarchy fromString(final String theString) throws IOException;
+}