You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@aurora.apache.org by js...@apache.org on 2016/04/12 06:31:51 UTC

aurora git commit: Make `ServerSet.JSON_CODEC` robust to errors.

Repository: aurora
Updated Branches:
  refs/heads/master ccf23820d -> 140b6e7c9


Make `ServerSet.JSON_CODEC` robust to errors.

Previously bad or partial input (deserialize), and errors writing output
(serialize) could throw exceptions not matching the `Codec` contract.
Factor the codec into its own top-level class to allow testing the
serialization failure case and expand coverage to include bad inputs.

Bugs closed: AURORA-1468

Reviewed at https://reviews.apache.org/r/45992/


Project: http://git-wip-us.apache.org/repos/asf/aurora/repo
Commit: http://git-wip-us.apache.org/repos/asf/aurora/commit/140b6e7c
Tree: http://git-wip-us.apache.org/repos/asf/aurora/tree/140b6e7c
Diff: http://git-wip-us.apache.org/repos/asf/aurora/diff/140b6e7c

Branch: refs/heads/master
Commit: 140b6e7c993b93a67959c950a0d169f630f8af22
Parents: ccf2382
Author: John Sirois <js...@apache.org>
Authored: Mon Apr 11 22:31:49 2016 -0600
Committer: John Sirois <jo...@gmail.com>
Committed: Mon Apr 11 22:31:49 2016 -0600

----------------------------------------------------------------------
 build.gradle                                    |   2 +
 .../aurora/common/zookeeper/JsonCodec.java      | 139 +++++++++++++++++
 .../aurora/common/zookeeper/ServerSet.java      |  78 +---------
 .../aurora/common/zookeeper/JsonCodecTest.java  | 151 +++++++++++++++++++
 .../aurora/common/zookeeper/ServerSetTest.java  |  78 ----------
 5 files changed, 293 insertions(+), 155 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/aurora/blob/140b6e7c/build.gradle
----------------------------------------------------------------------
diff --git a/build.gradle b/build.gradle
index d981ab7..e4e42fd 100644
--- a/build.gradle
+++ b/build.gradle
@@ -178,6 +178,8 @@ project(':commons') {
     compile "junit:junit:${junitRev}"
 
     testCompile "junit:junit:${junitRev}"
+    testCompile "org.powermock:powermock-module-junit4:1.6.4"
+    testCompile "org.powermock:powermock-api-easymock:1.6.4"
   }
 }
 

