Return-Path: X-Original-To: apmail-crunch-user-archive@www.apache.org Delivered-To: apmail-crunch-user-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 48E0718611 for ; Thu, 1 Oct 2015 06:58:19 +0000 (UTC) Received: (qmail 35476 invoked by uid 500); 1 Oct 2015 06:58:19 -0000 Delivered-To: apmail-crunch-user-archive@crunch.apache.org Received: (qmail 35437 invoked by uid 500); 1 Oct 2015 06:58:19 -0000 Mailing-List: contact user-help@crunch.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: user@crunch.apache.org Delivered-To: mailing list user@crunch.apache.org Received: (qmail 35427 invoked by uid 99); 1 Oct 2015 06:58:19 -0000 Received: from Unknown (HELO spamd3-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 01 Oct 2015 06:58:19 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd3-us-west.apache.org (ASF Mail Server at spamd3-us-west.apache.org) with ESMTP id A32D5180333 for ; Thu, 1 Oct 2015 06:58:18 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd3-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: -0.121 X-Spam-Level: X-Spam-Status: No, score=-0.121 tagged_above=-999 required=6.31 tests=[DKIM_SIGNED=0.1, DKIM_VALID=-0.1, DKIM_VALID_AU=-0.1, RCVD_IN_MSPIKE_H3=-0.01, RCVD_IN_MSPIKE_WL=-0.01, SPF_PASS=-0.001] autolearn=disabled Authentication-Results: spamd3-us-west.apache.org (amavisd-new); dkim=pass (2048-bit key) header.d=gmail.com Received: from mx1-eu-west.apache.org ([10.40.0.8]) by localhost (spamd3-us-west.apache.org [10.40.0.10]) (amavisd-new, port 10024) with ESMTP id cHFDZbDaF95c for ; Thu, 1 Oct 2015 06:58:16 +0000 (UTC) Received: from mail-wi0-f174.google.com (mail-wi0-f174.google.com [209.85.212.174]) by mx1-eu-west.apache.org (ASF Mail Server at mx1-eu-west.apache.org) with ESMTPS id 41967201FB for ; Thu, 1 Oct 2015 06:58:16 +0000 (UTC) Received: by wiclk2 with SMTP id lk2so18206000wic.0 for ; Wed, 30 Sep 2015 23:58:16 -0700 (PDT) DKIM-Signature: v=1; a=rsa-sha256; c=relaxed/relaxed; d=gmail.com; s=20120113; h=mime-version:in-reply-to:references:date:message-id:subject:from:to :content-type; bh=BVygxkLONiFhRtU+jLUexXBTbgB8YPH3I3ptrHO/Xug=; b=qXXGETudfgc9RWcnLhqTrhRXJsWOeOWSgMdnlxr/aoxvZA2t18856CPChzRX2wq8JW NVdeGmk92FxusF9rMV8QeEq6LxUMjzyfzthG/Al7uibVSoL5ELBYAngQfzPXln8Q8KTt +Nmb4qJbzwl345MCsaeJMmHgpppGM8uzuVwDD1un2sOJiANUC7huZuqRXDozaPecpgG4 2SBs5NPqfiNM1C5Bnkyo77eP7pC5e4Tt2gYn+ebqQZ/k0l9VRx+xPmJvTA55dWrsltH5 gzKbxMQmautDOwKOLp//6i9bBG3naZ6x3Z+aAPwmnqmfkGSV7f22uAtOkba8aA8E8viv eYaw== MIME-Version: 1.0 X-Received: by 10.194.192.6 with SMTP id hc6mr8287436wjc.33.1443682695967; Wed, 30 Sep 2015 23:58:15 -0700 (PDT) Received: by 10.28.90.68 with HTTP; Wed, 30 Sep 2015 23:58:15 -0700 (PDT) In-Reply-To: References: Date: Thu, 1 Oct 2015 08:58:15 +0200 Message-ID: Subject: Re: GSSException: No valid credentials provided (Mechanism level: Failed to find any Kerberos tgt) From: Gabriel Reid To: "user@crunch.apache.org" Content-Type: text/plain; charset=UTF-8 If I'm reading that stack trace correctly, CEDoFn is reading from an HBase table in its initialize method (probably via a ReadableData) instance. It looks like the HBase instance is kerberized, which will mean that TableMapReduceUtil.initCredentials(Job) needs to be called before submitting the job. There was a relatively recent patch added in Crunch (see CRUNCH-536) to make it easier to add the call to TableMapReduceUtil.initCredentials. If you build a version of Crunch with CRUNCH-536 included, you should be able to add the following call during the setup of your pipeline: pipeline.addPrepareHook(new CrunchControlledJob.Hook() { @Override public void run(MRJob mrJob) throws IOException { TableMapReduceUtil.initCredentials(mrJob.getJob()); } }); - Gabriel On Wed, Sep 30, 2015 at 11:17 PM, Tahir Hameed wrote: > It is HDFS. The setup for both pipelines is the same too. > > > > On Wed, Sep 30, 2015 at 10:17 PM, Micah Whitacre > wrote: >> >> What is the datastore you are reading from? HBase? HDFS? Also is there >> any setup differences between the two pipelines? >> >> On Wed, Sep 30, 2015 at 3:13 PM, Tahir Hameed wrote: >>> >>> Hi, >>> >>> I am facing a queer problem. I have 2 MR pipelines. One of them is >>> working fine. The other is not. >>> >>> The difference lies in only one of the DoFN functions. >>> >>> >>> The DoFn function which fails is given below: >>> >>> public PTable >>> myFunction(PTable> joinedData, >>> PTable others) { >>> >>> ReadableData> readable = >>> others.asReadable(false); >>> ParallelDoOptions options = ParallelDoOptions.builder() >>> .sourceTargets(readable.getSourceTargets()) >>> .build(); >>> >>> return joinedData >>> .by(someMapFunction, >>> Avros.writables(ImmutableBytesWritable.class)) >>> .groupByKey() >>> .parallelDo("", new CEDoFN(readable, >>> others.getPTableType()), >>> >>> Avros.tableOf(Avros.writables(ImmutableBytesWritable.class), >>> Avros.reflects(CE.class)), options); >>> >>> } >>> >>> The stack trace is as follows : >>> >>> javax.security.sasl.SaslException: GSS initiate failed [Caused by >>> GSSException: No valid credentials provided (Mechanism level: Failed to find >>> any Kerberos tgt)] >>> at >>> com.sun.security.sasl.gsskerb.GssKrb5Client.evaluateChallenge(GssKrb5Client.java:212) >>> at >>> org.apache.hadoop.hbase.security.HBaseSaslRpcClient.saslConnect(HBaseSaslRpcClient.java:177) >>> at >>> org.apache.hadoop.hbase.ipc.RpcClient$Connection.setupSaslConnection(RpcClient.java:815) >>> at >>> org.apache.hadoop.hbase.ipc.RpcClient$Connection.access$800(RpcClient.java:349) >>> at >>> org.apache.hadoop.hbase.ipc.RpcClient$Connection$2.run(RpcClient.java:943) >>> at >>> org.apache.hadoop.hbase.ipc.RpcClient$Connection$2.run(RpcClient.java:940) >>> at java.security.AccessController.doPrivileged(Native Method) >>> at javax.security.auth.Subject.doAs(Subject.java:415) >>> at >>> org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1628) >>> at >>> org.apache.hadoop.hbase.ipc.RpcClient$Connection.setupIOstreams(RpcClient.java:940) >>> at >>> org.apache.hadoop.hbase.ipc.RpcClient$Connection.writeRequest(RpcClient.java:1094) >>> at >>> org.apache.hadoop.hbase.ipc.RpcClient$Connection.tracedWriteRequest(RpcClient.java:1061) >>> at org.apache.hadoop.hbase.ipc.RpcClient.call(RpcClient.java:1516) >>> at >>> org.apache.hadoop.hbase.ipc.RpcClient.callBlockingMethod(RpcClient.java:1724) >>> at >>> org.apache.hadoop.hbase.ipc.RpcClient$BlockingRpcChannelImplementation.callBlockingMethod(RpcClient.java:1777) >>> at >>> org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$BlockingStub.get(ClientProtos.java:30373) >>> at >>> org.apache.hadoop.hbase.protobuf.ProtobufUtil.getRowOrBefore(ProtobufUtil.java:1604) >>> at org.apache.hadoop.hbase.client.HTable$2.call(HTable.java:768) >>> at org.apache.hadoop.hbase.client.HTable$2.call(HTable.java:766) >>> at >>> org.apache.hadoop.hbase.client.RpcRetryingCaller.callWithRetries(RpcRetryingCaller.java:126) >>> at org.apache.hadoop.hbase.client.HTable.getRowOrBefore(HTable.java:772) >>> at >>> org.apache.hadoop.hbase.client.MetaScanner.metaScan(MetaScanner.java:160) >>> at >>> org.apache.hadoop.hbase.client.ConnectionManager$HConnectionImplementation.prefetchRegionCache(ConnectionManager.java:1254) >>> at >>> org.apache.hadoop.hbase.client.ConnectionManager$HConnectionImplementation.locateRegionInMeta(ConnectionManager.java:1318) >>> at >>> org.apache.hadoop.hbase.client.ConnectionManager$HConnectionImplementation.locateRegion(ConnectionManager.java:1167) >>> at >>> org.apache.hadoop.hbase.client.RpcRetryingCallerWithReadReplicas.getRegionLocations(RpcRetryingCallerWithReadReplicas.java:294) >>> at >>> org.apache.hadoop.hbase.client.ScannerCallableWithReplicas.call(ScannerCallableWithReplicas.java:130) >>> at >>> org.apache.hadoop.hbase.client.ScannerCallableWithReplicas.call(ScannerCallableWithReplicas.java:55) >>> at >>> org.apache.hadoop.hbase.client.RpcRetryingCaller.callWithoutRetries(RpcRetryingCaller.java:201) >>> at >>> org.apache.hadoop.hbase.client.ClientScanner.call(ClientScanner.java:288) >>> at >>> org.apache.hadoop.hbase.client.ClientScanner.nextScanner(ClientScanner.java:268) >>> at >>> org.apache.hadoop.hbase.client.ClientScanner.initializeScannerInConstruction(ClientScanner.java:140) >>> at >>> org.apache.hadoop.hbase.client.ClientScanner.(ClientScanner.java:135) >>> at org.apache.hadoop.hbase.client.HTable.getScanner(HTable.java:802) >>> at >>> org.apache.crunch.io.hbase.HTableIterator.(HTableIterator.java:47) >>> at >>> org.apache.crunch.io.hbase.HTableIterable.iterator(HTableIterable.java:43) >>> at >>> org.apache.crunch.util.DelegatingReadableData$1.iterator(DelegatingReadableData.java:63) >>> at com.bol.step.enrichmentdashboard.fn.CEDoFN.initialize(CEDoFN.java:45) >>> at org.apache.crunch.impl.mr.run.RTNode.initialize(RTNode.java:71) >>> at org.apache.crunch.impl.mr.run.RTNode.initialize(RTNode.java:73) >>> at >>> org.apache.crunch.impl.mr.run.CrunchReducer.setup(CrunchReducer.java:44) >>> at org.apache.hadoop.mapreduce.Reducer.run(Reducer.java:168) >>> at >>> org.apache.hadoop.mapred.ReduceTask.runNewReducer(ReduceTask.java:627) >>> at org.apache.hadoop.mapred.ReduceTask.run(ReduceTask.java:389) >>> at org.apache.hadoop.mapred.YarnChild$2.run(YarnChild.java:163) >>> at java.security.AccessController.doPrivileged(Native Method) >>> at javax.security.auth.Subject.doAs(Subject.java:415) >>> at >>> org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1628) >>> at org.apache.hadoop.mapred.YarnChild.main(YarnChild.java:158) >>> Caused by: GSSException: No valid credentials provided (Mechanism level: >>> Failed to find any Kerberos tgt) >>> >>> >>> In the CEDoFunction, the readable is used in the initialization phase to >>> create a HashMap. This is the place where the stack trace error also points >>> to. >>> >>> In the function which succeeds, the parallelDo is performed directly on >>> the joinedData which is also a PTable, and there are no errors. The >>> initialization phases for both functions are exactly the same. >>> >>> I fail to understand the cause of the errors because the underlying >>> implementations for the both PTable and PGroupedTable is the same because >>> both seem to be extending the PCollectionImpl interface. >>> >>> Tahir >>> >>> >>> >>> >>> >>> >>> >>> >> >