You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@beam.apache.org by da...@apache.org on 2016/06/20 22:16:19 UTC

[26/50] [abbrv] incubator-beam git commit: Touch up BigDecimalCoder and tests

Touch up BigDecimalCoder and tests


Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/6491100a
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/6491100a
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/6491100a

Branch: refs/heads/python-sdk
Commit: 6491100a5d655cb9f6c702767d6354269208f650
Parents: 8268f1d
Author: Kenneth Knowles <kl...@google.com>
Authored: Thu Jun 9 13:24:28 2016 -0700
Committer: Davor Bonaci <da...@google.com>
Committed: Mon Jun 20 15:14:30 2016 -0700

----------------------------------------------------------------------
 .../apache/beam/sdk/coders/BigDecimalCoder.java |  56 ++++------
 .../beam/sdk/coders/BigDecimalCoderTest.java    | 105 +++++++++++++------
 2 files changed, 95 insertions(+), 66 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6491100a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigDecimalCoder.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigDecimalCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigDecimalCoder.java
index 3b723b9..c4b7cd1 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigDecimalCoder.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigDecimalCoder.java
@@ -17,20 +17,22 @@
  */
 package org.apache.beam.sdk.coders;
 
+import static com.google.common.base.Preconditions.checkNotNull;
+
 import com.fasterxml.jackson.annotation.JsonCreator;
 
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
 import java.math.BigDecimal;
 import java.math.BigInteger;
