hive-dev mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "J. Chris Folsom (Created) (JIRA)" <j...@apache.org>
Subject [jira] [Created] (HIVE-2485) LOAD DATA LOCAL Leaves Files Hanging Open
Date Wed, 05 Oct 2011 19:30:29 GMT
LOAD DATA LOCAL Leaves Files Hanging Open
-----------------------------------------

                 Key: HIVE-2485
                 URL: https://issues.apache.org/jira/browse/HIVE-2485
             Project: Hive
          Issue Type: Bug
          Components: CLI
    Affects Versions: 0.7.1
         Environment: Linux - Hadoop 0.20.204 Hive 0.7.0, Hive 0.7.1, Hive 0.8.0, trunk
            Reporter: J. Chris Folsom


When loading multiple files using the statement:

DROP TABLE partition_test;
CREATE TABLE partition_test (key INT, value STRING) PARTITIONED BY (ts STRING) STORED AS TEXTFILE;
LOAD DATA LOCAL INPATH 'file:/myfile.txt' OVERWRITE INTO TABLE partition_test PARTITION(ts='1');
LOAD DATA LOCAL INPATH 'file:/myfile2.txt' OVERWRITE INTO TABLE partition_test PARTITION(ts='2');
LOAD DATA LOCAL INPATH 'file:/myfile3.txt' OVERWRITE INTO TABLE partition_test PARTITION(ts='3');
etc. up to 250...

Hive CLI fails with error:

Failed with exception null

hive.log:

2011-10-05 15:07:11,899 WARN  hdfs.DFSClient (DFSClient.java:processDatanodeError(2667)) -
Error Recovery for block blk_-7990368440974156305_1721 bad datanode[0] nodes == null
2011-10-05 15:07:11,899 WARN  hdfs.DFSClient (DFSClient.java:processDatanodeError(2695)) -
Could not get block locations. Source file "/tmp/hive-cfolsom/hive_2011-10-05_15-06-47_812_4664863850423838867/-ext-10000/kv1.txt"
- Aborting...
2011-10-05 15:07:11,900 ERROR exec.Task (SessionState.java:printError(365)) - Failed with
exception null
java.io.EOFException
        at java.io.DataInputStream.readShort(DataInputStream.java:298)
        at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.createBlockOutputStream(DFSClient.java:3065)
        at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.nextBlockOutputStream(DFSClient.java:2988)
        at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.access$2000(DFSClient.java:2260)
        at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream$DataStreamer.run(DFSClient.java:2451)

2011-10-05 15:07:11,901 ERROR hdfs.DFSClient (DFSClient.java:close(1125)) - Exception closing
file /tmp/hive-cfolsom/hive_2011-10-05_15-06-47_812_4664863850423838867/-ext-10000/kv1.txt
: java.io.EOFException
java.io.EOFException
        at java.io.DataInputStream.readShort(DataInputStream.java:298)
        at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.createBlockOutputStream(DFSClient.java:3065)
        at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.nextBlockOutputStream(DFSClient.java:2988)
        at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.access$2000(DFSClient.java:2260)
        at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream$DataStreamer.run(DFSClient.java:2451)
2011-10-05 15:07:12,002 ERROR ql.Driver (SessionState.java:printError(365)) - FAILED: Execution
Error, return code 1 from org.apache.hadoop.hive.ql.exec.CopyTask


Hadoop datanode log: there are too many open files:

java.io.FileNotFoundException: /tmp/hadoop-hadoop/dfs/data/current/subdir4/blk_207900366872942737
(Too many open files)
        at java.io.RandomAccessFile.open(Native Method)
        at java.io.RandomAccessFile.<init>(RandomAccessFile.java:212)
        at org.apache.hadoop.hdfs.server.datanode.FSDataset.getBlockInputStream(FSDataset.java:862)
        at org.apache.hadoop.hdfs.server.datanode.BlockSender.<init>(BlockSender.java:166)
        at org.apache.hadoop.hdfs.server.datanode.DataXceiver.readBlock(DataXceiver.java:189)
        at org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:99)
        at java.lang.Thread.run(Thread.java:619)

2011-10-05 15:06:14,532 ERROR org.apache.hadoop.hdfs.server.datanode.DataNode: DatanodeRegistration(127.0.0.1:50010,
storageID=DS-1718046113-127.0.0.1-50010-1317838257821, infoPort=50075, ipcPort=50020):DataXceiver
java.io.FileNotFoundException: /tmp/hadoop-hadoop/dfs/data/current/subdir4/blk_207900366872942737
(Too many open files)
        at java.io.RandomAccessFile.open(Native Method)
        at java.io.RandomAccessFile.<init>(RandomAccessFile.java:212)
        at org.apache.hadoop.hdfs.server.datanode.FSDataset.getBlockInputStream(FSDataset.java:862)
        at org.apache.hadoop.hdfs.server.datanode.BlockSender.<init>(BlockSender.java:166)
        at org.apache.hadoop.hdfs.server.datanode.DataXceiver.readBlock(DataXceiver.java:189)
        at org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:99)
        at java.lang.Thread.run(Thread.java:619)
2011-10-05 15:06:14,772 WARN org.apache.hadoop.hdfs.server.datanode.DataNode: java.io.IOException:
Call to localhost/127.0.0.1:9000 failed on local exception: java.io.IOException: Too many
open files
        at org.apache.hadoop.ipc.Client.wrapException(Client.java:1065)
        at org.apache.hadoop.ipc.Client.call(Client.java:1033)
        at org.apache.hadoop.ipc.RPC$Invoker.invoke(RPC.java:224)
        at $Proxy5.sendHeartbeat(Unknown Source)
        at org.apache.hadoop.hdfs.server.datanode.DataNode.offerService(DataNode.java:853)
        at org.apache.hadoop.hdfs.server.datanode.DataNode.run(DataNode.java:1356)
        at java.lang.Thread.run(Thread.java:619)
