[jira] [Created] (HIVE-19406) HiveKVResultCache.setupOutput hangs when the file creation failed

2018-05-03 Thread John Doe (JIRA)
John Doe created HIVE-19406:
---

 Summary: HiveKVResultCache.setupOutput hangs when the file 
creation failed
 Key: HIVE-19406
 URL: https://issues.apache.org/jira/browse/HIVE-19406
 Project: Hive
  Issue Type: Bug
  Components: SQL
Affects Versions: 2.3.2
Reporter: John Doe


The while loop in the HiveKVResultCache.setupOutput function hangs endlessly 
when the file creation failed, causing the delete() returns false.

The file creation failure can be caused by different reasons, e.g., disk full.

Here is the code snippet.

 
{code:java}
  private void setupOutput() throws IOException {
if (parentFile == null) {
  while (true) {
parentFile = File.createTempFile("hive-resultcache", "");
if (parentFile.delete() && parentFile.mkdir()) {
  parentFile.deleteOnExit();
  break;
}
if (LOG.isDebugEnabled()) {
  LOG.debug("Retry creating tmp result-cache directory...");
}
  }
}
...
  }
{code}

A similar case is [HIVE-19391|https://issues.apache.org/jira/browse/HIVE-19391]




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


[jira] [Created] (HIVE-19395) OutStream.write hangs with misconfigured bufferSize

2018-05-02 Thread John Doe (JIRA)
John Doe created HIVE-19395:
---

 Summary: OutStream.write hangs with misconfigured bufferSize
 Key: HIVE-19395
 URL: https://issues.apache.org/jira/browse/HIVE-19395
 Project: Hive
  Issue Type: Bug
  Components: SQL
Affects Versions: 1.0.0
Reporter: John Doe


When the bufferSize is configured to be 0 in the class initialization, the 
while loop in OutStream.write function hangs endlessly.
This is because when the bufferSize is 0, current.remaining is 0, length will 
always > 0.
Here is the code snippet.
{code:java}
   OutStream(String name, int bufferSize, CompressionCodec codec, 
OutputReceiver receiver) throws IOException {
...
this.bufferSize = bufferSize; //bufferSize can be configured with 0
...
  }

private void getNewInputBuffer() throws IOException {
  ...
  current = ByteBuffer.allocate(bufferSize);
  ...
  }

  public void write(byte[] bytes, int offset, int length) throws IOException {
if (current == null) {
  getNewInputBuffer();
}
int remaining = Math.min(current.remaining(), length);
current.put(bytes, offset, remaining);
uncompressedBytes += remaining;
length -= remaining;
while (length != 0) {//length > 0
  spill();
  offset += remaining;
  remaining = Math.min(current.remaining(), length);//current.remaining() 
== 0
  current.put(bytes, offset, remaining);
  uncompressedBytes += remaining;
  length -= remaining;
}
  }
{code}
The similar case is 
[HDFS-13513|https://issues.apache.org/jira/browse/HDFS-13513], 
[HDFS-13514|https://issues.apache.org/jira/browse/HDFS-13514]



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


[jira] [Created] (HIVE-19393) NonSyncDataInputBuffer.skipBytes hangs when the file is corrupted

2018-05-02 Thread John Doe (JIRA)
John Doe created HIVE-19393:
---

 Summary: NonSyncDataInputBuffer.skipBytes hangs when the file is 
corrupted 
 Key: HIVE-19393
 URL: https://issues.apache.org/jira/browse/HIVE-19393
 Project: Hive
  Issue Type: Bug
  Components: SQL
Affects Versions: 1.0.0
Reporter: John Doe


When an InputStream is corrupted, the InputStream.skip can return 0, causing 
the while loop in NonSyncDataInputBuffer.skipBytes become infinite.

{code:java}
  public final int skipBytes(int count) throws IOException {
int skipped = 0;
long skip;
while (skipped < count && (skip = in.skip(count - skipped)) != 0) {
  skipped += skip;
}
if (skipped < 0) {
  throw new EOFException();
}
return skipped;
  }
{code}

Similar bugs are 
[Hadoop-8614|https://issues.apache.org/jira/browse/HADOOP-8614], 
[Yarn-2905|https://issues.apache.org/jira/browse/YARN-2905], 
[Yarn-163|https://issues.apache.org/jira/browse/YARN-163], 
[Mapreduce-6990|https://issues.apache.org/jira/browse/MAPREDUCE-6990]



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


[jira] [Created] (HIVE-19392) unsynchronized index causes DataInputByteBuffer$Buffer.read hangs

2018-05-02 Thread John Doe (JIRA)
John Doe created HIVE-19392:
---

 Summary: unsynchronized index causes 
DataInputByteBuffer$Buffer.read hangs
 Key: HIVE-19392
 URL: https://issues.apache.org/jira/browse/HIVE-19392
 Project: Hive
  Issue Type: Bug
  Components: SQL
Affects Versions: 1.0.0
Reporter: John Doe


In DataInputByteBuffer$Buffer class, the fields bidx and buffers, etc are 
unsynchronized when used in read() and reset() function. In certain 
circumstances, e.g., the reset() is invoked in a loop, the unsynchronized bidx 
and buffers can trigger a concurrency bug.
This bug also shows in 
[Hadoop-15429|https://issues.apache.org/jira/browse/HADOOP-15429].



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


[jira] [Created] (HIVE-19391) RowContainer.setupWriter hangs when the file creation failed

2018-05-02 Thread John Doe (JIRA)
John Doe created HIVE-19391:
---

 Summary: RowContainer.setupWriter hangs when the file creation 
failed
 Key: HIVE-19391
 URL: https://issues.apache.org/jira/browse/HIVE-19391
 Project: Hive
  Issue Type: Bug
  Components: SQL
Affects Versions: 1.0.0
Reporter: John Doe


The while loop in the RowContainer.setupWriter function hangs endlessly when 
the file creation failed, causing the delete() returns false.

The file creation failure can be caused by different reasons, e.g., disk full.

Here is the code snippet.

 
{code:java}
  protected void setupWriter() throws HiveException {
try {

  if ( tmpFile != null ) {
return;
  }

  String suffix = ".tmp";
  if (this.keyObject != null) {
suffix = "." + this.keyObject.toString() + suffix;
  }

  while (true) {
parentFile = File.createTempFile("hive-rowcontainer", ""); //file 
creation failed
boolean success = parentFile.delete() && parentFile.mkdir(); //delete 
returns false
if (success) {
  break;
}
LOG.debug("retry creating tmp row-container directory...");
  }
  ...
  }
{code}




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


[jira] [Created] (HIVE-18219) When InputStream is corrupted, the skip() returns -1, causing infinite loop

2017-12-04 Thread John Doe (JIRA)
John Doe created HIVE-18219:
---

 Summary: When InputStream is corrupted, the skip() returns -1, 
causing infinite loop
 Key: HIVE-18219
 URL: https://issues.apache.org/jira/browse/HIVE-18219
 Project: Hive
  Issue Type: Bug
Affects Versions: 2.3.2
Reporter: John Doe


Similar like 
[CASSANDRA-7330|https://issues.apache.org/jira/browse/CASSANDRA-7330], when 
InputStream is corrupted, skip() returns -1, causing the following loop be 
infinite.

{code:java}
  public final int skipBytes(int count) throws IOException {
int skipped = 0;
long skip;
while (skipped < count && (skip = in.skip(count - skipped)) != 0) {
  skipped += skip;
}
if (skipped < 0) {
  throw new EOFException();
}
return skipped;
  }
{code}




--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Created] (HIVE-18217) When Text is corrupted, populateMappings() hangs indefinitely

2017-12-04 Thread John Doe (JIRA)
John Doe created HIVE-18217:
---

 Summary: When Text is corrupted, populateMappings() hangs 
indefinitely
 Key: HIVE-18217
 URL: https://issues.apache.org/jira/browse/HIVE-18217
 Project: Hive
  Issue Type: Bug
Affects Versions: 2.3.2
Reporter: John Doe


Similar like [HIVE-18216|https://issues.apache.org/jira/browse/HIVE-18216],
when the Text is corrupted, the following loop become infinite.

{code:java}
  private void populateMappings(Text from, Text to) {
replacementMap.clear();
deletionSet.clear();

ByteBuffer fromBytes = ByteBuffer.wrap(from.getBytes(), 0, 
from.getLength());
ByteBuffer toBytes = ByteBuffer.wrap(to.getBytes(), 0, to.getLength());

// Traverse through the from string, one code point at a time
while (fromBytes.hasRemaining()) {
  // This will also move the iterator ahead by one code point
  int fromCodePoint = Text.bytesToCodePoint(fromBytes);
  // If the to string has more code points, make sure to traverse it too
  if (toBytes.hasRemaining()) {
int toCodePoint = Text.bytesToCodePoint(toBytes);
// If the code point from from string already has a replacement or is 
to be deleted, we
// don't need to do anything, just move on to the next code point
if (replacementMap.containsKey(fromCodePoint) || 
deletionSet.contains(fromCodePoint)) {
  continue;
}
replacementMap.put(fromCodePoint, toCodePoint);
  } else {
// If the code point from from string already has a replacement or is 
to be deleted, we
// don't need to do anything, just move on to the next code point
if (replacementMap.containsKey(fromCodePoint) || 
deletionSet.contains(fromCodePoint)) {
  continue;
}
deletionSet.add(fromCodePoint);
  }
}
  }
{code}




--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Created] (HIVE-18216) When Text is corrupted, processInput() hangs indefinitely

2017-12-04 Thread John Doe (JIRA)
John Doe created HIVE-18216:
---

 Summary: When Text is corrupted, processInput() hangs indefinitely
 Key: HIVE-18216
 URL: https://issues.apache.org/jira/browse/HIVE-18216
 Project: Hive
  Issue Type: Bug
Affects Versions: 2.3.2
Reporter: John Doe


When the Text is corrupted, the following loop become infinite.
This is because in hadoop.io.Text.bytesToCodePoint(), when extraBytesToRead == 
-1, the index in the ByteBuffer is not moved, and thus, ByteBuffer.remaining() 
is always > 0.
And it deletionSet.contains(-1), then this loop become infinite.

{code:java}
  private String processInput(Text input) {
StringBuilder resultBuilder = new StringBuilder();
// Obtain the byte buffer from the input string so we can traverse it code 
point by code point
ByteBuffer inputBytes = ByteBuffer.wrap(input.getBytes(), 0, 
input.getLength());
// Traverse the byte buffer containing the input string one code point at a 
time
while (inputBytes.hasRemaining()) {
  int inputCodePoint = Text.bytesToCodePoint(inputBytes);
  // If the code point exists in deletion set, no need to emit out anything 
for this code point.
  // Continue on to the next code point
  if (deletionSet.contains(inputCodePoint)) {
continue;
  }

  Integer replacementCodePoint = replacementMap.get(inputCodePoint);
  // If a replacement exists for this code point, emit out the replacement 
and append it to the
  // output string. If no such replacement exists, emit out the original 
input code point
  char[] charArray = Character.toChars((replacementCodePoint != null) ? 
replacementCodePoint
  : inputCodePoint);
  resultBuilder.append(charArray);
}
String resultString = resultBuilder.toString();
return resultString;
  }
{code}

Here is the hadoop.io.Text.bytesToCodePoint() function.

{code:java}
  public static int bytesToCodePoint(ByteBuffer bytes) {
bytes.mark();
byte b = bytes.get();
bytes.reset();
int extraBytesToRead = bytesFromUTF8[(b & 0xFF)];
if (extraBytesToRead < 0) return -1; // trailing byte!
int ch = 0;

switch (extraBytesToRead) {
case 5: ch += (bytes.get() & 0xFF); ch <<= 6; /* remember, illegal UTF-8 */
case 4: ch += (bytes.get() & 0xFF); ch <<= 6; /* remember, illegal UTF-8 */
case 3: ch += (bytes.get() & 0xFF); ch <<= 6;
case 2: ch += (bytes.get() & 0xFF); ch <<= 6;
case 1: ch += (bytes.get() & 0xFF); ch <<= 6;
case 0: ch += (bytes.get() & 0xFF);
}
ch -= offsetsFromUTF8[extraBytesToRead];

return ch;
  }
{code}





--
This message was sent by Atlassian JIRA
(v6.4.14#64029)