Return-Path: X-Original-To: apmail-spark-reviews-archive@minotaur.apache.org Delivered-To: apmail-spark-reviews-archive@minotaur.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id E2D451836C for ; Thu, 29 Oct 2015 00:16:29 +0000 (UTC) Received: (qmail 56465 invoked by uid 500); 29 Oct 2015 00:16:29 -0000 Delivered-To: apmail-spark-reviews-archive@spark.apache.org Received: (qmail 56445 invoked by uid 500); 29 Oct 2015 00:16:29 -0000 Mailing-List: contact reviews-help@spark.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Delivered-To: mailing list reviews@spark.apache.org Received: (qmail 56434 invoked by uid 99); 29 Oct 2015 00:16:29 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 29 Oct 2015 00:16:29 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 54730DFBD6; Thu, 29 Oct 2015 00:16:29 +0000 (UTC) From: vanzin To: reviews@spark.apache.org Reply-To: reviews@spark.apache.org References: In-Reply-To: Subject: [GitHub] spark pull request: [SPARK-11265] [YARN] YarnClient can't get toke... Content-Type: text/plain Message-Id: <20151029001629.54730DFBD6@git1-us-west.apache.org> Date: Thu, 29 Oct 2015 00:16:29 +0000 (UTC) Github user vanzin commented on a diff in the pull request: https://github.com/apache/spark/pull/9232#discussion_r43336052 --- Diff: yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala --- @@ -245,4 +247,55 @@ class YarnSparkHadoopUtilSuite extends SparkFunSuite with Matchers with Logging System.clearProperty("SPARK_YARN_MODE") } } + + test("Obtain tokens For HiveMetastore") { + val hadoopConf = new Configuration() + hadoopConf.set("hive.metastore.kerberos.principal", "bob") + // thrift picks up on port 0 and bails out, without trying to talk to endpoint + hadoopConf.set("hive.metastore.uris", "http://localhost:0") + val util = new YarnSparkHadoopUtil + val e = intercept[InvocationTargetException] { + util.obtainTokenForHiveMetastoreInner(hadoopConf, "alice") + } + assertNestedHiveException(e) + // expect exception trapping code to unwind this hive-side exception + assertNestedHiveException(intercept[InvocationTargetException] { + util.obtainTokenForHiveMetastore(hadoopConf) + }) + } + + def assertNestedHiveException(e: InvocationTargetException): Throwable = { + val inner = e.getCause + if (inner == null) { + fail("No inner cause", e) + } + if (!inner.isInstanceOf[HiveException]) { + fail(s"Not a hive exception", inner) + } + inner + } + + test("handleTokenIntrospectionFailure") { + val util = new YarnSparkHadoopUtil + // downgraded exceptions + util.handleTokenIntrospectionFailure("hive", new ClassNotFoundException("cnfe")) --- End diff -- Or yet another option is to have the method that handles exception take a closure, instead of the current approach of a method that matches on an exception parameter. e.g.: def tryToGetTokens(service: String)(fn: () => Option[Token]): Option[Token] = { try { fn() } catch { ... } } def obtainTokenForHiveMetastore... = { tryToGetTokens("Hive") { obtainTokenForHiveMetastoreInner(...) } } I mostly dislike that exception handling feels like it's scattered around. You have a catch block in one place, then match on the exception somewhere else, it makes it hard to see what's really being done in one look. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastructure@apache.org or file a JIRA ticket with INFRA. --- --------------------------------------------------------------------- To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org For additional commands, e-mail: reviews-help@spark.apache.org