Repository: incubator-reef
Updated Branches:
  refs/heads/master 2c994ec97 -> 6702ed586


[REEF-463] Consolidate EvaluatorRequest serialization

This addressed the issue by
  * Defining Avro schema for EvaluatorRequest
  * Implementing Avro EvaluatorRequest serialization
  * Replacing use of `EvaluatorRequestSerializer` in our code with
    `AvroEvaluatorRequestSerializer`
  * Deleting `EvaluatorRequestSerializer`

JIRA:
  [REEF-463](https://issues.apache.org/jira/browse/REEF-463)

Pull Request:
  This closes #297


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

Branch: refs/heads/master
Commit: 6702ed58607d0792325c384d8af51c62c1b69c28
Parents: 2c994ec
Author: Mariia Mykhailova <[email protected]>
Authored: Wed Jul 15 13:38:48 2015 -0700
Committer: Markus Weimer <[email protected]>
Committed: Tue Aug 4 10:44:06 2015 -0700

----------------------------------------------------------------------
 .../reef-io/src/main/avro/EvaluatorRequest.avsc |  32 ++++++
 .../reef/io/data/loading/api/DataLoader.java    |  10 +-
 .../loading/api/DataLoadingRequestBuilder.java  |   8 +-
 .../impl/AvroEvaluatorRequestSerializer.java    | 109 +++++++++++++++++++
 .../impl/EvaluatorRequestSerializer.java        |  87 ---------------
 .../AvroEvaluatorRequestSerializerTest.java     |  66 +++++++++++
 .../impl/EvaluatorRequestSerializerTest.java    |  67 ------------
 7 files changed, 216 insertions(+), 163 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6702ed58/lang/java/reef-io/src/main/avro/EvaluatorRequest.avsc
----------------------------------------------------------------------
diff --git a/lang/java/reef-io/src/main/avro/EvaluatorRequest.avsc 
b/lang/java/reef-io/src/main/avro/EvaluatorRequest.avsc
new file mode 100644
index 0000000..ae06969
--- /dev/null
+++ b/lang/java/reef-io/src/main/avro/EvaluatorRequest.avsc
@@ -0,0 +1,32 @@
+/*
+ * 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.
+ */
+[
+  {
+    "namespace": "org.apache.reef.io.data.loading.avro",
+    "type": "record",
+    "name": "AvroEvaluatorRequest",
+    "fields": [
+      {"name": "megaBytes", "type": "int"},
+      {"name": "number", "type": "int"},
+      {"name": "cores", "type": "int"},
+      {"name": "nodeNames", "type": {"type": "array", "items": "string"}},
+      {"name": "rackNames", "type": {"type": "array", "items": "string"}}
+    ]
+  }
+]
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6702ed58/lang/java/reef-io/src/main/java/org/apache/reef/io/data/loading/api/DataLoader.java
----------------------------------------------------------------------
diff --git 
a/lang/java/reef-io/src/main/java/org/apache/reef/io/data/loading/api/DataLoader.java
 
b/lang/java/reef-io/src/main/java/org/apache/reef/io/data/loading/api/DataLoader.java
index f3b0f4c..5e38d8c 100644
--- 
a/lang/java/reef-io/src/main/java/org/apache/reef/io/data/loading/api/DataLoader.java
+++ 
b/lang/java/reef-io/src/main/java/org/apache/reef/io/data/loading/api/DataLoader.java
@@ -25,7 +25,7 @@ import org.apache.reef.driver.evaluator.AllocatedEvaluator;
 import org.apache.reef.driver.evaluator.EvaluatorRequest;
 import org.apache.reef.driver.evaluator.EvaluatorRequestor;
 import org.apache.reef.driver.evaluator.FailedEvaluator;
-import org.apache.reef.io.data.loading.impl.EvaluatorRequestSerializer;
+import org.apache.reef.io.data.loading.impl.AvroEvaluatorRequestSerializer;
 import org.apache.reef.io.network.util.Pair;
 import org.apache.reef.tang.Configuration;
 import org.apache.reef.tang.annotations.Parameter;
@@ -100,7 +100,7 @@ public class DataLoader {
       @Parameter(DataLoadingRequestBuilder.DataLoadingComputeRequest.class) 
final String serializedComputeRequest) {
     this(clock, requestor, dataLoadingService, new HashSet<String>(
         Arrays.asList(serializedComputeRequest)), new HashSet<String>(
-        Arrays.asList(EvaluatorRequestSerializer.serialize(EvaluatorRequest
+        Arrays.asList(AvroEvaluatorRequestSerializer.toString(EvaluatorRequest
             .newBuilder().setMemory(dataEvalMemoryMB)
             .setNumberOfCores(dataEvalCore).build()))));
   }
@@ -155,7 +155,7 @@ public class DataLoader {
       // Keep the maximum number of cores and memory requested, in case some
       // evaluator fails, we will try to reallocate based on that.
       for (final String serializedComputeRequest : serializedComputeRequests) {
-        final EvaluatorRequest computeRequest = 
EvaluatorRequestSerializer.deserialize(serializedComputeRequest);
+        final EvaluatorRequest computeRequest = 
AvroEvaluatorRequestSerializer.fromString(serializedComputeRequest);
         this.numComputeRequestsToSubmit.addAndGet(computeRequest.getNumber());
         this.computeEvalMemoryMB = Math.max(this.computeEvalMemoryMB, 
computeRequest.getMegaBytes());
         this.computeEvalCore = Math.max(this.computeEvalCore, 
computeRequest.getNumberOfCores());
@@ -163,14 +163,14 @@ public class DataLoader {
       }
     }
     // Deserialize each data requests.
-    // We distribute the partitions evenly accross the DCs.
+    // We distribute the partitions evenly across the DCs.
     // The number of partitions extracted from the dataLoadingService override
     // the number of evaluators requested (this preserves previous 
functionality)
     final int dcs = serializedDataRequests.size();
     final int partitionsPerDataCenter = 
this.dataLoadingService.getNumberOfPartitions() / dcs;
     int missing = this.dataLoadingService.getNumberOfPartitions() % dcs;
     for (final String serializedDataRequest : serializedDataRequests) {
-      EvaluatorRequest dataRequest = 
EvaluatorRequestSerializer.deserialize(serializedDataRequest);
+      EvaluatorRequest dataRequest = 
AvroEvaluatorRequestSerializer.fromString(serializedDataRequest);
       this.dataEvalMemoryMB = Math.max(this.dataEvalMemoryMB, 
dataRequest.getMegaBytes());
       this.dataEvalCore = Math.max(this.dataEvalCore, 
dataRequest.getNumberOfCores());
       // clone the request but update the number of evaluators based on the 
number of partitions

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6702ed58/lang/java/reef-io/src/main/java/org/apache/reef/io/data/loading/api/DataLoadingRequestBuilder.java
----------------------------------------------------------------------
diff --git 
a/lang/java/reef-io/src/main/java/org/apache/reef/io/data/loading/api/DataLoadingRequestBuilder.java
 
b/lang/java/reef-io/src/main/java/org/apache/reef/io/data/loading/api/DataLoadingRequestBuilder.java
index a54ecfd..79275a0 100644
--- 
a/lang/java/reef-io/src/main/java/org/apache/reef/io/data/loading/api/DataLoadingRequestBuilder.java
+++ 
b/lang/java/reef-io/src/main/java/org/apache/reef/io/data/loading/api/DataLoadingRequestBuilder.java
@@ -24,7 +24,7 @@ import org.apache.hadoop.mapred.TextInputFormat;
 import org.apache.reef.client.DriverConfiguration;
 import org.apache.reef.driver.evaluator.EvaluatorRequest;
 import 
org.apache.reef.io.data.loading.impl.DistributedDataSetPartitionSerializer;
-import org.apache.reef.io.data.loading.impl.EvaluatorRequestSerializer;
+import org.apache.reef.io.data.loading.impl.AvroEvaluatorRequestSerializer;
 import 
org.apache.reef.io.data.loading.impl.SingleDataCenterEvaluatorToPartitionStrategy;
 import org.apache.reef.io.data.loading.impl.DistributedDataSetPartition;
 import org.apache.reef.io.data.loading.impl.InputFormatLoadingService;
@@ -229,7 +229,7 @@ public final class DataLoadingRequestBuilder
   /**
    * Sets the distributed data set.
    *
-   * @param dataSet
+   * @param distributedDataSet
    *          the distributed data set
    * @return this
    */
@@ -294,13 +294,13 @@ public final class DataLoadingRequestBuilder
     // at this point data requests cannot be empty, either we use the one we 
created based on the
     // deprecated fields, or the ones created by the user
     for (final EvaluatorRequest request : this.dataRequests) {
-      jcb.bindSetEntry(DataLoadingDataRequests.class, 
EvaluatorRequestSerializer.serialize(request));
+      jcb.bindSetEntry(DataLoadingDataRequests.class, 
AvroEvaluatorRequestSerializer.toString(request));
     }
 
     // compute requests can be empty to maintain compatibility with previous 
code.
     if (!this.computeRequests.isEmpty()) {
       for (final EvaluatorRequest request : this.computeRequests) {
-        jcb.bindSetEntry(DataLoadingComputeRequests.class, 
EvaluatorRequestSerializer.serialize(request));
+        jcb.bindSetEntry(DataLoadingComputeRequests.class, 
AvroEvaluatorRequestSerializer.toString(request));
       }
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6702ed58/lang/java/reef-io/src/main/java/org/apache/reef/io/data/loading/impl/AvroEvaluatorRequestSerializer.java
----------------------------------------------------------------------
diff --git 
a/lang/java/reef-io/src/main/java/org/apache/reef/io/data/loading/impl/AvroEvaluatorRequestSerializer.java
 
b/lang/java/reef-io/src/main/java/org/apache/reef/io/data/loading/impl/AvroEvaluatorRequestSerializer.java
new file mode 100644
index 0000000..ae9ff19
--- /dev/null
+++ 
b/lang/java/reef-io/src/main/java/org/apache/reef/io/data/loading/impl/AvroEvaluatorRequestSerializer.java
@@ -0,0 +1,109 @@
+/*
+ * 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.reef.io.data.loading.impl;
+
+import org.apache.avro.io.*;
+import org.apache.avro.specific.SpecificDatumReader;
+import org.apache.avro.specific.SpecificDatumWriter;
+import org.apache.reef.annotations.audience.Private;
+import org.apache.reef.driver.evaluator.EvaluatorRequest;
+import org.apache.reef.io.data.loading.avro.AvroEvaluatorRequest;
+import org.apache.reef.webserver.AvroHttpSerializer;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Serialize and deserialize EvaluatorRequest objects using Avro.
+ */
+@Private
+public final class AvroEvaluatorRequestSerializer {
+  private static AvroEvaluatorRequest toAvro(final EvaluatorRequest request) {
+    final List<CharSequence> nodeNames = new ArrayList<>();
+    for (final String nodeName : request.getNodeNames()) {
+      nodeNames.add(nodeName);
+    }
+
+    final List<CharSequence> rackNames = new ArrayList<>();
+    for (final String rackName : request.getRackNames()) {
+      rackNames.add(rackName);
+    }
+
+    return AvroEvaluatorRequest.newBuilder()
+        .setCores(request.getNumberOfCores())
+        .setMegaBytes(request.getMegaBytes())
+        .setNumber(request.getNumber())
+        .setNodeNames(nodeNames)
+        .setRackNames(rackNames)
+        .build();
+  }
+
+  private static EvaluatorRequest fromAvro(final AvroEvaluatorRequest 
avroRequest) {
+    final EvaluatorRequest.Builder builder = EvaluatorRequest.newBuilder()
+        .setNumberOfCores(avroRequest.getCores())
+        .setMemory(avroRequest.getMegaBytes())
+        .setNumber(avroRequest.getNumber());
+    for (final CharSequence nodeName : avroRequest.getNodeNames()) {
+      builder.addNodeName(nodeName.toString());
+    }
+    for (final CharSequence rackName : avroRequest.getRackNames()) {
+      builder.addRackName(rackName.toString());
+    }
+    return builder.build();
+  }
+
+  /**
+   * Serialize EvaluatorRequest.
+   */
+  public static String toString(final EvaluatorRequest request) {
+    AvroEvaluatorRequest avroRequest = toAvro(request);
+    final DatumWriter<AvroEvaluatorRequest> datumWriter = new 
SpecificDatumWriter<>(AvroEvaluatorRequest.class);
+    try (final ByteArrayOutputStream out = new ByteArrayOutputStream()) {
+      final JsonEncoder encoder = 
EncoderFactory.get().jsonEncoder(avroRequest.getSchema(), out);
+      datumWriter.write(avroRequest, encoder);
+      encoder.flush();
+      out.close();
+      return out.toString(AvroHttpSerializer.JSON_CHARSET);
+    } catch (final IOException ex) {
+      throw new RuntimeException("Unable to serialize compute request", ex);
+    }
+  }
+
+  /**
+   * Deserialize EvaluatorRequest.
+   */
+  public static EvaluatorRequest fromString(final String serializedRequest) {
+    try {
+      final Decoder decoder =
+          
DecoderFactory.get().jsonDecoder(AvroEvaluatorRequest.getClassSchema(), 
serializedRequest);
+      final SpecificDatumReader<AvroEvaluatorRequest> reader = new 
SpecificDatumReader<>(AvroEvaluatorRequest.class);
+      return fromAvro(reader.read(null, decoder));
+    } catch (final IOException ex) {
+      throw new RuntimeException("Unable to deserialize compute request", ex);
+    }
+  }
+
+  /**
+   * Empty private constructor to prohibit instantiation of utility class.
+   */
+  private AvroEvaluatorRequestSerializer() {
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6702ed58/lang/java/reef-io/src/main/java/org/apache/reef/io/data/loading/impl/EvaluatorRequestSerializer.java
----------------------------------------------------------------------
diff --git 
a/lang/java/reef-io/src/main/java/org/apache/reef/io/data/loading/impl/EvaluatorRequestSerializer.java
 
b/lang/java/reef-io/src/main/java/org/apache/reef/io/data/loading/impl/EvaluatorRequestSerializer.java
deleted file mode 100644
index 0177d74..0000000
--- 
a/lang/java/reef-io/src/main/java/org/apache/reef/io/data/loading/impl/EvaluatorRequestSerializer.java
+++ /dev/null
@@ -1,87 +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.reef.io.data.loading.impl;
-
-import org.apache.commons.codec.binary.Base64;
-import org.apache.reef.driver.evaluator.EvaluatorRequest;
-
-import java.io.*;
-import java.util.List;
-
-/**
- * Serialize and deserialize EvaluatorRequest objects.
- * Supports number, memory, cores, nodeNames and rackNames serialization
- */
-public final class EvaluatorRequestSerializer {
-  public static String serialize(final EvaluatorRequest request) {
-    try (ByteArrayOutputStream baos = new ByteArrayOutputStream()) {
-      try (DataOutputStream daos = new DataOutputStream(baos)) {
-
-        daos.writeInt(request.getNumber());
-        daos.writeInt(request.getMegaBytes());
-        daos.writeInt(request.getNumberOfCores());
-        final List<String> nodeNames = request.getNodeNames();
-        final List<String> rackNames = request.getRackNames();
-        daos.writeInt(nodeNames.size());
-        daos.writeInt(rackNames.size());
-        for (final String nodeName : nodeNames) {
-          daos.writeUTF(nodeName);
-        }
-        for (final String rackName : rackNames) {
-          daos.writeUTF(rackName);
-        }
-
-      } catch (final IOException e) {
-        throw e;
-      }
-
-      return Base64.encodeBase64String(baos.toByteArray());
-    } catch (final IOException e1) {
-      throw new RuntimeException("Unable to serialize compute request", e1);
-    }
-  }
-
-  public static EvaluatorRequest deserialize(final String serializedRequest) {
-    try (ByteArrayInputStream bais = new 
ByteArrayInputStream(Base64.decodeBase64(serializedRequest))) {
-      try (DataInputStream dais = new DataInputStream(bais)) {
-        final EvaluatorRequest.Builder builder = EvaluatorRequest.newBuilder()
-            .setNumber(dais.readInt())
-            .setMemory(dais.readInt())
-            .setNumberOfCores(dais.readInt());
-        final int numNodes = dais.readInt();
-        final int numRacks = dais.readInt();
-        for (int i = 0; i < numNodes; i++) {
-          builder.addNodeName(dais.readUTF());
-        }
-        for (int i = 0; i < numRacks; i++) {
-          builder.addRackName(dais.readUTF());
-        }
-        return builder.build();
-      }
-    } catch (final IOException e) {
-      throw new RuntimeException("Unable to de-serialize compute request", e);
-    }
-  }
-
-  /**
-   * Empty private constructor to prohibit instantiation of utility class.
-   */
-  private EvaluatorRequestSerializer() {
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6702ed58/lang/java/reef-io/src/test/java/org/apache/reef/io/data/loading/impl/AvroEvaluatorRequestSerializerTest.java
----------------------------------------------------------------------
diff --git 
a/lang/java/reef-io/src/test/java/org/apache/reef/io/data/loading/impl/AvroEvaluatorRequestSerializerTest.java
 
b/lang/java/reef-io/src/test/java/org/apache/reef/io/data/loading/impl/AvroEvaluatorRequestSerializerTest.java
new file mode 100644
index 0000000..5fff5a6
--- /dev/null
+++ 
b/lang/java/reef-io/src/test/java/org/apache/reef/io/data/loading/impl/AvroEvaluatorRequestSerializerTest.java
@@ -0,0 +1,66 @@
+/*
+ * 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.reef.io.data.loading.impl;
+
+import org.apache.reef.driver.evaluator.EvaluatorRequest;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Test AvroEvaluatorRequestSerializer.
+ */
+public class AvroEvaluatorRequestSerializerTest {
+  @Test
+  public void testSerializeDeserializeCompleteRequest() {
+    final EvaluatorRequest originalRequest = EvaluatorRequest.newBuilder()
+        .setNumber(1)
+        .setMemory(10)
+        .setNumberOfCores(3)
+        .addRackName("rack1")
+        .addRackName("rack2")
+        .addNodeName("nodename1")
+        .addNodeName("nodename2")
+        .build();
+    final String serializedRequest = 
AvroEvaluatorRequestSerializer.toString(originalRequest);
+    final EvaluatorRequest deserializedRequest = 
AvroEvaluatorRequestSerializer.fromString(serializedRequest);
+
+    Assert.assertEquals(originalRequest.getMegaBytes(), 
deserializedRequest.getMegaBytes());
+    Assert.assertEquals(originalRequest.getNumber(), 
deserializedRequest.getNumber());
+    Assert.assertEquals(originalRequest.getNumberOfCores(), 
deserializedRequest.getNumberOfCores());
+    Assert.assertEquals(originalRequest.getRackNames(), 
deserializedRequest.getRackNames());
+    Assert.assertEquals(originalRequest.getNodeNames(), 
deserializedRequest.getNodeNames());
+  }
+
+  @Test
+  public void testSerializeDeserializeNoRacksNorNodesRequest() {
+    final EvaluatorRequest originalRequest = EvaluatorRequest.newBuilder()
+        .setNumber(1)
+        .setMemory(10)
+        .setNumberOfCores(3)
+        .build();
+    final String serialized = 
AvroEvaluatorRequestSerializer.toString(originalRequest);
+    final EvaluatorRequest deserializedRequest = 
AvroEvaluatorRequestSerializer.fromString(serialized);
+
+    Assert.assertEquals(originalRequest.getMegaBytes(), 
deserializedRequest.getMegaBytes());
+    Assert.assertEquals(originalRequest.getNumber(), 
deserializedRequest.getNumber());
+    Assert.assertEquals(originalRequest.getNumberOfCores(), 
deserializedRequest.getNumberOfCores());
+    Assert.assertTrue(deserializedRequest.getRackNames().size() == 0);
+    Assert.assertTrue(deserializedRequest.getNodeNames().size() == 0);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-reef/blob/6702ed58/lang/java/reef-io/src/test/java/org/apache/reef/io/data/loading/impl/EvaluatorRequestSerializerTest.java
----------------------------------------------------------------------
diff --git 
a/lang/java/reef-io/src/test/java/org/apache/reef/io/data/loading/impl/EvaluatorRequestSerializerTest.java
 
b/lang/java/reef-io/src/test/java/org/apache/reef/io/data/loading/impl/EvaluatorRequestSerializerTest.java
deleted file mode 100644
index 7852b2c..0000000
--- 
a/lang/java/reef-io/src/test/java/org/apache/reef/io/data/loading/impl/EvaluatorRequestSerializerTest.java
+++ /dev/null
@@ -1,67 +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.reef.io.data.loading.impl;
-
-import org.apache.reef.driver.evaluator.EvaluatorRequest;
-import org.junit.Assert;
-import org.junit.Test;
-
-/**
- * Test serialization and deserialization of EvaluatorRequest objects.
- */
-public final class EvaluatorRequestSerializerTest {
-
-  @Test
-  public void testSerializeDeserializeCompleteRequest() {
-    final EvaluatorRequest expected = EvaluatorRequest.newBuilder()
-                                            .setNumber(1)
-                                            .setMemory(10)
-                                            .setNumberOfCores(3)
-                                            .addRackName("rack1")
-                                            .addRackName("rack2")
-                                            .addNodeName("nodename1")
-                                            .addNodeName("nodename2")
-                                            .build();
-    final String serialized = EvaluatorRequestSerializer.serialize(expected);
-    final EvaluatorRequest actual = 
EvaluatorRequestSerializer.deserialize(serialized);
-
-    Assert.assertEquals(expected.getMegaBytes(), actual.getMegaBytes());
-    Assert.assertEquals(expected.getNumber(), actual.getNumber());
-    Assert.assertEquals(expected.getNumberOfCores(), 
actual.getNumberOfCores());
-    Assert.assertEquals(expected.getRackNames(), actual.getRackNames());
-    Assert.assertEquals(expected.getNodeNames(), actual.getNodeNames());
-  }
-
-  @Test
-  public void testSerializeDeserializeNoRacksNorNodesRequest() {
-    final EvaluatorRequest expected = EvaluatorRequest.newBuilder()
-                                            .setNumber(1)
-                                            .setMemory(10)
-                                            .setNumberOfCores(3)
-                                            .build();
-    final String serialized = EvaluatorRequestSerializer.serialize(expected);
-    final EvaluatorRequest actual = 
EvaluatorRequestSerializer.deserialize(serialized);
-
-    Assert.assertEquals(expected.getMegaBytes(), actual.getMegaBytes());
-    Assert.assertEquals(expected.getNumber(), actual.getNumber());
-    Assert.assertEquals(expected.getNumberOfCores(), 
actual.getNumberOfCores());
-    Assert.assertTrue(actual.getRackNames().size() == 0);
-    Assert.assertTrue(actual.getNodeNames().size() == 0);
-  }
-}

Reply via email to