Caused by: java.io.IOException: Too many open files
        at sun.nio.ch.EPollArrayWrapper.epollCreate(Native Method)
        at sun.nio.ch.EPollArrayWrapper.<init>(EPollArrayWrapper.java:69)
        at sun.nio.ch.EPollSelectorImpl.<init>(EPollSelectorImpl.java:52)
        at sun.nio.ch.EPollSelectorProvider.openSelector(EPollSelectorProvider.java:18)
        at org.apache.hadoop.net.SocketIOWithTimeout$SelectorPool.get(SocketIOWithTimeout.java:407)
        at org.apache.hadoop.net.SocketIOWithTimeout$SelectorPool.select(SocketIOWithTimeout.java:322)
        at org.apache.hadoop.net.SocketIOWithTimeout.doIO(SocketIOWithTimeout.java:157)
        at org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:155)
        at org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:128)
        at java.io.FilterInputStream.read(FilterInputStream.java:116)
        at org.apache.hadoop.ipc.Client$Connection$PingInputStream.read(Client.java:343)
        at java.io.BufferedInputStream.fill(BufferedInputStream.java:218)
        at java.io.BufferedInputStream.read(BufferedInputStream.java:237)
        at java.io.DataInputStream.readInt(DataInputStream.java:370)
        at org.apache.hadoop.ipc.Client$Connection.receiveResponse(Client.java:767)
        at org.apache.hadoop.ipc.Client$Connection.run(Client.java:712)

2011-10-05 15:06:17,535 WARN org.apache.hadoop.hdfs.server.datanode.DataNode: DatanodeRegistration(127.0.0.1:50010,
storageID=DS-1718046113-127.0.0.1-50010-1317838257821, infoPort=50075, ipcPort=50020):Got
exception while serving blk_207900366872942737_1719 to /127.0.0.1:
java.io.FileNotFoundException: /tmp/hadoop-hadoop/dfs/data/current/subdir4/blk_207900366872942737_1719.meta
(Too many open files)
        at java.io.FileInputStream.open(Native Method)
        at java.io.FileInputStream.<init>(FileInputStream.java:106)
        at org.apache.hadoop.hdfs.server.datanode.FSDataset.getMetaDataInputStream(FSDataset.java:751)
        at org.apache.hadoop.hdfs.server.datanode.BlockSender.<init>(BlockSender.java:97)
        at org.apache.hadoop.hdfs.server.datanode.DataXceiver.readBlock(DataXceiver.java:189)
        at org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:99)
        at java.lang.Thread.run(Thread.java:619)

2011-10-05 15:06:17,535 ERROR org.apache.hadoop.hdfs.server.datanode.DataNode: DatanodeRegistration(127.0.0.1:50010,
storageID=DS-1718046113-127.0.0.1-50010-1317838257821, infoPort=50075, ipcPort=50020):DataXceiver
java.io.FileNotFoundException: /tmp/hadoop-hadoop/dfs/data/current/subdir4/blk_207900366872942737_1719.meta
(Too many open files)
        at java.io.FileInputStream.open(Native Method)
        at java.io.FileInputStream.<init>(FileInputStream.java:106)
        at org.apache.hadoop.hdfs.server.datanode.FSDataset.getMetaDataInputStream(FSDataset.java:751)
        at org.apache.hadoop.hdfs.server.datanode.BlockSender.<init>(BlockSender.java:97)
        at org.apache.hadoop.hdfs.server.datanode.DataXceiver.readBlock(DataXceiver.java:189)
        at org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:99)
        at java.lang.Thread.run(Thread.java:619)
2011-10-05 15:06:18,771 INFO org.apache.hadoop.ipc.Client: Retrying connect to server: localhost/127.0.0.1:9000.
Already tried 0 time(s).
2011-10-05 15:06:19,772 INFO org.apache.hadoop.ipc.Client: Retrying connect to server: localhost/127.0.0.1:9000.
Already tried 1 time(s).
2011-10-05 15:06:20,539 WARN org.apache.hadoop.hdfs.server.datanode.DataNode: DatanodeRegistration(127.0.0.1:50010,
storageID=DS-1718046113-127.0.0.1-50010-1317838257821, infoPort=50075, ipcPort=50020):Got
exception while serving blk_207900366872942737_1719 to /127.0.0.1:
java.io.FileNotFoundException: /tmp/hadoop-hadoop/dfs/data/current/subdir4/blk_207900366872942737_1719.meta
(Too many open files)
        at java.io.FileInputStream.open(Native Method)
        at java.io.FileInputStream.<init>(FileInputStream.java:106)
        at org.apache.hadoop.hdfs.server.datanode.FSDataset.getMetaDataInputStream(FSDataset.java:751)
        at org.apache.hadoop.hdfs.server.datanode.BlockSender.<init>(BlockSender.java:97)
        at org.apache.hadoop.hdfs.server.datanode.DataXceiver.readBlock(DataXceiver.java:189)
        at org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:99)
        at java.lang.Thread.run(Thread.java:619)

a quick lsof reveals that the datanode pid has 1092 open files.

At first, I suspected that this was because the CopyTask was opening instances of FileSystem
that it did not close, but updating CopyTask to make sure that the source and destination
FileSystems were closed did not resolve the issue. I suspect that some other task has left
files hanging open elsewhere. In any case, this causes more than a bit of a problem as the
datanode is completely unavailable after a number of file loads. The process is still running,
but the node is effectively dead.






--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

Mime
View raw message