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

2022-08-21 Thread GitBox


shangxinli commented on code in PR #960:
URL: https://github.com/apache/parquet-mr/pull/960#discussion_r950908609


##
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:
   the try/catch is also more expensive than checking. I agree with Chao to 
have the check instead of try/catch.



##
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:
   The try/catch is also more expensive than checking. I agree with Chao to 
have the check instead of try/catch.



-- 
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] shangxinli commented on a diff in pull request #960: Performance optimization: Move all LittleEndianDataInputStream functionality into ByteBufferInputStream

2022-08-21 Thread GitBox


shangxinli commented on code in PR #960:
URL: https://github.com/apache/parquet-mr/pull/960#discussion_r950908296


##
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:
   If they are identical, we should remove the duplicate one. 



-- 
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] shangxinli commented on a diff in pull request #960: Performance optimization: Move all LittleEndianDataInputStream functionality into ByteBufferInputStream

2022-08-21 Thread GitBox


shangxinli commented on code in PR #960:
URL: https://github.com/apache/parquet-mr/pull/960#discussion_r950908215


##
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:
   Duplicating code is hard to maintain. 



-- 
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] shangxinli commented on a diff in pull request #960: Performance optimization: Move all LittleEndianDataInputStream functionality into ByteBufferInputStream

2022-08-21 Thread GitBox


shangxinli commented on code in PR #960:
URL: https://github.com/apache/parquet-mr/pull/960#discussion_r950908127


##
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:
   Don't see where it is used. Don't know why it is '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: dev-unsubscr...@parquet.apache.org

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



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

2022-08-21 Thread GitBox


shangxinli commented on code in PR #960:
URL: https://github.com/apache/parquet-mr/pull/960#discussion_r950907839


##
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:
   The difference between this method and read() is mainly to precheck if there 
is enough remaining length. I believe this can be done by wrapping up the 
read() method and adding the prechecks. Duplicating the code makes it harder to 
maintain. 



-- 
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] shangxinli commented on a diff in pull request #960: Performance optimization: Move all LittleEndianDataInputStream functionality into ByteBufferInputStream

2022-08-21 Thread GitBox


shangxinli commented on code in PR #960:
URL: https://github.com/apache/parquet-mr/pull/960#discussion_r950906824


##
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:
   Why we need to be different between line 137 ' byte[] b' and 141 ' byte b[]'?



-- 
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] shangxinli commented on a diff in pull request #960: Performance optimization: Move all LittleEndianDataInputStream functionality into ByteBufferInputStream

2022-08-21 Thread GitBox


shangxinli commented on code in PR #960:
URL: https://github.com/apache/parquet-mr/pull/960#discussion_r950903406


##
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:
   Check this 
https://github.com/apache/parquet-mr/blob/master/parquet-common/src/main/java/org/apache/parquet/bytes/BytesUtils.java#L110



-- 
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] shangxinli commented on a diff in pull request #960: Performance optimization: Move all LittleEndianDataInputStream functionality into ByteBufferInputStream

2022-08-21 Thread GitBox


shangxinli commented on code in PR #960:
URL: https://github.com/apache/parquet-mr/pull/960#discussion_r950903342


##
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 meant the method here: 
https://github.com/apache/parquet-mr/blob/master/parquet-common/src/main/java/org/apache/parquet/bytes/BytesUtils.java#L120.
 It is exactly same but just different signature. 



-- 
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] shangxinli commented on a diff in pull request #960: Performance optimization: Move all LittleEndianDataInputStream functionality into ByteBufferInputStream

2022-07-24 Thread GitBox


shangxinli commented on code in PR #960:
URL: https://github.com/apache/parquet-mr/pull/960#discussion_r928315330


##
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:
   USE Long.BYTES instead of 8



-- 
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] shangxinli commented on a diff in pull request #960: Performance optimization: Move all LittleEndianDataInputStream functionality into ByteBufferInputStream

2022-07-24 Thread GitBox


shangxinli commented on code in PR #960:
URL: https://github.com/apache/parquet-mr/pull/960#discussion_r928315196


##
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:
   This seems duplicate with above line 244. 



-- 
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] shangxinli commented on a diff in pull request #960: Performance optimization: Move all LittleEndianDataInputStream functionality into ByteBufferInputStream

2022-07-24 Thread GitBox


shangxinli commented on code in PR #960:
URL: https://github.com/apache/parquet-mr/pull/960#discussion_r928314985


##
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:
   Can you cast a light why we need to add the implementation readFully() here? 
For performance improvement? 



-- 
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] shangxinli commented on a diff in pull request #960: Performance optimization: Move all LittleEndianDataInputStream functionality into ByteBufferInputStream

2022-07-24 Thread GitBox


shangxinli commented on code in PR #960:
URL: https://github.com/apache/parquet-mr/pull/960#discussion_r928313950


##
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:
   Is it copied from BytesUtils.java? I wonder why we don't use that directly?



-- 
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] shangxinli commented on a diff in pull request #960: Performance optimization: Move all LittleEndianDataInputStream functionality into ByteBufferInputStream

2022-07-24 Thread GitBox


shangxinli commented on code in PR #960:
URL: https://github.com/apache/parquet-mr/pull/960#discussion_r928314142


##
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:
   Is it copied from BytesUtils.java? I wonder why we don't use that directly?



##
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:
   Is it copied from BytesUtils.java? I wonder why we don't use that directly?



-- 
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] shangxinli commented on a diff in pull request #960: Performance optimization: Move all LittleEndianDataInputStream functionality into ByteBufferInputStream

2022-07-24 Thread GitBox


shangxinli commented on code in PR #960:
URL: https://github.com/apache/parquet-mr/pull/960#discussion_r928313950


##
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:
   Is it copied from ByteUnites? I wonder why we don't use that directly?



##
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:
   Is it copied from ByteUnites? I wonder why we don't use that directly?



-- 
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] shangxinli commented on a diff in pull request #960: Performance optimization: Move all LittleEndianDataInputStream functionality into ByteBufferInputStream

2022-07-24 Thread GitBox


shangxinli commented on code in PR #960:
URL: https://github.com/apache/parquet-mr/pull/960#discussion_r928313950


##
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:
   Is it copied from ByteUnites? I wonder why we don't use that directly?>



-- 
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