http://git-wip-us.apache.org/repos/asf/aurora/blob/140b6e7c/commons/src/main/java/org/apache/aurora/common/zookeeper/JsonCodec.java
----------------------------------------------------------------------
diff --git a/commons/src/main/java/org/apache/aurora/common/zookeeper/JsonCodec.java b/commons/src/main/java/org/apache/aurora/common/zookeeper/JsonCodec.java
new file mode 100644
index 0000000..9d31608
--- /dev/null
+++ b/commons/src/main/java/org/apache/aurora/common/zookeeper/JsonCodec.java
@@ -0,0 +1,139 @@
+/**
+ * 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 org.apache.aurora.common.zookeeper;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.io.Writer;
+import java.nio.charset.Charset;
+import java.util.Map;
+
+import javax.annotation.Nullable;
+
+import com.google.common.base.Charsets;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Maps;
+import com.google.gson.Gson;
+import com.google.gson.JsonIOException;
+import com.google.gson.JsonParseException;
+
+import org.apache.aurora.common.io.Codec;
+import org.apache.aurora.common.thrift.Endpoint;
+import org.apache.aurora.common.thrift.ServiceInstance;
+import org.apache.aurora.common.thrift.Status;
+
+import static java.util.Objects.requireNonNull;
+
+class JsonCodec implements Codec<ServiceInstance> {
+
+  private static void assertRequiredField(String fieldName, Object fieldValue) {
+    if (fieldValue == null) {
+      throw new JsonParseException(String.format("Field %s is required", fieldName));
+    }
+  }
+
+  private static class EndpointSchema {
+    private final String host;
+    private final Integer port;
+
+    EndpointSchema(Endpoint endpoint) {
+      host = endpoint.getHost();
+      port = endpoint.getPort();
+    }
+
+    Endpoint asEndpoint() {
+      assertRequiredField("host", host);
+      assertRequiredField("port", port);
+
+      return new Endpoint(host, port);
+    }
+  }
+
+  private static class ServiceInstanceSchema {
+    private final EndpointSchema serviceEndpoint;
+    private final Map<String, EndpointSchema> additionalEndpoints;
+    private final Status status;
+    private final @Nullable Integer shard;
+
+    ServiceInstanceSchema(ServiceInstance instance) {
+      serviceEndpoint = new EndpointSchema(instance.getServiceEndpoint());
+      if (instance.isSetAdditionalEndpoints()) {
+        additionalEndpoints =
+            Maps.transformValues(instance.getAdditionalEndpoints(), EndpointSchema::new);
+      } else {
+        additionalEndpoints = ImmutableMap.of();
+      }
+      status  = instance.getStatus();
+      shard = instance.isSetShard() ? instance.getShard() : null;
+    }
+
+    ServiceInstance asServiceInstance() {
+      assertRequiredField("serviceEndpoint", serviceEndpoint);
+      assertRequiredField("status", status);
+
+      Map<String, EndpointSchema> extraEndpoints =
+          additionalEndpoints == null ? ImmutableMap.of() : additionalEndpoints;
+
+      ServiceInstance instance =
+          new ServiceInstance(
+              serviceEndpoint.asEndpoint(),
+              Maps.transformValues(extraEndpoints, EndpointSchema::asEndpoint),
+              status);
+      if (shard != null) {
+        instance.setShard(shard);
+      }
+      return instance;
+    }
+  }
+
+  private static final Charset ENCODING = Charsets.UTF_8;
+
+  private final Gson gson;
+
+  JsonCodec() {
+    this(new Gson());
+  }
+
+  JsonCodec(Gson gson) {
+    this.gson = requireNonNull(gson);
+  }
+
+  @Override
+  public void serialize(ServiceInstance instance, OutputStream sink) throws IOException {
+    Writer writer = new OutputStreamWriter(sink, ENCODING);
+    try {
+      gson.toJson(new ServiceInstanceSchema(instance), writer);
+    } catch (JsonIOException e) {
+      throw new IOException(String.format("Problem serializing %s to JSON", instance), e);
+    }
+    writer.flush();
+  }
+
+  @Override
+  public ServiceInstance deserialize(InputStream source) throws IOException {
+    InputStreamReader reader = new InputStreamReader(source, ENCODING);
+    try {
+      @Nullable ServiceInstanceSchema schema = gson.fromJson(reader, ServiceInstanceSchema.class);
+      if (schema == null) {
+        throw new IOException("JSON did not include a ServiceInstance object");
+      }
+      return schema.asServiceInstance();
+    } catch (JsonParseException e) {
+      throw new IOException("Problem parsing JSON ServiceInstance.", e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/aurora/blob/140b6e7c/commons/src/main/java/org/apache/aurora/common/zookeeper/ServerSet.java
----------------------------------------------------------------------
diff --git a/commons/src/main/java/org/apache/aurora/common/zookeeper/ServerSet.java b/commons/src/main/java/org/apache/aurora/common/zookeeper/ServerSet.java
index 2d978c1..aeea02d 100644
--- a/commons/src/main/java/org/apache/aurora/common/zookeeper/ServerSet.java
+++ b/commons/src/main/java/org/apache/aurora/common/zookeeper/ServerSet.java
@@ -13,26 +13,11 @@
  */
 package org.apache.aurora.common.zookeeper;
 
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.InputStreamReader;
-import java.io.OutputStream;
-import java.io.OutputStreamWriter;
-import java.io.Writer;
 import java.net.InetSocketAddress;
-import java.nio.charset.Charset;
 import java.util.Map;
 
-import javax.annotation.Nullable;
-
-import com.google.common.base.Charsets;
-import com.google.common.collect.Maps;
-import com.google.gson.Gson;
-
 import org.apache.aurora.common.io.Codec;
-import org.apache.aurora.common.thrift.Endpoint;
 import org.apache.aurora.common.thrift.ServiceInstance;