+import java.math.MathContext;
 
 /**
- * A {@link BigDecimalCoder} encodes {@link BigDecimal} in an integer and
- * a byte array. The integer represents the scale and the byte array
- * represents a {@link BigInteger}. The integer is in 4 bytes, big-endian.
+ * A {@link BigDecimalCoder} encodes a {@link BigDecimal} as an integer scale encoded with
+ * {@link VarIntCoder} and a {@link BigInteger} encoded using {@link BigIntegerCoder}. The
+ * {@link BigInteger}, when scaled (with unlimited precision, aka {@link MathContext#UNLIMITED}),
+ * yields the expected {@link BigDecimal}.
  */
 public class BigDecimalCoder extends AtomicCoder<BigDecimal> {
 
@@ -43,37 +45,25 @@ public class BigDecimalCoder extends AtomicCoder<BigDecimal> {
 
   private static final BigDecimalCoder INSTANCE = new BigDecimalCoder();
 
+  private final VarIntCoder integerCoder = VarIntCoder.of();
+  private final BigIntegerCoder bigIntegerCoder = BigIntegerCoder.of();
+
   private BigDecimalCoder() {}
 
   @Override
   public void encode(BigDecimal value, OutputStream outStream, Context context)
       throws IOException, CoderException {
-    if (value == null) {
-      throw new CoderException("cannot encode a null BigDecimal");
-    }
-
-    byte[] bigIntBytes = value.unscaledValue().toByteArray();
-
-    DataOutputStream dataOutputStream = new DataOutputStream(outStream);
-    dataOutputStream.writeInt(value.scale());
-    dataOutputStream.writeInt(bigIntBytes.length);
-    dataOutputStream.write(bigIntBytes);
+    checkNotNull(value, String.format("cannot encode a null %s", BigDecimal.class.getSimpleName()));
+    integerCoder.encode(value.scale(), outStream, context.nested());
+    bigIntegerCoder.encode(value.unscaledValue(), outStream, context.nested());
   }
 
   @Override
   public BigDecimal decode(InputStream inStream, Context context)
       throws IOException, CoderException {
-    DataInputStream dataInputStream = new DataInputStream(inStream);
-    int scale = dataInputStream.readInt();
-    int bigIntBytesSize = dataInputStream.readInt();
-
-    byte[] bigIntBytes = new byte[bigIntBytesSize];
-    dataInputStream.readFully(bigIntBytes);
-
-    BigInteger bigInteger = new BigInteger(bigIntBytes);
-    BigDecimal bigDecimal = new BigDecimal(bigInteger, scale);
-
-    return bigDecimal;
+    int scale = integerCoder.decode(inStream, context.nested());
+    BigInteger bigInteger = bigIntegerCoder.decode(inStream, context.nested());
+    return new BigDecimal(bigInteger, scale);
   }
 
   /**
@@ -99,14 +89,14 @@ public class BigDecimalCoder extends AtomicCoder<BigDecimal> {
   /**
    * {@inheritDoc}
    *
-   * @return {@code 8} plus the size of the {@link BigInteger} bytes.
+   * @return {@code 4} (the size of an integer denoting the scale) plus {@code 4} (the size of an
+   * integer length prefix for the following bytes) plus the size of the two's-complement
+   * representation of the {@link BigInteger} that, when scaled, equals the given value.
    */
   @Override
-  protected long getEncodedElementByteSize(BigDecimal value, Context context)
-      throws Exception {
-    if (value == null) {
-      throw new CoderException("cannot encode a null BigDecimal");
-    }
-    return 8 + value.unscaledValue().toByteArray().length;
+  protected long getEncodedElementByteSize(BigDecimal value, Context context) throws Exception {
+    checkNotNull(value, String.format("cannot encode a null %s", BigDecimal.class.getSimpleName()));
+    return integerCoder.getEncodedElementByteSize(value.scale(), context.nested())
+        + bigIntegerCoder.getEncodedElementByteSize(value.unscaledValue(), context.nested());
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/6491100a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BigDecimalCoderTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BigDecimalCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BigDecimalCoderTest.java
index 5911535..033f076 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BigDecimalCoderTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BigDecimalCoderTest.java
@@ -17,8 +17,17 @@
  */
 package org.apache.beam.sdk.coders;
 
+import static org.hamcrest.Matchers.equalTo;
+import static org.junit.Assert.assertThat;
+
 import org.apache.beam.sdk.testing.CoderProperties;
 import org.apache.beam.sdk.util.CoderUtils;
+import org.apache.beam.sdk.util.common.Counter;
+import org.apache.beam.sdk.util.common.Counter.AggregationKind;
+import org.apache.beam.sdk.util.common.CounterName;
+import org.apache.beam.sdk.util.common.ElementByteSizeObserver;
+
+import com.google.common.collect.ImmutableList;
 
 import org.junit.Rule;
 import org.junit.Test;
@@ -27,7 +36,6 @@ import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
 
 import java.math.BigDecimal;
-import java.util.Arrays;
 import java.util.List;
 
 /**
@@ -36,19 +44,21 @@ import java.util.List;
 @RunWith(JUnit4.class)
 public class BigDecimalCoderTest {
 
+  @Rule public ExpectedException thrown = ExpectedException.none();
+
   private static final Coder<BigDecimal> TEST_CODER = BigDecimalCoder.of();
 
-  private static final List<BigDecimal> TEST_VALUES = Arrays.asList(
-      new BigDecimal(Double.MIN_VALUE),
-      new BigDecimal(-11),
-      new BigDecimal(-3),
-      new BigDecimal(-1),
-      new BigDecimal(0),
-      new BigDecimal(1),
-      new BigDecimal(5),
-      new BigDecimal(13),
-      new BigDecimal(29),
-      new BigDecimal(Double.MAX_VALUE));
+  private static final List<BigDecimal> TEST_VALUES =
+      ImmutableList.of(
+          new BigDecimal(Double.MIN_VALUE).divide(BigDecimal.TEN),
+          new BigDecimal(Double.MIN_VALUE),
+          new BigDecimal(-10.5),
+          new BigDecimal(-1),
+          new BigDecimal(0),
+          new BigDecimal(1),
+          new BigDecimal(13.258),
+          new BigDecimal(Double.MAX_VALUE),
+          new BigDecimal(Double.MAX_VALUE).multiply(BigDecimal.TEN));
 
   @Test
   public void testDecodeEncodeEqual() throws Exception {
@@ -69,37 +79,66 @@ public class BigDecimalCoderTest {
    * Generated data to check that the wire format has not changed. To regenerate, see
    * {@link org.apache.beam.sdk.coders.PrintBase64Encodings}.
    */
-  private static final List<String> TEST_ENCODINGS = Arrays.asList(
-      "AAAEMgAAATg12KOw51bHBNnjNkPn-wPiaWQ_AsohTe-mXyOGWcybUGt9TKi2FHqY2OH-gV0_GWqRbjNAGsSskI7K3" +
-          "xf9JmTjf1ySZXuvF9S9PsgV3kT-sgypaRw_i1MK_orzcJVg_s3cEGTjTY1_Xor3JM9UBVKiQy3Vpulf7aN9LM" +
-          "kiQEfO28mXQibyUtXL4yoLIwujoo8ArC9SayfbH5HmUxX9G0e506_cefoYIGByfq3M8GLp1_METj97ViU38je" +
-          "xsXkggqxXrMG8PO6pCYNB8P_jcf9i5OagpPafem18giZ8-v3fWJPN63vkbuOtaHb9u9yGQfrN25aLpNW9ooU9" +
-          "eYbL-1ewSBwENptcIT5SMhkulcVY6e9LyAqamGWdvnbevpwW84rTQpkeJePOkIt6G1_slfkQn6VBw7Jz3Vk",
-      "AAAAAAAAAAH1",
-      "AAAAAAAAAAH9",
-      "AAAAAAAAAAH_",
-      "AAAAAAAAAAEA",
-      "AAAAAAAAAAEB",
-      "AAAAAAAAAAEF",
-      "AAAAAAAAAAEN",
-      "AAAAAAAAAAEd",
-      "AAAAAAAAAIEA________-AAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAA" +
-          "AAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAA" +
-          "AAAAAAAAA");
+  private static final List<String> TEST_ENCODINGS =
+      ImmutableList.of(
+          "swi4AjXYo7DnVscE2eM2Q-f7A-JpZD8CyiFN76ZfI4ZZzJtQa31MqLYUepjY4f6BXT8ZapFuM"
+              + "0AaxKyQjsrfF_0mZON_XJJle68X1L0-yBXeRP6yDKlpHD-LUwr-ivNwlWD-zdwQZONNjX9"
+              + "eivckz1QFUqJDLdWm6V_to30sySJAR87byZdCJvJS1cvjKgsjC6OijwCsL1JrJ9sfkeZTF"
+              + "f0bR7nTr9x5-hggYHJ-rczwYunX8wROP3tWJTfyN7GxeSCCrFeswbw87qkJg0Hw_-Nx_2L"
+              + "k5qCk9p96bXyCJnz6_d9Yk83re-Ru461odv273IZB-s3blouk1b2ihT15hsv7V7BIHAQ2m"
+              + "1whPlIyGS6VxVjp70vICpqYZZ2-dt6-nBbzitNCmR4l486Qi3obX-yV-RCfpUHDsnPdWQ",
+          "sgi4AjXYo7DnVscE2eM2Q-f7A-JpZD8CyiFN76ZfI4ZZzJtQa31MqLYUepjY4f6BXT8ZapFu"
+              + "M0AaxKyQjsrfF_0mZON_XJJle68X1L0-yBXeRP6yDKlpHD-LUwr-ivNwlWD-zdwQZONNj"
+              + "X9eivckz1QFUqJDLdWm6V_to30sySJAR87byZdCJvJS1cvjKgsjC6OijwCsL1JrJ9sfke"
+              + "ZTFf0bR7nTr9x5-hggYHJ-rczwYunX8wROP3tWJTfyN7GxeSCCrFeswbw87qkJg0Hw_-Nx"
+              + "_2Lk5qCk9p96bXyCJnz6_d9Yk83re-Ru461odv273IZB-s3blouk1b2ihT15hsv7V7BIHA"
+              + "Q2m1whPlIyGS6VxVjp70vICpqYZZ2-dt6-nBbzitNCmR4l486Qi3obX-yV-RCfpUHDsnPdWQ",
+          "AQGX",
+          "AAH_",
+          "AAEA",
+          "AAEB",
+          "MBUJEk1IAgE1H9Gsru39PDZgUqT1NnU",
+          "AIEBAP________gAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAA"
+              + "AAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAA"
+              + "AAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAA",
+          "AIEBCf_______7AAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAA"
+              + "AAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAA"
+              + "AAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAA");
 
   @Test
   public void testWireFormatEncode() throws Exception {
     CoderProperties.coderEncodesBase64(TEST_CODER, TEST_VALUES, TEST_ENCODINGS);
   }
 
-  @Rule
-  public ExpectedException thrown = ExpectedException.none();
+  @Test
+  public void testGetEncodedElementByteSize() throws Exception {
+    Counter<Long> counter = Counter.longs(CounterName.named("dummy"), AggregationKind.SUM);
+    ElementByteSizeObserver observer = new ElementByteSizeObserver(counter);
+    for (BigDecimal value : TEST_VALUES) {
+      TEST_CODER.registerByteSizeObserver(value, observer, Coder.Context.OUTER);
+      observer.advance();
+      assertThat(
+          counter.getAggregate(),
+          equalTo((long) CoderUtils.encodeToByteArray(TEST_CODER, value).length));
+      counter.resetToValue(0L);
+    }
+  }
 
   @Test
-  public void encodeNullThrowsCoderException() throws Exception {
-    thrown.expect(CoderException.class);
+  public void encodeNullThrowsException() throws Exception {
+    thrown.expect(NullPointerException.class);
     thrown.expectMessage("cannot encode a null BigDecimal");
 
     CoderUtils.encodeToBase64(TEST_CODER, null);
   }
+
+  /**
+   * This is a change-detector test. If this test fails, then the encoding id of
+   * {@link BigDecimalCoder} must change.
+   */
+  @Test
+  public void testCoderIdDependencies() {
+    assertThat(VarIntCoder.of().getEncodingId(), equalTo(""));
+    assertThat(BigIntegerCoder.of().getEncodingId(), equalTo(""));
+  }
 }