beam-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "ASF GitHub Bot (JIRA)" <j...@apache.org>
Subject [jira] [Work logged] (BEAM-3874) Switch AvroIO sink default codec to Snappy
Date Fri, 23 Mar 2018 19:09:00 GMT

     [ https://issues.apache.org/jira/browse/BEAM-3874?focusedWorklogId=83771&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-83771
]

ASF GitHub Bot logged work on BEAM-3874:
----------------------------------------

                Author: ASF GitHub Bot
            Created on: 23/Mar/18 19:08
            Start Date: 23/Mar/18 19:08
    Worklog Time Spent: 10m 
      Work Description: jkff closed pull request #4904: [BEAM-3874] Switched AvroIO default
codec to snappyCodec.
URL: https://github.com/apache/beam/pull/4904
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroIO.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroIO.java
index 39a400e9c9a..478a7159f1e 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroIO.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroIO.java
@@ -184,7 +184,7 @@
  * custom file naming policy.
  *
  * <p>By default, {@link AvroIO.Write} produces output files that are compressed using
the {@link
- * org.apache.avro.file.Codec CodecFactory.deflateCodec(6)}. This default can be changed
or
+ * org.apache.avro.file.Codec CodecFactory.snappyCodec()}. This default can be changed or
  * overridden using {@link AvroIO.Write#withCodec}.
  *
  * <h3>Writing specific or generic records</h3>
@@ -848,7 +848,7 @@ public CreateParseSourceFn(SerializableFunction<GenericRecord, T>
parseFn, Coder
   @AutoValue
   public abstract static class TypedWrite<UserT, DestinationT, OutputT>
       extends PTransform<PCollection<UserT>, WriteFilesResult<DestinationT>>
{
-    static final CodecFactory DEFAULT_CODEC = CodecFactory.deflateCodec(6);
+    static final CodecFactory DEFAULT_CODEC = CodecFactory.snappyCodec();
     static final SerializableAvroCodecFactory DEFAULT_SERIALIZABLE_CODEC =
         new SerializableAvroCodecFactory(DEFAULT_CODEC);
 
@@ -1409,7 +1409,7 @@ public void populateDisplayData(DisplayData.Builder builder) {
 
     /**
      * Specifies to use the given {@link CodecFactory} for each generated file. By default,
{@code
-     * CodecFactory.deflateCodec(6)}.
+     * CodecFactory.snappyCodec()}.
      */
     public Sink<ElementT> withCodec(CodecFactory codec) {
       return toBuilder().setCodec(new SerializableAvroCodecFactory(codec)).build();
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroIOTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroIOTest.java
index a1bbe61165d..6dda36cd59f 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroIOTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroIOTest.java
@@ -1055,7 +1055,7 @@ public void testDynamicDestinationsViaSinkWithNumShards() throws Exception
{
   @Test
   public void testWriteWithDefaultCodec() throws Exception {
     AvroIO.Write<String> write = AvroIO.write(String.class).to("/tmp/foo/baz");
-    assertEquals(CodecFactory.deflateCodec(6).toString(), write.inner.getCodec().toString());
+    assertEquals(CodecFactory.snappyCodec().toString(), write.inner.getCodec().toString());
   }
 
   @Test
@@ -1222,7 +1222,7 @@ public void testWriteDisplayData() {
             .withShardNameTemplate("-SS-of-NN-")
             .withSuffix("bar")
             .withNumShards(100)
-            .withCodec(CodecFactory.snappyCodec());
+            .withCodec(CodecFactory.deflateCodec(6));
 
     DisplayData displayData = DisplayData.from(write);
 
@@ -1237,6 +1237,6 @@ public void testWriteDisplayData() {
                 + ".AvroIOTest$\",\"fields\":[{\"name\":\"intField\",\"type\":\"int\"},"
                 + "{\"name\":\"stringField\",\"type\":\"string\"}]}"));
     assertThat(displayData, hasDisplayItem("numShards", 100));
-    assertThat(displayData, hasDisplayItem("codec", CodecFactory.snappyCodec().toString()));
+    assertThat(displayData, hasDisplayItem("codec", CodecFactory.deflateCodec(6).toString()));
   }
 }


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Issue Time Tracking
-------------------

    Worklog Id:     (was: 83771)
    Time Spent: 20m  (was: 10m)

> Switch AvroIO sink default codec to Snappy
> ------------------------------------------
>
>                 Key: BEAM-3874
>                 URL: https://issues.apache.org/jira/browse/BEAM-3874
>             Project: Beam
>          Issue Type: Improvement
>          Components: io-java-avro
>            Reporter: Marian Dvorsky
>            Assignee: Eugene Kirpichov
>            Priority: Minor
>          Time Spent: 20m
>  Remaining Estimate: 0h
>
> AvroIO currently uses CodecFactory.deflateCodec(6) as the default codec for writes.
> That compresses well, but is quite expensive.
> Snappy codec offers sparser, but much faster compression, and is typically a better CPU/storage
tradeoff except for very long lived files. 
> We should consider switching the default to Snappy.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Mime
View raw message