Uploaded image for project: 'Hive'
  1. Hive
  2. HIVE-14297

OrcRecordUpdater floods logs trying to create _orc_acid_version file

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Closed
    • Major
    • Resolution: Fixed
    • 1.3.0
    • 1.3.0, 2.1.1, 2.2.0
    • Transactions
    • None

    Description

          try {
            FSDataOutputStream strm = fs.create(new Path(path, ACID_FORMAT), false);
            strm.writeInt(ORC_ACID_VERSION);
            strm.close();
          } catch (IOException ioe) {
            if (LOG.isDebugEnabled()) {
              LOG.debug("Failed to create " + path + "/" + ACID_FORMAT + " with " +
                  ioe);
            }
          }
      

      this file is created in the table/partition dir. So in streaming ingest cases this happens repeatedly and HDFS prints long stack trace with a WARN

      2016-07-18 09:22:13.051 o.a.h.i.r.RetryInvocationHandler [WARN] Exception while invoking ClientNamenodeProtocolTranslatorPB.create over null. Not retrying because try once and fail.
      org.apache.hadoop.ipc.RemoteException: /apps/hive/warehouse/stormdb.db/store_sales/dt=2016%2F07%2F18/_orc_acid_version for client 172.22.111.42 already exists
      	at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.startFileInternal(FSNamesystem.java:2639)
      	at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.startFileInt(FSNamesystem.java:2526)
      	at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.startFile(FSNamesystem.java:2410)
      	at org.apache.hadoop.hdfs.server.namenode.NameNodeRpcServer.create(NameNodeRpcServer.java:729)
      	at org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolServerSideTranslatorPB.create(ClientNamenodeProtocolServerSideTranslatorPB.java:405)
      	at org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos$ClientNamenodeProtocol$2.callBlockingMethod(ClientNamenodeProtocolProtos.java)
      	at org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:640)
      	at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:982)
      	at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2313)
      	at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2309)
      	at java.security.AccessController.doPrivileged(Native Method)
      	at javax.security.auth.Subject.doAs(Subject.java:422)
      	at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1724)
      	at org.apache.hadoop.ipc.Server$Handler.run(Server.java:2307)
      
      	at org.apache.hadoop.ipc.Client.getRpcResponse(Client.java:1552) ~[stormjar.jar:?]
      	at org.apache.hadoop.ipc.Client.call(Client.java:1496) ~[stormjar.jar:?]
      	at org.apache.hadoop.ipc.Client.call(Client.java:1396) ~[stormjar.jar:?]
      	at org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:233) ~[stormjar.jar:?]
      	at com.sun.proxy.$Proxy44.create(Unknown Source) ~[?:?]
      	at org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB.create(ClientNamenodeProtocolTranslatorPB.java:311) ~[stormjar.jar:?]
      	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) ~[?:1.8.0_77]
      	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) ~[?:1.8.0_77]
      	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) ~[?:1.8.0_77]
      	at java.lang.reflect.Method.invoke(Method.java:498) ~[?:1.8.0_77]
      	at org.apache.hadoop.io.retry.RetryInvocationHandler.invokeMethod(RetryInvocationHandler.java:278) [stormjar.jar:?]
      	at org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:194) [stormjar.jar:?]
      	at org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:176) [stormjar.jar:?]
      	at com.sun.proxy.$Proxy45.create(Unknown Source) [?:?]
      	at org.apache.hadoop.hdfs.DFSOutputStream.newStreamForCreate(DFSOutputStream.java:1719) [stormjar.jar:?]
      	at org.apache.hadoop.hdfs.DFSClient.create(DFSClient.java:1699) [stormjar.jar:?]
      	at org.apache.hadoop.hdfs.DFSClient.create(DFSClient.java:1634) [stormjar.jar:?]
      	at org.apache.hadoop.hdfs.DistributedFileSystem$8.doCall(DistributedFileSystem.java:478) [stormjar.jar:?]
      	at org.apache.hadoop.hdfs.DistributedFileSystem$8.doCall(DistributedFileSystem.java:474) [stormjar.jar:?]
      	at org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81) [stormjar.jar:?]
      	at org.apache.hadoop.hdfs.DistributedFileSystem.create(DistributedFileSystem.java:474) [stormjar.jar:?]
      	at org.apache.hadoop.hdfs.DistributedFileSystem.create(DistributedFileSystem.java:415) [stormjar.jar:?]
      	at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:926) [stormjar.jar:?]
      	at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:907) [stormjar.jar:?]
      	at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:803) [stormjar.jar:?]
      	at org.apache.hadoop.hive.ql.io.orc.OrcRecordUpdater.<init>(OrcRecordUpdater.java:238) [stormjar.jar:?]
      	at org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat.getRecordUpdater(OrcOutputFormat.java:289) [stormjar.jar:?]
      	at org.apache.hive.hcatalog.streaming.AbstractRecordWriter.createRecordUpdater(AbstractRecordWriter.java:253) [stormjar.jar:?]
      	at org.apache.hive.hcatalog.streaming.AbstractRecordWriter.createRecordUpdaters(AbstractRecordWriter.java:245) [stormjar.jar:?]
      	at org.apache.hive.hcatalog.streaming.AbstractRecordWriter.newBatch(AbstractRecordWriter.java:189) [stormjar.jar:?]
      	at org.apache.hive.hcatalog.streaming.DelimitedInputWriter.newBatch(DelimitedInputWriter.java:50) [stormjar.jar:?]
      	at org.apache.hive.hcatalog.streaming.HiveEndPoint$TransactionBatchImpl.<init>(HiveEndPoint.java:607) [stormjar.jar:?]
      	at org.apache.hive.hcatalog.streaming.HiveEndPoint$TransactionBatchImpl.<init>(HiveEndPoint.java:555) [stormjar.jar:?]
      	at org.apache.hive.hcatalog.streaming.HiveEndPoint$ConnectionImpl.fetchTransactionBatchImpl(HiveEndPoint.java:441) [stormjar.jar:?]
      	at org.apache.hive.hcatalog.streaming.HiveEndPoint$ConnectionImpl.access$600(HiveEndPoint.java:278) [stormjar.jar:?]
      	at org.apache.hive.hcatalog.streaming.HiveEndPoint$ConnectionImpl$2.run(HiveEndPoint.java:428) [stormjar.jar:?]
      	at org.apache.hive.hcatalog.streaming.HiveEndPoint$ConnectionImpl$2.run(HiveEndPoint.java:425) [stormjar.jar:?]
      	at java.security.AccessController.doPrivileged(Native Method) ~[?:1.8.0_77]
      	at javax.security.auth.Subject.doAs(Subject.java:422) [?:1.8.0_77]
      	at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1724) [stormjar.jar:?]
      	at org.apache.hive.hcatalog.streaming.HiveEndPoint$ConnectionImpl.fetchTransactionBatch(HiveEndPoint.java:424) [stormjar.jar:?]
      	at org.apache.storm.hive.common.HiveWriter$6.call(HiveWriter.java:259) [stormjar.jar:?]
      	at org.apache.storm.hive.common.HiveWriter$6.call(HiveWriter.java:256) [stormjar.jar:?]
      	at org.apache.storm.hive.common.HiveWriter$9.call(HiveWriter.java:369) [stormjar.jar:?]
      	at java.util.concurrent.FutureTask.run(FutureTask.java:266) [?:1.8.0_77]
      	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) [?:1.8.0_77]
      	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) [?:1.8.0_77]
      	at java.lang.Thread.run(Thread.java:745) [?:1.8.0_77]
      

      Should check for existence of the file first.
      also move "strm.close()" to finally block

      Attachments

        1. HIVE-14297.2.patch
          1 kB
          Eugene Koifman
        2. HIVE-14297.3.patch
          1 kB
          Eugene Koifman
        3. HIVE-14297.patch
          1 kB
          Owen O'Malley
        4. HIVE-14297.patch
          1 kB
          Eugene Koifman

        Activity

          People

            ekoifman Eugene Koifman
            ekoifman Eugene Koifman
            Votes:
            0 Vote for this issue
            Watchers:
            3 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: