You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@beam.apache.org by GitBox <gi...@apache.org> on 2022/03/19 15:09:00 UTC

[GitHub] [beam] steveniemitz opened a new pull request #17134: [BEAM-14134] Optimize memory allocations for various core coders

steveniemitz opened a new pull request #17134:
URL: https://github.com/apache/beam/pull/17134


   Many coders have significant overhead due to the usage of `DataInputStream`.  DataInputStream allocates a significant amount of internal buffers when instantiated, which adds unnecessary overhead for very simple operations like decoding a big-endian long.
   
   This changes most coders that use DataInputStream internally to use a more optimized big-endian decoder.  I actually benchmarked three different options here, the solution I arrived at was the best mix of performance and allocations.
   
   ```
   Benchmark                Mode  Cnt          Score         Error  Units
   readLongViaLocalBuffer  thrpt   10  204364633.343 ± 7412002.528  ops/s
   readLongViaTLBuffer     thrpt   10  108663164.381 ±  229471.991  ops/s
   readLongViaReadCalls    thrpt   10  160694853.195 ± 5272248.704  ops/s
   ```
   
   readLongViaLocalBuffer allocates an 8 byte buffer per call and reads it using a single read() call.
   readLongViaTLBuffer does the same, but uses a thread-local buffer rather than allocating a new one each call.
   readLongViaReadCalls simply calls read 8 times, storing the results in temporary variables.
   
   R: @lukecwik  maybe?  Not really sure who's the best to look at this.
   
   ------------------------
   
   Thank you for your contribution! Follow this checklist to help us incorporate your contribution quickly and easily:
   
    - [x] [**Choose reviewer(s)**](https://beam.apache.org/contribute/#make-your-change) and mention them in a comment (`R: @username`).
    - [x] Format the pull request title like `[BEAM-XXX] Fixes bug in ApproximateQuantiles`, where you replace `BEAM-XXX` with the appropriate JIRA issue, if applicable. This will automatically link the pull request to the issue.
    - [ ] Update `CHANGES.md` with noteworthy changes.
    - [x] If this contribution is large, please file an Apache [Individual Contributor License Agreement](https://www.apache.org/licenses/icla.pdf).
   
   See the [Contributor Guide](https://beam.apache.org/contribute) for more tips on [how to make review process smoother](https://beam.apache.org/contribute/#make-reviewers-job-easier).
   
   To check the build health, please visit [https://github.com/apache/beam/blob/master/.test-infra/BUILD_STATUS.md](https://github.com/apache/beam/blob/master/.test-infra/BUILD_STATUS.md)
   
   GitHub Actions Tests Status (on master branch)
   ------------------------------------------------------------------------------------------------
   [![Build python source distribution and wheels](https://github.com/apache/beam/workflows/Build%20python%20source%20distribution%20and%20wheels/badge.svg?branch=master&event=schedule)](https://github.com/apache/beam/actions?query=workflow%3A%22Build+python+source+distribution+and+wheels%22+branch%3Amaster+event%3Aschedule)
   [![Python tests](https://github.com/apache/beam/workflows/Python%20tests/badge.svg?branch=master&event=schedule)](https://github.com/apache/beam/actions?query=workflow%3A%22Python+Tests%22+branch%3Amaster+event%3Aschedule)
   [![Java tests](https://github.com/apache/beam/workflows/Java%20Tests/badge.svg?branch=master&event=schedule)](https://github.com/apache/beam/actions?query=workflow%3A%22Java+Tests%22+branch%3Amaster+event%3Aschedule)
   
   See [CI.md](https://github.com/apache/beam/blob/master/CI.md) for more information about GitHub Actions CI.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] steveniemitz commented on pull request #17134: [BEAM-14134] Optimize memory allocations for various core coders

Posted by GitBox <gi...@apache.org>.
steveniemitz commented on pull request #17134:
URL: https://github.com/apache/beam/pull/17134#issuecomment-1074457952


   oh also, any thoughts on using the guava Longs, Ints, Shorts.fromBytes methods here?  I wasn't sure what the stance on using the shaded guava generally in the core libraries was.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] lukecwik removed a comment on pull request #17134: [BEAM-14134] Optimize memory allocations for various core coders

Posted by GitBox <gi...@apache.org>.
lukecwik removed a comment on pull request #17134:
URL: https://github.com/apache/beam/pull/17134#issuecomment-1074447580


   I would suggest sticking with `read/writeLongViaLocalBuffer` since read/write calls can depend on many layers of I/O before hitting the lowest layer so that allows us to pushdown the number of bytes we want to read/write as close to the layer doing I/O. Benchmarking using `ByteArrayInput/OutputStream` will give very skewed results.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] lukecwik commented on pull request #17134: [BEAM-14134] Optimize memory allocations for various core coders

Posted by GitBox <gi...@apache.org>.
lukecwik commented on pull request #17134:
URL: https://github.com/apache/beam/pull/17134#issuecomment-1075895217


   Run Java PreCommit


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] steveniemitz commented on pull request #17134: [BEAM-14134] Optimize memory allocations for various core coders

Posted by GitBox <gi...@apache.org>.
steveniemitz commented on pull request #17134:
URL: https://github.com/apache/beam/pull/17134#issuecomment-1074498638


   > This is totally fine to use shaded guava internally. Just don't expose the types on the API surface of things that are public.
   
   Cool, updated the Long one to use it at least.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] lukecwik commented on a change in pull request #17134: [BEAM-14134] Optimize memory allocations for various core coders

Posted by GitBox <gi...@apache.org>.
lukecwik commented on a change in pull request #17134:
URL: https://github.com/apache/beam/pull/17134#discussion_r832369579



##########
File path: sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BitConverters.java
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.beam.sdk.coders;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.primitives.Longs;
+
+class BitConverters {
+  private BitConverters() {}
+
+  private static void readFully(InputStream in, byte[] b) throws IOException {
+    int len = b.length;
+    int n = 0;
+    while (n < len) {
+      int count = in.read(b, n, len - n);
+      if (count < 0) {
+        throw new EOFException();
+      }
+      n += count;
+    }
+  }

Review comment:
       ```suggestion
   
   import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.io.ByteStreams.readFully;
   
   import java.io.EOFException;
   import java.io.IOException;
   import java.io.InputStream;
   import java.io.OutputStream;
   import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.primitives.Longs;
   
   /** Convert between Java value types and specific encodings of those value types. */
   class BitConverters {
     private BitConverters() {}
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] lukecwik commented on pull request #17134: [BEAM-14134] Optimize memory allocations for various core coders

Posted by GitBox <gi...@apache.org>.
lukecwik commented on pull request #17134:
URL: https://github.com/apache/beam/pull/17134#issuecomment-1075921989


   Run Java PreCommit


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] steveniemitz commented on pull request #17134: [BEAM-14134] Optimize memory allocations for various core coders

Posted by GitBox <gi...@apache.org>.
steveniemitz commented on pull request #17134:
URL: https://github.com/apache/beam/pull/17134#issuecomment-1074537994


   Run Java_Examples_Dataflow PreCommit


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] lukecwik commented on pull request #17134: [BEAM-14134] Optimize memory allocations for various core coders

Posted by GitBox <gi...@apache.org>.
lukecwik commented on pull request #17134:
URL: https://github.com/apache/beam/pull/17134#issuecomment-1076423671


   Run Java PreCommit


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] steveniemitz commented on pull request #17134: [BEAM-14134] Optimize memory allocations for various core coders

Posted by GitBox <gi...@apache.org>.
steveniemitz commented on pull request #17134:
URL: https://github.com/apache/beam/pull/17134#issuecomment-1073067367


   Run Java PreCommit


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] lukecwik commented on a change in pull request #17134: [BEAM-14134] Optimize memory allocations for various core coders

Posted by GitBox <gi...@apache.org>.
lukecwik commented on a change in pull request #17134:
URL: https://github.com/apache/beam/pull/17134#discussion_r832366024



##########
File path: sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BitConverters.java
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.beam.sdk.coders;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.primitives.Longs;
+
+class BitConverters {
+  private BitConverters() {}
+
+  private static void readFully(InputStream in, byte[] b) throws IOException {

Review comment:
       You can replace this with guava ByteStreams#readFully(java.io.InputStream, byte[])




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] steveniemitz commented on pull request #17134: [BEAM-14134] Optimize memory allocations for various core coders

Posted by GitBox <gi...@apache.org>.
steveniemitz commented on pull request #17134:
URL: https://github.com/apache/beam/pull/17134#issuecomment-1074449346


   > I would suggest sticking with `read/writeLongViaLocalBuffer` since read/write calls can depend on many layers of I/O before 
   
   Maybe use a local byte[] for longs and read calls for everything else?  That seems consistent with what DataInputStream did as well.
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] lukecwik commented on pull request #17134: [BEAM-14134] Optimize memory allocations for various core coders

Posted by GitBox <gi...@apache.org>.
lukecwik commented on pull request #17134:
URL: https://github.com/apache/beam/pull/17134#issuecomment-1075904276


   Run Java PreCommit


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] lukecwik commented on pull request #17134: [BEAM-14134] Optimize memory allocations for various core coders

Posted by GitBox <gi...@apache.org>.
lukecwik commented on pull request #17134:
URL: https://github.com/apache/beam/pull/17134#issuecomment-1074447580


   I would suggest sticking with `read/writeLongViaLocalBuffer` since read/write calls can depend on many layers of I/O before hitting the lowest layer so that allows us to pushdown the number of bytes we want to read/write as close to the layer doing I/O. Benchmarking using `ByteArrayInput/OutputStream` will give very skewed results.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] lukecwik commented on pull request #17134: [BEAM-14134] Optimize memory allocations for various core coders

Posted by GitBox <gi...@apache.org>.
lukecwik commented on pull request #17134:
URL: https://github.com/apache/beam/pull/17134#issuecomment-1074452530


   > > I would suggest sticking with `read/writeLongViaLocalBuffer` since read/write calls can depend on many layers of I/O before
   > 
   > Maybe use a local byte[] for longs and read calls for everything else? That seems consistent with what DataInputStream did as well.
   
   It is difficult for me to say whether 4 reads will be cheaper than creating a byte array. I wish fixed length value types could go on the stack then this would be a no brainer.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] steveniemitz commented on pull request #17134: [BEAM-14134] Optimize memory allocations for various core coders

Posted by GitBox <gi...@apache.org>.
steveniemitz commented on pull request #17134:
URL: https://github.com/apache/beam/pull/17134#issuecomment-1075588410


   alright I've given up trying to get this precommit working today.  I'll give it another poke tomorrow.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] lukecwik edited a comment on pull request #17134: [BEAM-14134] Optimize memory allocations for various core coders

Posted by GitBox <gi...@apache.org>.
lukecwik edited a comment on pull request #17134:
URL: https://github.com/apache/beam/pull/17134#issuecomment-1074452530


   > > I would suggest sticking with `read/writeLongViaLocalBuffer` since read/write calls can depend on many layers of I/O before
   > 
   > Maybe use a local byte[] for longs and read calls for everything else? That seems consistent with what DataInputStream did as well.
   
   It is difficult for me to say whether 4 reads will be cheaper than creating a byte array. I wish fixed length value types could go on the stack then this would be a no brainer but it does look like a win over allocating the 100's of bytes for each `Data*Stream` object so I'll take your judgement call as to whether you want arrays or multiple reads.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] steveniemitz commented on pull request #17134: [BEAM-14134] Optimize memory allocations for various core coders

Posted by GitBox <gi...@apache.org>.
steveniemitz commented on pull request #17134:
URL: https://github.com/apache/beam/pull/17134#issuecomment-1074454281


   > I wish fixed length value types could go on the stack then this would be a no brainer 
   
   Time for the C# runner? :rofl: 
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] lukecwik commented on pull request #17134: [BEAM-14134] Optimize memory allocations for various core coders

Posted by GitBox <gi...@apache.org>.
lukecwik commented on pull request #17134:
URL: https://github.com/apache/beam/pull/17134#issuecomment-1075868687


   Run Java PreCommit


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] steveniemitz commented on pull request #17134: [BEAM-14134] Optimize memory allocations for various core coders

Posted by GitBox <gi...@apache.org>.
steveniemitz commented on pull request #17134:
URL: https://github.com/apache/beam/pull/17134#issuecomment-1075186202


   not sure what's going on with the precommit here, the failure seems unrelated in a metrics test.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] lukecwik commented on pull request #17134: [BEAM-14134] Optimize memory allocations for various core coders

Posted by GitBox <gi...@apache.org>.
lukecwik commented on pull request #17134:
URL: https://github.com/apache/beam/pull/17134#issuecomment-1074441009


   This is great. Taking a look now.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] lukecwik commented on pull request #17134: [BEAM-14134] Optimize memory allocations for various core coders

Posted by GitBox <gi...@apache.org>.
lukecwik commented on pull request #17134:
URL: https://github.com/apache/beam/pull/17134#issuecomment-1075349386


   Run Java PreCommit


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] steveniemitz commented on a change in pull request #17134: [BEAM-14134] Optimize memory allocations for various core coders

Posted by GitBox <gi...@apache.org>.
steveniemitz commented on a change in pull request #17134:
URL: https://github.com/apache/beam/pull/17134#discussion_r832370652



##########
File path: sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BitConverters.java
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.beam.sdk.coders;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.primitives.Longs;
+
+class BitConverters {
+  private BitConverters() {}
+
+  private static void readFully(InputStream in, byte[] b) throws IOException {

Review comment:
       oh neat, TIL that existed.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] lukecwik commented on pull request #17134: [BEAM-14134] Optimize memory allocations for various core coders

Posted by GitBox <gi...@apache.org>.
lukecwik commented on pull request #17134:
URL: https://github.com/apache/beam/pull/17134#issuecomment-1075901975


   Run Java PreCommit


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] lukecwik commented on a change in pull request #17134: [BEAM-14134] Optimize memory allocations for various core coders

Posted by GitBox <gi...@apache.org>.
lukecwik commented on a change in pull request #17134:
URL: https://github.com/apache/beam/pull/17134#discussion_r832370397



##########
File path: sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BitConverters.java
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.beam.sdk.coders;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.primitives.Longs;
+
+class BitConverters {
+  private BitConverters() {}
+
+  private static void readFully(InputStream in, byte[] b) throws IOException {
+    int len = b.length;
+    int n = 0;
+    while (n < len) {
+      int count = in.read(b, n, len - n);
+      if (count < 0) {
+        throw new EOFException();
+      }
+      n += count;
+    }
+  }

Review comment:
       ```suggestion
   
   import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.io.ByteStreams.readFully;
   
   import java.io.EOFException;
   import java.io.IOException;
   import java.io.InputStream;
   import java.io.OutputStream;
   import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.primitives.Longs;
   
   /** Convert between Java value types and specific encodings of those value types. */
   class BitConverters {
     private BitConverters() {}
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] steveniemitz commented on pull request #17134: [BEAM-14134] Optimize memory allocations for various core coders

Posted by GitBox <gi...@apache.org>.
steveniemitz commented on pull request #17134:
URL: https://github.com/apache/beam/pull/17134#issuecomment-1075430480


   Run Java PreCommit


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] lukecwik merged pull request #17134: [BEAM-14134] Optimize memory allocations for various core coders

Posted by GitBox <gi...@apache.org>.
lukecwik merged pull request #17134:
URL: https://github.com/apache/beam/pull/17134


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] lukecwik commented on pull request #17134: [BEAM-14134] Optimize memory allocations for various core coders

Posted by GitBox <gi...@apache.org>.
lukecwik commented on pull request #17134:
URL: https://github.com/apache/beam/pull/17134#issuecomment-1075790643


   Run Java PreCommit


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] lukecwik commented on pull request #17134: [BEAM-14134] Optimize memory allocations for various core coders

Posted by GitBox <gi...@apache.org>.
lukecwik commented on pull request #17134:
URL: https://github.com/apache/beam/pull/17134#issuecomment-1075599793


   > alright I've given up trying to get this precommit working today. I'll give it another poke tomorrow.
   
   I filed https://issues.apache.org/jira/browse/BEAM-14148 and started a rollback of the extremely flaky test in https://github.com/apache/beam/pull/17154


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] steveniemitz commented on pull request #17134: [BEAM-14134] Optimize memory allocations for various core coders

Posted by GitBox <gi...@apache.org>.
steveniemitz commented on pull request #17134:
URL: https://github.com/apache/beam/pull/17134#issuecomment-1075186352


   Run Java PreCommit
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] lukecwik commented on pull request #17134: [BEAM-14134] Optimize memory allocations for various core coders

Posted by GitBox <gi...@apache.org>.
lukecwik commented on pull request #17134:
URL: https://github.com/apache/beam/pull/17134#issuecomment-1074470245


   > oh also, any thoughts on using the guava Longs, Ints, Shorts.fromBytes methods here? I wasn't sure what the stance on using the shaded guava generally in the core libraries was.
   
   This is totally fine to use shaded guava internally. Just don't expose the types on the API surface of things that are public.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] steveniemitz commented on pull request #17134: [BEAM-14134] Optimize memory allocations for various core coders

Posted by GitBox <gi...@apache.org>.
steveniemitz commented on pull request #17134:
URL: https://github.com/apache/beam/pull/17134#issuecomment-1074692964


   Run Java PreCommit


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org