Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id 86739200B5A for ; Thu, 21 Jul 2016 00:27:22 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id 8519A160A86; Wed, 20 Jul 2016 22:27:22 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id B1B0E160A64 for ; Thu, 21 Jul 2016 00:27:21 +0200 (CEST) Received: (qmail 77663 invoked by uid 500); 20 Jul 2016 22:27:20 -0000 Mailing-List: contact issues-help@hive.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@hive.apache.org Delivered-To: mailing list issues@hive.apache.org Received: (qmail 77633 invoked by uid 99); 20 Jul 2016 22:27:20 -0000 Received: from arcas.apache.org (HELO arcas) (140.211.11.28) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 20 Jul 2016 22:27:20 +0000 Received: from arcas.apache.org (localhost [127.0.0.1]) by arcas (Postfix) with ESMTP id 8E8742C0031 for ; Wed, 20 Jul 2016 22:27:20 +0000 (UTC) Date: Wed, 20 Jul 2016 22:27:20 +0000 (UTC) From: "Eugene Koifman (JIRA)" To: issues@hive.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Commented] (HIVE-14297) OrcRecordUpdater floods logs trying to create _orc_acid_version file MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 archived-at: Wed, 20 Jul 2016 22:27:22 -0000 [ https://issues.apache.org/jira/browse/HIVE-14297?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15386734#comment-15386734 ] Eugene Koifman commented on HIVE-14297: --------------------------------------- [~owen.omalley] could you review please? > OrcRecordUpdater floods logs trying to create _orc_acid_version file > -------------------------------------------------------------------- > > Key: HIVE-14297 > URL: https://issues.apache.org/jira/browse/HIVE-14297 > Project: Hive > Issue Type: Bug > Components: Transactions > Affects Versions: 1.3.0 > Reporter: Eugene Koifman > Assignee: Eugene Koifman > Attachments: HIVE-14297.patch > > > {noformat} > 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); > } > } > {noformat} > 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 > {noformat} > 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.(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.(HiveEndPoint.java:607) [stormjar.jar:?] > at org.apache.hive.hcatalog.streaming.HiveEndPoint$TransactionBatchImpl.(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] > {noformat} > Should check for existence of the file first. > also move "strm.close()" to finally block -- This message was sent by Atlassian JIRA (v6.3.4#6332)