-import org.apache.aurora.common.thrift.Status;
 import org.apache.aurora.common.zookeeper.Group.JoinException;
 
 /**
@@ -49,68 +34,7 @@ public interface ServerSet {
    *
    * This is the default encoding for service instance data in ZooKeeper.
    */
-  Codec<ServiceInstance> JSON_CODEC = new Codec<ServiceInstance>() {
-    class EndpointSchema {
-      private final String host;
-      private final int port;
-
-      EndpointSchema(Endpoint endpoint) {
-        host = endpoint.getHost();
-        port = endpoint.getPort();
-      }
-
-      Endpoint asEndpoint() {
-        return new Endpoint(host, port);
-      }
-    }
-
-    class ServiceInstanceSchema {
-      private final EndpointSchema serviceEndpoint;
-      private final Map<String, EndpointSchema> additionalEndpoints;
-      private final Status status;
-      private final @Nullable Integer shard;
-
-      ServiceInstanceSchema(ServiceInstance instance) {
-        serviceEndpoint = new EndpointSchema(instance.getServiceEndpoint());
-        if (instance.getAdditionalEndpoints() != null) {
-          additionalEndpoints =
-              Maps.transformValues(instance.getAdditionalEndpoints(), EndpointSchema::new);
-        } else {
-          additionalEndpoints = Maps.newHashMap();
-        }
-        status  = instance.getStatus();
-        shard = instance.isSetShard() ? instance.getShard() : null;
-      }
-
-      ServiceInstance asServiceInstance() {
-        ServiceInstance instance =
-            new ServiceInstance(
-                serviceEndpoint.asEndpoint(),
-                Maps.transformValues(additionalEndpoints, EndpointSchema::asEndpoint),
-                status);
-        if (shard != null) {
-          instance.setShard(shard);
-        }
-        return instance;
-      }
-    }
-
-    private final Charset encoding = Charsets.UTF_8;
-    private final Gson gson = new Gson();
-
-    @Override
-    public void serialize(ServiceInstance instance, OutputStream sink) throws IOException {
-      Writer writer = new OutputStreamWriter(sink, encoding);
-      gson.toJson(new ServiceInstanceSchema(instance), writer);
-      writer.flush();
-    }
-
-    @Override
-    public ServiceInstance deserialize(InputStream source) throws IOException {
-      InputStreamReader reader = new InputStreamReader(source, encoding);
-      return gson.fromJson(reader, ServiceInstanceSchema.class).asServiceInstance();
-    }
-  };
+  Codec<ServiceInstance> JSON_CODEC = new JsonCodec();
 
   /**
    * Attempts to join a server set for this logical service group.

http://git-wip-us.apache.org/repos/asf/aurora/blob/140b6e7c/commons/src/test/java/org/apache/aurora/common/zookeeper/JsonCodecTest.java
----------------------------------------------------------------------
diff --git a/commons/src/test/java/org/apache/aurora/common/zookeeper/JsonCodecTest.java b/commons/src/test/java/org/apache/aurora/common/zookeeper/JsonCodecTest.java
new file mode 100644
index 0000000..2166123
--- /dev/null
+++ b/commons/src/test/java/org/apache/aurora/common/zookeeper/JsonCodecTest.java
@@ -0,0 +1,151 @@
+/**
+ * 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 org.apache.aurora.common.zookeeper;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+
+import com.google.common.base.Charsets;
+import com.google.common.collect.ImmutableMap;
+import com.google.gson.Gson;
+import com.google.gson.JsonIOException;
+
+import org.apache.aurora.common.io.Codec;
+import org.apache.aurora.common.thrift.Endpoint;
+import org.apache.aurora.common.thrift.ServiceInstance;
+import org.apache.aurora.common.thrift.Status;
+import org.easymock.EasyMock;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.powermock.api.easymock.PowerMock;
+import org.powermock.core.classloader.annotations.PrepareForTest;
+import org.powermock.modules.junit4.PowerMockRunner;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+@RunWith(PowerMockRunner.class)
+@PrepareForTest(Gson.class)
+public class JsonCodecTest {
+
+  private static final Codec<ServiceInstance> STANDARD_JSON_CODEC = new JsonCodec();
+
+  @Test
+  public void testJsonCodecRoundtrip() throws Exception {
+    Codec<ServiceInstance> codec = STANDARD_JSON_CODEC;
+    ServiceInstance instance1 = new ServiceInstance(
+        new Endpoint("foo", 1000),
+        ImmutableMap.of("http", new Endpoint("foo", 8080)),
+        Status.ALIVE)
+        .setShard(0);
+    byte[] data = ServerSets.serializeServiceInstance(instance1, codec);
+    assertTrue(ServerSets.deserializeServiceInstance(data, codec).getServiceEndpoint().isSetPort());
+    assertTrue(ServerSets.deserializeServiceInstance(data, codec).isSetShard());
+
+    ServiceInstance instance2 = new ServiceInstance(
+        new Endpoint("foo", 1000),
+        ImmutableMap.of("http-admin1", new Endpoint("foo", 8080)),
+        Status.ALIVE);
+    data = ServerSets.serializeServiceInstance(instance2, codec);
+    assertTrue(ServerSets.deserializeServiceInstance(data, codec).getServiceEndpoint().isSetPort());
+    assertFalse(ServerSets.deserializeServiceInstance(data, codec).isSetShard());
+
+    ServiceInstance instance3 = new ServiceInstance(
+        new Endpoint("foo", 1000),
+        ImmutableMap.<String, Endpoint>of(),
+        Status.ALIVE);
+    data = ServerSets.serializeServiceInstance(instance3, codec);
+    assertTrue(ServerSets.deserializeServiceInstance(data, codec).getServiceEndpoint().isSetPort());
+    assertFalse(ServerSets.deserializeServiceInstance(data, codec).isSetShard());
+  }
+
+  @Test
+  public void testJsonCompatibility() throws IOException {
+    ServiceInstance instance = new ServiceInstance(
+        new Endpoint("foo", 1000),
+        ImmutableMap.of("http", new Endpoint("foo", 8080)),
+        Status.ALIVE).setShard(42);
+
+    ByteArrayOutputStream results = new ByteArrayOutputStream();
+    STANDARD_JSON_CODEC.serialize(instance, results);
+    assertEquals(
+        "{\"serviceEndpoint\":{\"host\":\"foo\",\"port\":1000},"
+            + "\"additionalEndpoints\":{\"http\":{\"host\":\"foo\",\"port\":8080}},"
+            + "\"status\":\"ALIVE\","
+            + "\"shard\":42}",
+        results.toString());
+  }
+
+  @Test
+  public void testInvalidSerialize() {
+    // Gson is final so we need to call on PowerMock here.
+    Gson gson = PowerMock.createMock(Gson.class);
+    gson.toJson(EasyMock.isA(Object.class), EasyMock.isA(Appendable.class));
+    EasyMock.expectLastCall().andThrow(new JsonIOException("error"));
+    PowerMock.replay(gson);
+
+    ServiceInstance instance =
+        new ServiceInstance(new Endpoint("foo", 1000), ImmutableMap.of(), Status.ALIVE);
+
+    try {
+      new JsonCodec(gson).serialize(instance, new ByteArrayOutputStream());
+      fail();
+    } catch (IOException e) {
+      // Expected.
+    }
+
+    PowerMock.verify(gson);
+  }
+
+  @Test
+  public void testDeserializeMinimal() throws IOException {
+    String minimal = "{\"serviceEndpoint\":{\"host\":\"foo\",\"port\":1000},\"status\":\"ALIVE\"}";
+    ByteArrayInputStream source = new ByteArrayInputStream(minimal.getBytes(Charsets.UTF_8));
+    ServiceInstance actual = STANDARD_JSON_CODEC.deserialize(source);
+    ServiceInstance expected =
+        new ServiceInstance(new Endpoint("foo", 1000), ImmutableMap.of(), Status.ALIVE);
+    assertEquals(expected, actual);
+  }
+
+  @Test
+  public void testInvalidDeserialize() {
+    // Not JSON.
+    assertInvalidDeserialize(new byte[] {0xC, 0xA, 0xF, 0xE});
+
+    // No JSON object.
+    assertInvalidDeserialize("");
+    assertInvalidDeserialize("[]");
+
+    // Missing required fields.
+    assertInvalidDeserialize("{}");
+    assertInvalidDeserialize("{\"serviceEndpoint\":{\"host\":\"foo\",\"port\":1000}}");
+    assertInvalidDeserialize("{\"status\":\"ALIVE\"}");
+  }
+
+  private void assertInvalidDeserialize(String data) {
+    assertInvalidDeserialize(data.getBytes(Charsets.UTF_8));
+  }
+
+  private void assertInvalidDeserialize(byte[] data) {
+    try {
+      STANDARD_JSON_CODEC.deserialize(new ByteArrayInputStream(data));
+      fail();
+    } catch (IOException e) {
+      // Expected.
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/aurora/blob/140b6e7c/commons/src/test/java/org/apache/aurora/common/zookeeper/ServerSetTest.java
----------------------------------------------------------------------
diff --git a/commons/src/test/java/org/apache/aurora/common/zookeeper/ServerSetTest.java b/commons/src/test/java/org/apache/aurora/common/zookeeper/ServerSetTest.java
deleted file mode 100644
index b48c1f1..0000000
--- a/commons/src/test/java/org/apache/aurora/common/zookeeper/ServerSetTest.java
+++ /dev/null
@@ -1,78 +0,0 @@
-/**
- * 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 org.apache.aurora.common.zookeeper;
-
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-
-import com.google.common.collect.ImmutableMap;
-
-import org.apache.aurora.common.io.Codec;
-import org.apache.aurora.common.thrift.Endpoint;
-import org.apache.aurora.common.thrift.ServiceInstance;
-import org.apache.aurora.common.thrift.Status;
-import org.junit.Test;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-public class ServerSetTest {
-
-  @Test
-  public void testJsonCodecRoundtrip() throws Exception {
-    Codec<ServiceInstance> codec = ServerSet.JSON_CODEC;
-    ServiceInstance instance1 = new ServiceInstance(
-        new Endpoint("foo", 1000),
-        ImmutableMap.of("http", new Endpoint("foo", 8080)),
-        Status.ALIVE)
-        .setShard(0);
-    byte[] data = ServerSets.serializeServiceInstance(instance1, codec);
-    assertTrue(ServerSets.deserializeServiceInstance(data, codec).getServiceEndpoint().isSetPort());
-    assertTrue(ServerSets.deserializeServiceInstance(data, codec).isSetShard());
-
-    ServiceInstance instance2 = new ServiceInstance(
-        new Endpoint("foo", 1000),
-        ImmutableMap.of("http-admin1", new Endpoint("foo", 8080)),
-        Status.ALIVE);
-    data = ServerSets.serializeServiceInstance(instance2, codec);
-    assertTrue(ServerSets.deserializeServiceInstance(data, codec).getServiceEndpoint().isSetPort());
-    assertFalse(ServerSets.deserializeServiceInstance(data, codec).isSetShard());
-
-    ServiceInstance instance3 = new ServiceInstance(
-        new Endpoint("foo", 1000),
-        ImmutableMap.<String, Endpoint>of(),
-        Status.ALIVE);
-    data = ServerSets.serializeServiceInstance(instance3, codec);
-    assertTrue(ServerSets.deserializeServiceInstance(data, codec).getServiceEndpoint().isSetPort());
-    assertFalse(ServerSets.deserializeServiceInstance(data, codec).isSetShard());
-  }
-
-  @Test
-  public void testJsonCompatibility() throws IOException {
-    ServiceInstance instance = new ServiceInstance(
-        new Endpoint("foo", 1000),
-        ImmutableMap.of("http", new Endpoint("foo", 8080)),
-        Status.ALIVE).setShard(42);
-
-    ByteArrayOutputStream results = new ByteArrayOutputStream();
-    ServerSet.JSON_CODEC.serialize(instance, results);
-    assertEquals(
-        "{\"serviceEndpoint\":{\"host\":\"foo\",\"port\":1000},"
-            + "\"additionalEndpoints\":{\"http\":{\"host\":\"foo\",\"port\":8080}},"
-            + "\"status\":\"ALIVE\","
-            + "\"shard\":42}",
-        results.toString());
-  }
-}