[GitHub] [parquet-mr] theosib-amazon commented on a diff in pull request #960: Performance optimization: Move all LittleEndianDataInputStream functionality into ByteBufferInputStream

2022-11-21 Thread GitBox


theosib-amazon commented on code in PR #960:
URL: https://github.com/apache/parquet-mr/pull/960#discussion_r1021873458


##
parquet-common/src/main/java/org/apache/parquet/bytes/ByteBufferInputStream.java:
##
@@ -157,4 +165,80 @@ public void reset() throws IOException {
   public boolean markSupported() {
 return delegate.markSupported();
   }
+
+  public boolean readBoolean() throws IOException {
+return readByte() != 0;
+  }
+
+  public byte readByte() throws IOException {
+return delegate.readByte();
+  }
+
+  public int readUnsignedByte() throws IOException {
+return delegate.readUnsignedByte();
+  }
+
+  public short readShort() throws IOException {

Review Comment:
   I use the new batch read methods heavily in some optimizations I made to 
Trino. As for short, I can't say I recall any uses in Trino of readShorts(). 
readShort() is used indirectly through a method that reads a variable sized 
representation.



-- 
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: dev-unsubscr...@parquet.apache.org

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



[GitHub] [parquet-mr] theosib-amazon commented on a diff in pull request #960: Performance optimization: Move all LittleEndianDataInputStream functionality into ByteBufferInputStream

2022-11-14 Thread GitBox


theosib-amazon commented on code in PR #960:
URL: https://github.com/apache/parquet-mr/pull/960#discussion_r1021873458


##
parquet-common/src/main/java/org/apache/parquet/bytes/ByteBufferInputStream.java:
##
@@ -157,4 +165,80 @@ public void reset() throws IOException {
   public boolean markSupported() {
 return delegate.markSupported();
   }
+
+  public boolean readBoolean() throws IOException {
+return readByte() != 0;
+  }
+
+  public byte readByte() throws IOException {
+return delegate.readByte();
+  }
+
+  public int readUnsignedByte() throws IOException {
+return delegate.readUnsignedByte();
+  }
+
+  public short readShort() throws IOException {

Review Comment:
   I news the new batch read methods heavily in some optimizations I made to 
Trino. As for short, I can't say I recall any uses in Trino of readShorts(). 
readShort() is used indirectly through a method that reads a variable sized 
representation.



-- 
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: dev-unsubscr...@parquet.apache.org

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



[GitHub] [parquet-mr] theosib-amazon commented on a diff in pull request #960: Performance optimization: Move all LittleEndianDataInputStream functionality into ByteBufferInputStream

2022-11-14 Thread GitBox


theosib-amazon commented on code in PR #960:
URL: https://github.com/apache/parquet-mr/pull/960#discussion_r1021871752


##
parquet-common/src/main/java/org/apache/parquet/bytes/SingleBufferInputStream.java:
##
@@ -88,6 +136,21 @@ public long skip(long n) {
 return bytesToSkip;
   }
 
+  @Override
+  public void skipFully(long n) throws IOException {
+if (n < 0 || n > Integer.MAX_VALUE) {
+  throw new IllegalArgumentException();
+}
+
+try {
+  buffer.position(buffer.position() + (int)n);
+} catch (IllegalArgumentException e) {

Review Comment:
   I directly tested this, and it made a small but measurable difference.



-- 
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: dev-unsubscr...@parquet.apache.org

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



[GitHub] [parquet-mr] theosib-amazon commented on a diff in pull request #960: Performance optimization: Move all LittleEndianDataInputStream functionality into ByteBufferInputStream

2022-11-14 Thread GitBox


theosib-amazon commented on code in PR #960:
URL: https://github.com/apache/parquet-mr/pull/960#discussion_r1021870509


##
parquet-common/src/main/java/org/apache/parquet/bytes/MultiBufferInputStream.java:
##
@@ -238,8 +257,31 @@ public int read(byte[] bytes, int off, int len) {
   }
 
   @Override
-  public int read(byte[] bytes) {
-return read(bytes, 0, bytes.length);
+  public void readFully(byte[] bytes, int off, int len) throws IOException {
+if (len <= 0) {
+  if (len < 0) {
+throw new IndexOutOfBoundsException("Read length must be greater than 
0: " + len);
+  }
+  
+  return;
+}
+
+if (current == null || len > length) {
+  throw new EOFException();
+}
+
+int bytesRead = 0;
+while (bytesRead < len) {

Review Comment:
   I did test a lot of tradeoffs, but I don't think I tested this one thing 
directly. It's also been quite a while since I did this, so I don't think I'd 
be able to figure out which spreadsheets have the relevant data.



-- 
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: dev-unsubscr...@parquet.apache.org

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



[GitHub] [parquet-mr] theosib-amazon commented on a diff in pull request #960: Performance optimization: Move all LittleEndianDataInputStream functionality into ByteBufferInputStream

2022-09-23 Thread GitBox


theosib-amazon commented on code in PR #960:
URL: https://github.com/apache/parquet-mr/pull/960#discussion_r978947813


##
parquet-common/src/main/java/org/apache/parquet/bytes/ByteBufferInputStream.java:
##
@@ -138,6 +134,18 @@ public int read(byte[] b, int off, int len) throws 
IOException {
 return delegate.read(b, off, len);
   }
 
+  public int read(byte[] b) throws IOException {
+return read(b, 0, b.length);
+  }
+
+  public void readFully(byte b[]) throws IOException {
+readFully(b, 0, b.length);
+  }
+
+  public void readFully(byte b[], int off, int len) throws IOException {

Review Comment:
   Here's the complete preview of my changes to ParquetMR: 
https://github.com/theosib-amazon/parquet-mr/tree/batch-read-optimizations



-- 
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: dev-unsubscr...@parquet.apache.org

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



[GitHub] [parquet-mr] theosib-amazon commented on a diff in pull request #960: Performance optimization: Move all LittleEndianDataInputStream functionality into ByteBufferInputStream

2022-09-07 Thread GitBox


theosib-amazon commented on code in PR #960:
URL: https://github.com/apache/parquet-mr/pull/960#discussion_r965076867


##
parquet-common/src/main/java/org/apache/parquet/bytes/ByteBufferInputStream.java:
##
@@ -157,4 +165,80 @@ public void reset() throws IOException {
   public boolean markSupported() {
 return delegate.markSupported();
   }
+
+  public boolean readBoolean() throws IOException {
+return readByte() != 0;
+  }
+
+  public byte readByte() throws IOException {
+return delegate.readByte();
+  }
+
+  public int readUnsignedByte() throws IOException {
+return delegate.readUnsignedByte();
+  }
+
+  public short readShort() throws IOException {
+return delegate.readShort();
+  }
+
+  public int readUnsignedShort() throws IOException {
+return delegate.readUnsignedShort();
+  }
+
+  public int readInt() throws IOException {
+return delegate.readInt();
+  }
+
+  public long readLong() throws IOException {
+return delegate.readLong();
+  }
+
+  public float readFloat() throws IOException {
+return Float.intBitsToFloat(readInt());
+  }
+
+  public double readDouble() throws IOException {
+return Double.longBitsToDouble(readLong());
+  }
+
+  public int readIntLittleEndianOnThreeBytes() throws IOException {
+int ch1 = readUnsignedByte();
+int ch2 = readUnsignedByte();
+int ch3 = readUnsignedByte();
+return ((ch3 << 16) + (ch2 << 8) + (ch1 << 0));
+  }
+
+  public int readIntLittleEndianPaddedOnBitWidth(int bitWidth)

Review Comment:
   Wait. Are you referring to readIntLittleEndianPaddedOnBitWidth or 
readIntLittleEndianOnThreeBytes?
   
   The former is definitely faster. An argument could be made to remove the 
latter, although it'll take longer for the JIT to hide the extra layers of 
virtual calls.



-- 
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: dev-unsubscr...@parquet.apache.org

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



[GitHub] [parquet-mr] theosib-amazon commented on a diff in pull request #960: Performance optimization: Move all LittleEndianDataInputStream functionality into ByteBufferInputStream

2022-09-07 Thread GitBox


theosib-amazon commented on code in PR #960:
URL: https://github.com/apache/parquet-mr/pull/960#discussion_r965074182


##
parquet-common/src/main/java/org/apache/parquet/bytes/ByteBufferInputStream.java:
##
@@ -138,6 +134,18 @@ public int read(byte[] b, int off, int len) throws 
IOException {
 return delegate.read(b, off, len);
   }
 
+  public int read(byte[] b) throws IOException {
+return read(b, 0, b.length);
+  }
+
+  public void readFully(byte b[]) throws IOException {
+readFully(b, 0, b.length);
+  }
+
+  public void readFully(byte b[], int off, int len) throws IOException {

Review Comment:
   This is what the method signatures for DataInputStream.readFully look like.
   
   I also have a whole bunch of other performance improvements I want to 
contribute 
(https://docs.google.com/document/d/1fBGpF_LgtfaeHnPD5CFEIpA2Ga_lTITmFdFIcO9Af-g/edit?usp=sharing),
 and I think this might get used in some of that code.
   
   I'm very soon going to publish an open preview of all of my proposed changes 
to a branch of my own fork, so we'll be able to check this out.



-- 
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: dev-unsubscr...@parquet.apache.org

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



[GitHub] [parquet-mr] theosib-amazon commented on a diff in pull request #960: Performance optimization: Move all LittleEndianDataInputStream functionality into ByteBufferInputStream

2022-09-07 Thread GitBox


theosib-amazon commented on code in PR #960:
URL: https://github.com/apache/parquet-mr/pull/960#discussion_r965065058


##
parquet-common/src/main/java/org/apache/parquet/bytes/ByteBufferInputStream.java:
##
@@ -138,6 +134,18 @@ public int read(byte[] b, int off, int len) throws 
IOException {
 return delegate.read(b, off, len);
   }
 
+  public int read(byte[] b) throws IOException {
+return read(b, 0, b.length);
+  }
+
+  public void readFully(byte b[]) throws IOException {

Review Comment:
   This is almost certainly the result of a copy/paste of the method signature, 
and I just didn't notice. I went ahead and fixed it.



-- 
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: dev-unsubscr...@parquet.apache.org

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



[GitHub] [parquet-mr] theosib-amazon commented on a diff in pull request #960: Performance optimization: Move all LittleEndianDataInputStream functionality into ByteBufferInputStream

2022-09-07 Thread GitBox


theosib-amazon commented on code in PR #960:
URL: https://github.com/apache/parquet-mr/pull/960#discussion_r965063218


##
parquet-common/src/main/java/org/apache/parquet/bytes/MultiBufferInputStream.java:
##
@@ -379,4 +427,120 @@ public void remove() {
   second.remove();
 }
   }
+
+  @Override
+  public byte readByte() throws IOException {
+return (byte)readUnsignedByte();
+  }
+
+  @Override
+  public int readUnsignedByte() throws IOException {

Review Comment:
   Looks like I took care of this already.



-- 
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: dev-unsubscr...@parquet.apache.org

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



[GitHub] [parquet-mr] theosib-amazon commented on a diff in pull request #960: Performance optimization: Move all LittleEndianDataInputStream functionality into ByteBufferInputStream

2022-09-07 Thread GitBox


theosib-amazon commented on code in PR #960:
URL: https://github.com/apache/parquet-mr/pull/960#discussion_r965058610


##
parquet-common/src/main/java/org/apache/parquet/bytes/SingleBufferInputStream.java:
##
@@ -88,6 +136,21 @@ public long skip(long n) {
 return bytesToSkip;
   }
 
+  @Override
+  public void skipFully(long n) throws IOException {
+if (n < 0 || n > Integer.MAX_VALUE) {
+  throw new IllegalArgumentException();
+}
+
+try {
+  buffer.position(buffer.position() + (int)n);
+} catch (IllegalArgumentException e) {

Review Comment:
   A try/catch is basically free when no exception is thrown, while a check is 
not. I have tested this, and the try/catch is empirically faster, since the 
no-exception case is the common case. Putting in the check means we have to 
wait until the C2 compiler produces a trace without the branch. But even then, 
there's always the overhead of a check somewhere to be able to fall back to 
interpretation if the condition is not correct for the trace. I'm avoiding all 
of that entirely, making this faster in of the most common case.



-- 
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: dev-unsubscr...@parquet.apache.org

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



[GitHub] [parquet-mr] theosib-amazon commented on a diff in pull request #960: Performance optimization: Move all LittleEndianDataInputStream functionality into ByteBufferInputStream

2022-09-07 Thread GitBox


theosib-amazon commented on code in PR #960:
URL: https://github.com/apache/parquet-mr/pull/960#discussion_r965054827


##
parquet-common/src/main/java/org/apache/parquet/bytes/MultiBufferInputStream.java:
##
@@ -238,8 +257,31 @@ public int read(byte[] bytes, int off, int len) {
   }
 
   @Override
-  public int read(byte[] bytes) {
-return read(bytes, 0, bytes.length);
+  public void readFully(byte[] bytes, int off, int len) throws IOException {
+if (len <= 0) {
+  if (len < 0) {
+throw new IndexOutOfBoundsException("Read length must be greater than 
0: " + len);
+  }
+  
+  return;
+}
+
+if (current == null || len > length) {
+  throw new EOFException();
+}
+
+int bytesRead = 0;
+while (bytesRead < len) {

Review Comment:
   Well, my objective here is to maximize performance. So we have to decide 
between maintainability and performance. Let's deliberate over this a bit more, 
and I'll do what you think is best.



-- 
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: dev-unsubscr...@parquet.apache.org

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



[GitHub] [parquet-mr] theosib-amazon commented on a diff in pull request #960: Performance optimization: Move all LittleEndianDataInputStream functionality into ByteBufferInputStream

2022-09-07 Thread GitBox


theosib-amazon commented on code in PR #960:
URL: https://github.com/apache/parquet-mr/pull/960#discussion_r965053720


##
parquet-common/src/main/java/org/apache/parquet/bytes/MultiBufferInputStream.java:
##
@@ -238,8 +257,31 @@ public int read(byte[] bytes, int off, int len) {
   }
 
   @Override
-  public int read(byte[] bytes) {
-return read(bytes, 0, bytes.length);
+  public void readFully(byte[] bytes, int off, int len) throws IOException {

Review Comment:
   I'll make these changes if you insist. But those prechecks are expensive, 
which is why I'm trying to avoid them when possible in a performance critical 
path.



-- 
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: dev-unsubscr...@parquet.apache.org

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



[GitHub] [parquet-mr] theosib-amazon commented on a diff in pull request #960: Performance optimization: Move all LittleEndianDataInputStream functionality into ByteBufferInputStream

2022-09-07 Thread GitBox


theosib-amazon commented on code in PR #960:
URL: https://github.com/apache/parquet-mr/pull/960#discussion_r965052612


##
parquet-common/src/main/java/org/apache/parquet/bytes/ByteBufferInputStream.java:
##
@@ -157,4 +165,80 @@ public void reset() throws IOException {
   public boolean markSupported() {
 return delegate.markSupported();
   }
+
+  public boolean readBoolean() throws IOException {
+return readByte() != 0;
+  }
+
+  public byte readByte() throws IOException {
+return delegate.readByte();
+  }
+
+  public int readUnsignedByte() throws IOException {
+return delegate.readUnsignedByte();
+  }
+
+  public short readShort() throws IOException {
+return delegate.readShort();
+  }
+
+  public int readUnsignedShort() throws IOException {
+return delegate.readUnsignedShort();
+  }
+
+  public int readInt() throws IOException {
+return delegate.readInt();
+  }
+
+  public long readLong() throws IOException {
+return delegate.readLong();
+  }
+
+  public float readFloat() throws IOException {
+return Float.intBitsToFloat(readInt());
+  }
+
+  public double readDouble() throws IOException {
+return Double.longBitsToDouble(readLong());
+  }
+
+  public int readIntLittleEndianOnThreeBytes() throws IOException {

Review Comment:
   See my other comment on this. These two methods have the the same outcome, 
but mine is faster. I believe this is warranted for a performance critical path.



-- 
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: dev-unsubscr...@parquet.apache.org

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



[GitHub] [parquet-mr] theosib-amazon commented on a diff in pull request #960: Performance optimization: Move all LittleEndianDataInputStream functionality into ByteBufferInputStream

2022-09-07 Thread GitBox


theosib-amazon commented on code in PR #960:
URL: https://github.com/apache/parquet-mr/pull/960#discussion_r965051616


##
parquet-common/src/main/java/org/apache/parquet/bytes/ByteBufferInputStream.java:
##
@@ -157,4 +165,80 @@ public void reset() throws IOException {
   public boolean markSupported() {
 return delegate.markSupported();
   }
+
+  public boolean readBoolean() throws IOException {
+return readByte() != 0;
+  }
+
+  public byte readByte() throws IOException {
+return delegate.readByte();
+  }
+
+  public int readUnsignedByte() throws IOException {
+return delegate.readUnsignedByte();
+  }
+
+  public short readShort() throws IOException {
+return delegate.readShort();
+  }
+
+  public int readUnsignedShort() throws IOException {
+return delegate.readUnsignedShort();
+  }
+
+  public int readInt() throws IOException {
+return delegate.readInt();
+  }
+
+  public long readLong() throws IOException {
+return delegate.readLong();
+  }
+
+  public float readFloat() throws IOException {
+return Float.intBitsToFloat(readInt());
+  }
+
+  public double readDouble() throws IOException {
+return Double.longBitsToDouble(readLong());
+  }
+
+  public int readIntLittleEndianOnThreeBytes() throws IOException {
+int ch1 = readUnsignedByte();
+int ch2 = readUnsignedByte();
+int ch3 = readUnsignedByte();
+return ((ch3 << 16) + (ch2 << 8) + (ch1 << 0));
+  }
+
+  public int readIntLittleEndianPaddedOnBitWidth(int bitWidth)

Review Comment:
   I know about that method. The BytesUtils code always reads one byte at a 
time. My version will read a whole word at a time for short and int. This is 
faster.



-- 
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: dev-unsubscr...@parquet.apache.org

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



[GitHub] [parquet-mr] theosib-amazon commented on a diff in pull request #960: Performance optimization: Move all LittleEndianDataInputStream functionality into ByteBufferInputStream

2022-08-17 Thread GitBox


theosib-amazon commented on code in PR #960:
URL: https://github.com/apache/parquet-mr/pull/960#discussion_r948043918


##
parquet-common/src/main/java/org/apache/parquet/bytes/SingleBufferInputStream.java:
##
@@ -38,6 +39,34 @@ class SingleBufferInputStream extends ByteBufferInputStream {
 // duplicate the buffer because its state will be modified
 this.buffer = buffer.duplicate();
 this.startPosition = buffer.position();
+this.buffer.order(java.nio.ByteOrder.LITTLE_ENDIAN);
+  }
+
+  SingleBufferInputStream(ByteBuffer buffer, int start, int length) {
+// duplicate the buffer because its state will be modified
+this.buffer = buffer.duplicate();
+this.startPosition = start;
+this.buffer.position(start);
+this.buffer.limit(start + length);
+this.buffer.order(java.nio.ByteOrder.LITTLE_ENDIAN);
+  }
+
+  SingleBufferInputStream(byte[] inBuf) {

Review Comment:
   I went ahead and added two tests to cover the unused SingleBufferInputStream 
constructors. I considered just deleting these constructors, but I decided that 
it might be valuable to include them as documentation on how to do this in a 
way that is congruent to the behavior of HeapByteBuffer, just in case anyone 
wanted to do this in the future. There's also the risk that someone would think 
they have to wrap an array with ByteBuffer before using 
SingleBufferInputStream, but it would be better to avoid the overhead of 
ByteBuffer.duplicate(). (ByteBuffer.duplicate() appears to take constant time 
by making a reference to the same backing array, but it's a big constant with 
loads of checks.)



-- 
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: dev-unsubscr...@parquet.apache.org

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



[GitHub] [parquet-mr] theosib-amazon commented on a diff in pull request #960: Performance optimization: Move all LittleEndianDataInputStream functionality into ByteBufferInputStream

2022-08-01 Thread GitBox


theosib-amazon commented on code in PR #960:
URL: https://github.com/apache/parquet-mr/pull/960#discussion_r934673092


##
parquet-common/src/main/java/org/apache/parquet/bytes/SingleBufferInputStream.java:
##
@@ -88,6 +136,21 @@ public long skip(long n) {
 return bytesToSkip;
   }
 
+  @Override
+  public void skipFully(long n) throws IOException {
+if (n < 0 || n > Integer.MAX_VALUE) {
+  throw new IllegalArgumentException();
+}
+
+try {
+  buffer.position(buffer.position() + (int)n);
+} catch (IllegalArgumentException e) {

Review Comment:
   I already don't like the fact that I have to check the argument to make sure 
it's not negative and not bigger than int max. buffer.positiion() already 
checks for the position going out of bounds and throws an exception, so it 
would be redundant to have another check for the exact same thing here. A catch 
for an exception that never happens is basically always free, while a test for 
a condition that never happens is not free until profiling gets enough info 
about it for the C2 compiler to eliminate it.



-- 
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: dev-unsubscr...@parquet.apache.org

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



[GitHub] [parquet-mr] theosib-amazon commented on a diff in pull request #960: Performance optimization: Move all LittleEndianDataInputStream functionality into ByteBufferInputStream

2022-08-01 Thread GitBox


theosib-amazon commented on code in PR #960:
URL: https://github.com/apache/parquet-mr/pull/960#discussion_r934670047


##
parquet-common/src/main/java/org/apache/parquet/bytes/SingleBufferInputStream.java:
##
@@ -38,6 +39,34 @@ class SingleBufferInputStream extends ByteBufferInputStream {
 // duplicate the buffer because its state will be modified
 this.buffer = buffer.duplicate();
 this.startPosition = buffer.position();
+this.buffer.order(java.nio.ByteOrder.LITTLE_ENDIAN);
+  }
+
+  SingleBufferInputStream(ByteBuffer buffer, int start, int length) {
+// duplicate the buffer because its state will be modified
+this.buffer = buffer.duplicate();
+this.startPosition = start;
+this.buffer.position(start);
+this.buffer.limit(start + length);
+this.buffer.order(java.nio.ByteOrder.LITTLE_ENDIAN);
+  }
+
+  SingleBufferInputStream(byte[] inBuf) {

Review Comment:
   I removed this constructor because it's much better to have this be a 
compile-time error than a runtime error.



-- 
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: dev-unsubscr...@parquet.apache.org

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



[GitHub] [parquet-mr] theosib-amazon commented on a diff in pull request #960: Performance optimization: Move all LittleEndianDataInputStream functionality into ByteBufferInputStream

2022-08-01 Thread GitBox


theosib-amazon commented on code in PR #960:
URL: https://github.com/apache/parquet-mr/pull/960#discussion_r934669363


##
parquet-common/src/main/java/org/apache/parquet/bytes/MultiBufferInputStream.java:
##
@@ -379,4 +427,120 @@ public void remove() {
   second.remove();
 }
   }
+
+  @Override
+  public byte readByte() throws IOException {
+return (byte)readUnsignedByte();

Review Comment:
   got it



-- 
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: dev-unsubscr...@parquet.apache.org

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



[GitHub] [parquet-mr] theosib-amazon commented on a diff in pull request #960: Performance optimization: Move all LittleEndianDataInputStream functionality into ByteBufferInputStream

2022-08-01 Thread GitBox


theosib-amazon commented on code in PR #960:
URL: https://github.com/apache/parquet-mr/pull/960#discussion_r934668939


##
parquet-common/src/main/java/org/apache/parquet/bytes/MultiBufferInputStream.java:
##
@@ -89,6 +91,15 @@ public long skip(long n) {
 return bytesSkipped;
   }
 
+  @Override
+  public void skipFully(long n) throws IOException {
+if (current == null || n > length) {
+  throw new EOFException();

Review Comment:
   Got it



-- 
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: dev-unsubscr...@parquet.apache.org

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



[GitHub] [parquet-mr] theosib-amazon commented on a diff in pull request #960: Performance optimization: Move all LittleEndianDataInputStream functionality into ByteBufferInputStream

2022-08-01 Thread GitBox


theosib-amazon commented on code in PR #960:
URL: https://github.com/apache/parquet-mr/pull/960#discussion_r934649460


##
parquet-common/src/main/java/org/apache/parquet/bytes/SingleBufferInputStream.java:
##
@@ -174,4 +254,64 @@ public boolean markSupported() {
   public int available() {
 return buffer.remaining();
   }
+
+  @Override
+  public byte readByte() throws IOException {
+try {
+  return buffer.get();
+} catch (BufferUnderflowException e) {
+  throw new EOFException(e.getMessage());
+}
+  }
+
+  @Override
+  public int readUnsignedByte() throws IOException {
+try {
+  return buffer.get() & 0xFF;
+} catch (BufferUnderflowException e) {
+  throw new EOFException(e.getMessage());
+}
+  }
+
+  @Override
+  public short readShort() throws IOException {
+try {
+  return buffer.getShort();
+} catch (BufferUnderflowException e) {
+  throw new EOFException(e.getMessage());
+}
+  }
+
+  @Override
+  public int readUnsignedShort() throws IOException {
+try {
+  return buffer.getShort() & 0x;
+} catch (BufferUnderflowException e) {
+  throw new EOFException(e.getMessage());
+}
+  }
+
+  /*
+  Use ByteBuffer.getInt(), which takes advantage of platform intrinsics
+  */
+  @Override
+  public int readInt() throws IOException {
+try {
+  return buffer.getInt();
+} catch (BufferUnderflowException e) {
+  throw new EOFException(e.getMessage());
+}
+  }
+
+  /*
+  Use ByteBuffer.getLonmg(), which takes advantage of platform intrinsics

Review Comment:
   got it



-- 
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: dev-unsubscr...@parquet.apache.org

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



[GitHub] [parquet-mr] theosib-amazon commented on a diff in pull request #960: Performance optimization: Move all LittleEndianDataInputStream functionality into ByteBufferInputStream

2022-08-01 Thread GitBox


theosib-amazon commented on code in PR #960:
URL: https://github.com/apache/parquet-mr/pull/960#discussion_r934648806


##
parquet-common/src/main/java/org/apache/parquet/bytes/SingleBufferInputStream.java:
##
@@ -88,6 +136,21 @@ public long skip(long n) {
 return bytesToSkip;
   }
 
+  @Override
+  public void skipFully(long n) throws IOException {
+if (n < 0 || n > Integer.MAX_VALUE) {
+  throw new IllegalArgumentException();

Review Comment:
   got it



##
parquet-common/src/main/java/org/apache/parquet/bytes/SingleBufferInputStream.java:
##
@@ -70,9 +105,22 @@ public int read(byte[] bytes, int offset, int length) 
throws IOException {
 
 return bytesToRead;
   }
-  
+
+  @Override
+  public void readFully(byte[] bytes, int offset, int length) throws 
IOException {
+try {
+  buffer.get(bytes, offset, length);
+} catch (BufferUnderflowException|IndexOutOfBoundsException e) {
+  throw new EOFException(e.getMessage());
+}
+  }
+
   @Override
   public long skip(long n) {
+if (n < 0) {
+  throw new IllegalArgumentException();

Review Comment:
   got it



-- 
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: dev-unsubscr...@parquet.apache.org

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



[GitHub] [parquet-mr] theosib-amazon commented on a diff in pull request #960: Performance optimization: Move all LittleEndianDataInputStream functionality into ByteBufferInputStream

2022-08-01 Thread GitBox


theosib-amazon commented on code in PR #960:
URL: https://github.com/apache/parquet-mr/pull/960#discussion_r934644315


##
parquet-common/src/main/java/org/apache/parquet/bytes/SingleBufferInputStream.java:
##
@@ -70,9 +105,22 @@ public int read(byte[] bytes, int offset, int length) 
throws IOException {
 
 return bytesToRead;
   }
-  
+
+  @Override
+  public void readFully(byte[] bytes, int offset, int length) throws 
IOException {
+try {
+  buffer.get(bytes, offset, length);
+} catch (BufferUnderflowException|IndexOutOfBoundsException e) {

Review Comment:
   Got it.



-- 
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: dev-unsubscr...@parquet.apache.org

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



[GitHub] [parquet-mr] theosib-amazon commented on a diff in pull request #960: Performance optimization: Move all LittleEndianDataInputStream functionality into ByteBufferInputStream

2022-08-01 Thread GitBox


theosib-amazon commented on code in PR #960:
URL: https://github.com/apache/parquet-mr/pull/960#discussion_r934641756


##
parquet-common/src/main/java/org/apache/parquet/bytes/MultiBufferInputStream.java:
##
@@ -379,4 +427,120 @@ public void remove() {
   second.remove();
 }
   }
+
+  @Override
+  public byte readByte() throws IOException {
+return (byte)readUnsignedByte();
+  }
+
+  @Override
+  public int readUnsignedByte() throws IOException {

Review Comment:
   You're right. They're identical. I think the C1 compiler will eliminate the 
overhead of read() calling readUnsignedByte().



-- 
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: dev-unsubscr...@parquet.apache.org

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



[GitHub] [parquet-mr] theosib-amazon commented on a diff in pull request #960: Performance optimization: Move all LittleEndianDataInputStream functionality into ByteBufferInputStream

2022-08-01 Thread GitBox


theosib-amazon commented on code in PR #960:
URL: https://github.com/apache/parquet-mr/pull/960#discussion_r934629584


##
parquet-common/src/main/java/org/apache/parquet/bytes/MultiBufferInputStream.java:
##
@@ -238,8 +257,31 @@ public int read(byte[] bytes, int off, int len) {
   }
 
   @Override
-  public int read(byte[] bytes) {
-return read(bytes, 0, bytes.length);
+  public void readFully(byte[] bytes, int off, int len) throws IOException {

Review Comment:
   Normally, the user of the class would read exactly the right number of 
bytes. These checks and exceptions exist only to catch bugs elsewhere. This is 
one reason why it's important to minimize the overhead of these checks in such 
performance-critical methods.



-- 
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: dev-unsubscr...@parquet.apache.org

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



[GitHub] [parquet-mr] theosib-amazon commented on a diff in pull request #960: Performance optimization: Move all LittleEndianDataInputStream functionality into ByteBufferInputStream

2022-08-01 Thread GitBox


theosib-amazon commented on code in PR #960:
URL: https://github.com/apache/parquet-mr/pull/960#discussion_r934627078


##
parquet-common/src/main/java/org/apache/parquet/bytes/SingleBufferInputStream.java:
##
@@ -38,6 +39,34 @@ class SingleBufferInputStream extends ByteBufferInputStream {
 // duplicate the buffer because its state will be modified
 this.buffer = buffer.duplicate();
 this.startPosition = buffer.position();
+this.buffer.order(java.nio.ByteOrder.LITTLE_ENDIAN);
+  }
+
+  SingleBufferInputStream(ByteBuffer buffer, int start, int length) {
+// duplicate the buffer because its state will be modified
+this.buffer = buffer.duplicate();
+this.startPosition = start;
+this.buffer.position(start);
+this.buffer.limit(start + length);
+this.buffer.order(java.nio.ByteOrder.LITTLE_ENDIAN);
+  }
+
+  SingleBufferInputStream(byte[] inBuf) {
+this.buffer = ByteBuffer.wrap(inBuf);
+this.buffer.order(java.nio.ByteOrder.LITTLE_ENDIAN);
+this.startPosition = 0;
+  }
+
+  SingleBufferInputStream(byte[] inBuf, int start, int length) {
+this.buffer = ByteBuffer.wrap(inBuf, start, length);
+this.buffer.order(java.nio.ByteOrder.LITTLE_ENDIAN);
+// This seems to be consistent with HeapByteBuffer.wrap(), which leaves
+// the internal "offset" at zero and sets the starting position at start.
+this.startPosition = 0;
+  }
+
+  SingleBufferInputStream(List inBufs) {

Review Comment:
   IIRC, I used to have a similar constructor for ByteBufferInputStream, but I 
was advised to remove it, so I had this here for uniformity. We can remove this.



-- 
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: dev-unsubscr...@parquet.apache.org

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



[GitHub] [parquet-mr] theosib-amazon commented on a diff in pull request #960: Performance optimization: Move all LittleEndianDataInputStream functionality into ByteBufferInputStream

2022-08-01 Thread GitBox


theosib-amazon commented on code in PR #960:
URL: https://github.com/apache/parquet-mr/pull/960#discussion_r934623419


##
parquet-common/src/main/java/org/apache/parquet/bytes/SingleBufferInputStream.java:
##
@@ -38,6 +39,34 @@ class SingleBufferInputStream extends ByteBufferInputStream {
 // duplicate the buffer because its state will be modified
 this.buffer = buffer.duplicate();
 this.startPosition = buffer.position();
+this.buffer.order(java.nio.ByteOrder.LITTLE_ENDIAN);
+  }
+
+  SingleBufferInputStream(ByteBuffer buffer, int start, int length) {
+// duplicate the buffer because its state will be modified
+this.buffer = buffer.duplicate();
+this.startPosition = start;
+this.buffer.position(start);
+this.buffer.limit(start + length);
+this.buffer.order(java.nio.ByteOrder.LITTLE_ENDIAN);
+  }
+
+  SingleBufferInputStream(byte[] inBuf) {

Review Comment:
   Yeah. I put this here I think because I'm trying to make BufferInputStream 
have the same functionality as multiple other things that I'm combining into 
one. Some tests would be good. I'll see about writing some tests soon.



-- 
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: dev-unsubscr...@parquet.apache.org

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



[GitHub] [parquet-mr] theosib-amazon commented on a diff in pull request #960: Performance optimization: Move all LittleEndianDataInputStream functionality into ByteBufferInputStream

2022-08-01 Thread GitBox


theosib-amazon commented on code in PR #960:
URL: https://github.com/apache/parquet-mr/pull/960#discussion_r934619931


##
parquet-common/src/main/java/org/apache/parquet/bytes/SingleBufferInputStream.java:
##
@@ -88,6 +136,21 @@ public long skip(long n) {
 return bytesToSkip;
   }
 
+  @Override
+  public void skipFully(long n) throws IOException {
+if (n < 0 || n > Integer.MAX_VALUE) {
+  throw new IllegalArgumentException();
+}
+
+try {
+  buffer.position(buffer.position() + (int)n);
+} catch (IllegalArgumentException e) {

Review Comment:
   I did it this way on purpose. This way is always faster. Doing the check has 
to wait until there's profiling info and the C2 compiler gets hold of this code.



-- 
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: dev-unsubscr...@parquet.apache.org

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



[GitHub] [parquet-mr] theosib-amazon commented on a diff in pull request #960: Performance optimization: Move all LittleEndianDataInputStream functionality into ByteBufferInputStream

2022-07-25 Thread GitBox


theosib-amazon commented on code in PR #960:
URL: https://github.com/apache/parquet-mr/pull/960#discussion_r929020852


##
parquet-common/src/main/java/org/apache/parquet/bytes/MultiBufferInputStream.java:
##
@@ -379,4 +427,120 @@ public void remove() {
   second.remove();
 }
   }
+
+  @Override
+  public byte readByte() throws IOException {
+return (byte)readUnsignedByte();
+  }
+
+  @Override
+  public int readUnsignedByte() throws IOException {
+if (current == null) {
+  throw new EOFException();
+}
+
+this.position += 1;
+while (true) {
+  try {
+return current.get() & 0xFF;
+  } catch (BufferUnderflowException e) {
+if (!nextBuffer()) {
+  // there are no more buffers
+  throw new EOFException();
+}
+  }
+}
+  }
+
+  /**
+   * When reading a short will cross a buffer boundary, read one byte at a 
time.
+   * @return a short value
+   * @throws IOException
+   */
+  private int getShortSlow() throws IOException {
+int c0 = readUnsignedByte();
+int c1 = readUnsignedByte();
+return ((c0 << 0) + (c1 << 8));
+  }
+
+  public short readShort() throws IOException {
+if (current == null) {
+  throw new EOFException();
+}
+
+if (current.remaining() >= 2) {
+  // If the whole short can be read from the current buffer, use intrinsics
+  this.position += 2;
+  return current.getShort();
+} else {
+  // Otherwise get the short one byte at a time
+  return (short)getShortSlow();
+}
+  }
+
+  public int readUnsignedShort() throws IOException {
+return readShort() & 0x;
+  }
+
+  /**
+   * When reading an int will cross a buffer boundary, read one byte at a time.
+   * @return an int value
+   * @throws IOException
+   */
+  private int getIntSlow() throws IOException {
+int c0 = readUnsignedByte();
+int c1 = readUnsignedByte();
+int c2 = readUnsignedByte();
+int c3 = readUnsignedByte();
+return ((c0 << 0) + (c1 << 8)) + ((c2 << 16) + (c3 << 24));
+  }
+
+  @Override
+  public int readInt() throws IOException {
+if (current == null) {
+  throw new EOFException();
+}
+
+if (current.remaining() >= 4) {
+  // If the whole int can be read from the current buffer, use intrinsics
+  this.position += 4;
+  return current.getInt();
+} else {
+  // Otherwise get the int one byte at a time
+  return getIntSlow();
+}
+  }
+
+  /**
+   * When reading a long will cross a buffer boundary, read one byte at a time.
+   * @return a long value
+   * @throws IOException
+   */
+  private long getLongSlow() throws IOException {
+long ch0 = (long)readUnsignedByte() << 0;
+long ch1 = (long)readUnsignedByte() << 8;
+long ch2 = (long)readUnsignedByte() << 16;
+long ch3 = (long)readUnsignedByte() << 24;
+long ch4 = (long)readUnsignedByte() << 32;
+long ch5 = (long)readUnsignedByte() << 40;
+long ch6 = (long)readUnsignedByte() << 48;
+long ch7 = (long)readUnsignedByte() << 56;
+return ((ch0 + ch1) + (ch2 + ch3)) + ((ch4 + ch5) + (ch6 + ch7));
+  }
+
+  @Override
+  public long readLong() throws IOException {
+if (current == null) {
+  throw new EOFException();
+}
+
+if (current.remaining() >= 8) {
+  // If the whole short can be read from the current buffer, use intrinsics
+  this.position += 8;

Review Comment:
   Done.



-- 
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: dev-unsubscr...@parquet.apache.org

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



[GitHub] [parquet-mr] theosib-amazon commented on a diff in pull request #960: Performance optimization: Move all LittleEndianDataInputStream functionality into ByteBufferInputStream

2022-07-25 Thread GitBox


theosib-amazon commented on code in PR #960:
URL: https://github.com/apache/parquet-mr/pull/960#discussion_r929018213


##
parquet-common/src/main/java/org/apache/parquet/bytes/MultiBufferInputStream.java:
##
@@ -238,8 +257,31 @@ public int read(byte[] bytes, int off, int len) {
   }
 
   @Override
-  public int read(byte[] bytes) {
-return read(bytes, 0, bytes.length);
+  public void readFully(byte[] bytes, int off, int len) throws IOException {
+if (len <= 0) {
+  if (len < 0) {
+throw new IndexOutOfBoundsException("Read length must be greater than 
0: " + len);
+  }
+  
+  return;
+}
+
+if (current == null || len > length) {
+  throw new EOFException();
+}
+
+int bytesRead = 0;
+while (bytesRead < len) {

Review Comment:
   There are two key differences that make it hard to combine them without 
hurting performance for one, the other, or both, and they're both performance 
critical.



-- 
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: dev-unsubscr...@parquet.apache.org

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



[GitHub] [parquet-mr] theosib-amazon commented on a diff in pull request #960: Performance optimization: Move all LittleEndianDataInputStream functionality into ByteBufferInputStream

2022-07-25 Thread GitBox


theosib-amazon commented on code in PR #960:
URL: https://github.com/apache/parquet-mr/pull/960#discussion_r929011710


##
parquet-common/src/main/java/org/apache/parquet/bytes/MultiBufferInputStream.java:
##
@@ -238,8 +257,31 @@ public int read(byte[] bytes, int off, int len) {
   }
 
   @Override
-  public int read(byte[] bytes) {
-return read(bytes, 0, bytes.length);
+  public void readFully(byte[] bytes, int off, int len) throws IOException {

Review Comment:
   There are situations where we need to read an exact number of bytes and 
throw an exception if not enough are available. This is faster than reading 
maybe enough and then checking, and this is a performance critical path.



-- 
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: dev-unsubscr...@parquet.apache.org

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



[GitHub] [parquet-mr] theosib-amazon commented on a diff in pull request #960: Performance optimization: Move all LittleEndianDataInputStream functionality into ByteBufferInputStream

2022-07-25 Thread GitBox


theosib-amazon commented on code in PR #960:
URL: https://github.com/apache/parquet-mr/pull/960#discussion_r929010259


##
parquet-common/src/main/java/org/apache/parquet/bytes/ByteBufferInputStream.java:
##
@@ -157,4 +165,80 @@ public void reset() throws IOException {
   public boolean markSupported() {
 return delegate.markSupported();
   }
+
+  public boolean readBoolean() throws IOException {
+return readByte() != 0;
+  }
+
+  public byte readByte() throws IOException {
+return delegate.readByte();
+  }
+
+  public int readUnsignedByte() throws IOException {
+return delegate.readUnsignedByte();
+  }
+
+  public short readShort() throws IOException {
+return delegate.readShort();
+  }
+
+  public int readUnsignedShort() throws IOException {
+return delegate.readUnsignedShort();
+  }
+
+  public int readInt() throws IOException {
+return delegate.readInt();
+  }
+
+  public long readLong() throws IOException {
+return delegate.readLong();
+  }
+
+  public float readFloat() throws IOException {
+return Float.intBitsToFloat(readInt());
+  }
+
+  public double readDouble() throws IOException {
+return Double.longBitsToDouble(readLong());
+  }
+
+  public int readIntLittleEndianOnThreeBytes() throws IOException {

Review Comment:
   See my other comments.



-- 
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: dev-unsubscr...@parquet.apache.org

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



[GitHub] [parquet-mr] theosib-amazon commented on a diff in pull request #960: Performance optimization: Move all LittleEndianDataInputStream functionality into ByteBufferInputStream

2022-07-25 Thread GitBox


theosib-amazon commented on code in PR #960:
URL: https://github.com/apache/parquet-mr/pull/960#discussion_r929008981


##
parquet-common/src/main/java/org/apache/parquet/bytes/ByteBufferInputStream.java:
##
@@ -157,4 +165,80 @@ public void reset() throws IOException {
   public boolean markSupported() {
 return delegate.markSupported();
   }
+
+  public boolean readBoolean() throws IOException {
+return readByte() != 0;
+  }
+
+  public byte readByte() throws IOException {
+return delegate.readByte();
+  }
+
+  public int readUnsignedByte() throws IOException {
+return delegate.readUnsignedByte();
+  }
+
+  public short readShort() throws IOException {
+return delegate.readShort();
+  }
+
+  public int readUnsignedShort() throws IOException {
+return delegate.readUnsignedShort();
+  }
+
+  public int readInt() throws IOException {
+return delegate.readInt();
+  }
+
+  public long readLong() throws IOException {
+return delegate.readLong();
+  }
+
+  public float readFloat() throws IOException {
+return Float.intBitsToFloat(readInt());
+  }
+
+  public double readDouble() throws IOException {
+return Double.longBitsToDouble(readLong());
+  }
+
+  public int readIntLittleEndianOnThreeBytes() throws IOException {
+int ch1 = readUnsignedByte();
+int ch2 = readUnsignedByte();
+int ch3 = readUnsignedByte();
+return ((ch3 << 16) + (ch2 << 8) + (ch1 << 0));
+  }
+
+  public int readIntLittleEndianPaddedOnBitWidth(int bitWidth)

Review Comment:
   The one that reads three bytes may or may not be a win. A level of 
abstraction is eliminated by doing this. It's hard to say whether or not the 
JIT will be smart enough to do that automatically.



-- 
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: dev-unsubscr...@parquet.apache.org

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



[GitHub] [parquet-mr] theosib-amazon commented on a diff in pull request #960: Performance optimization: Move all LittleEndianDataInputStream functionality into ByteBufferInputStream

2022-07-25 Thread GitBox


theosib-amazon commented on code in PR #960:
URL: https://github.com/apache/parquet-mr/pull/960#discussion_r929007342


##
parquet-common/src/main/java/org/apache/parquet/bytes/ByteBufferInputStream.java:
##
@@ -157,4 +165,80 @@ public void reset() throws IOException {
   public boolean markSupported() {
 return delegate.markSupported();
   }
+
+  public boolean readBoolean() throws IOException {
+return readByte() != 0;
+  }
+
+  public byte readByte() throws IOException {
+return delegate.readByte();
+  }
+
+  public int readUnsignedByte() throws IOException {
+return delegate.readUnsignedByte();
+  }
+
+  public short readShort() throws IOException {
+return delegate.readShort();
+  }
+
+  public int readUnsignedShort() throws IOException {
+return delegate.readUnsignedShort();
+  }
+
+  public int readInt() throws IOException {
+return delegate.readInt();
+  }
+
+  public long readLong() throws IOException {
+return delegate.readLong();
+  }
+
+  public float readFloat() throws IOException {
+return Float.intBitsToFloat(readInt());
+  }
+
+  public double readDouble() throws IOException {
+return Double.longBitsToDouble(readLong());
+  }
+
+  public int readIntLittleEndianOnThreeBytes() throws IOException {
+int ch1 = readUnsignedByte();
+int ch2 = readUnsignedByte();
+int ch3 = readUnsignedByte();
+return ((ch3 << 16) + (ch2 << 8) + (ch1 << 0));
+  }
+
+  public int readIntLittleEndianPaddedOnBitWidth(int bitWidth)
+throws IOException {
+
+int bytesWidth = BytesUtils.paddedByteCountFromBits(bitWidth);
+switch (bytesWidth) {
+  case 0:
+return 0;
+  case 1:
+return readUnsignedByte();
+  case 2:
+return readUnsignedShort();
+  case 3:
+return readIntLittleEndianOnThreeBytes();
+  case 4:
+return readInt();
+  default:
+throw new IOException(
+  String.format("Encountered bitWidth (%d) that requires more than 4 
bytes", bitWidth));
+}
+  }
+
+  public int readUnsignedVarInt() throws IOException {

Review Comment:
   Not exactly. The one in BytesUtils calls methods that read one byte at a 
time. This one can take advantage of faster methods that read whole words at a 
time. This is a critical-path method, so it's a performance win to eliminate 
the extra level of abstraction and all the extra overhead fetching individual 
bytes and shifting.



-- 
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: dev-unsubscr...@parquet.apache.org

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



[GitHub] [parquet-mr] theosib-amazon commented on a diff in pull request #960: Performance optimization: Move all LittleEndianDataInputStream functionality into ByteBufferInputStream

2022-04-27 Thread GitBox


theosib-amazon commented on code in PR #960:
URL: https://github.com/apache/parquet-mr/pull/960#discussion_r860183442


##
parquet-common/src/main/java/org/apache/parquet/bytes/ByteBufferInputStream.java:
##
@@ -74,11 +97,26 @@ public ByteBufferInputStream(ByteBuffer buffer) {
*/
   @Deprecated
   public ByteBufferInputStream(ByteBuffer buffer, int offset, int count) {
+// This is necessary to pass "TestDeprecatedBufferInputStream"...
 ByteBuffer temp = buffer.duplicate();
 temp.position(offset);
 ByteBuffer byteBuf = temp.slice();
 byteBuf.limit(count);
 delegate = wrap(byteBuf);
+// ... but it would probably be faster to do this:
+//delegate = wrap(buffer, offset, count);

Review Comment:
   I have removed this at your request. The reason I put it there is that this 
is an inefficient way to do it. Future developers might want to know that and 
improve performance by doing it the other way and altering the test that 
unnecessarily forced this inefficient way of constructing.



-- 
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: dev-unsubscr...@parquet.apache.org

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



[GitHub] [parquet-mr] theosib-amazon commented on a diff in pull request #960: Performance optimization: Move all LittleEndianDataInputStream functionality into ByteBufferInputStream

2022-04-27 Thread GitBox


theosib-amazon commented on code in PR #960:
URL: https://github.com/apache/parquet-mr/pull/960#discussion_r860183308


##
parquet-common/src/main/java/org/apache/parquet/bytes/ByteBufferInputStream.java:
##
@@ -49,6 +59,19 @@ public static ByteBufferInputStream wrap(List 
buffers) {
 }
   }
 
+  public static ByteBufferInputStream wrap(ByteBuffer buffer, int offset, int 
count) {

Review Comment:
   This was to make ByteBufferInputStream resemble ByteBuffer. ByteBuffer has 
methods like this. I've gone ahead and removed them at your request.



##
parquet-common/src/main/java/org/apache/parquet/bytes/ByteBufferInputStream.java:
##
@@ -74,11 +97,26 @@ public ByteBufferInputStream(ByteBuffer buffer) {
*/
   @Deprecated
   public ByteBufferInputStream(ByteBuffer buffer, int offset, int count) {
+// This is necessary to pass "TestDeprecatedBufferInputStream"...
 ByteBuffer temp = buffer.duplicate();
 temp.position(offset);
 ByteBuffer byteBuf = temp.slice();
 byteBuf.limit(count);
 delegate = wrap(byteBuf);
+// ... but it would probably be faster to do this:
+//delegate = wrap(buffer, offset, count);

Review Comment:
   Done.



-- 
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: dev-unsubscr...@parquet.apache.org

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



[GitHub] [parquet-mr] theosib-amazon commented on a diff in pull request #960: Performance optimization: Move all LittleEndianDataInputStream functionality into ByteBufferInputStream

2022-04-27 Thread GitBox


theosib-amazon commented on code in PR #960:
URL: https://github.com/apache/parquet-mr/pull/960#discussion_r860180350


##
parquet-common/src/main/java/org/apache/parquet/bytes/ByteBufferInputStream.java:
##
@@ -28,6 +28,16 @@
 
 import org.apache.parquet.ShouldNeverHappenException;
 
+/*
+Changes implemented:

Review Comment:
   Removed



-- 
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: dev-unsubscr...@parquet.apache.org

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