[jira] [Created] (SPARK-24526) Spaces in the build dir causes failures in the build/mvn script
Trystan Leftwich created SPARK-24526: Summary: Spaces in the build dir causes failures in the build/mvn script Key: SPARK-24526 URL: https://issues.apache.org/jira/browse/SPARK-24526 Project: Spark Issue Type: Bug Components: Build Affects Versions: 2.3.0 Reporter: Trystan Leftwich If you are running make-distribution in a path that contains a space in it the build/mvn script will fail: {code:bash} mkdir /tmp/test\ spaces cd /tmp/test\ spaces git clone https://github.com/apache/spark.git cd spark # Remove all mvn references in PATH so the script will download mvn to the local dir ./build/mvn -DskipTests clean package{code} You will get the following errors: {code:bash} Using `mvn` from path: /tmp/test spaces/spark/build/apache-maven-3.3.9/bin/mvn ./build/mvn: line 157: /tmp/test: No such file or directory {code} -- This message was sent by Atlassian JIRA (v7.6.3#76005) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Commented] (SPARK-16017) YarnClientSchedulerBackend now registers backends as IPs instead of Hostnames which causes all tasks to run with RACK_LOCAL locality.
[ https://issues.apache.org/jira/browse/SPARK-16017?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15337154#comment-15337154 ] Trystan Leftwich commented on SPARK-16017: -- [~zsxwing] I've tested your fix locally and it all is working as expected. Thanks. > YarnClientSchedulerBackend now registers backends as IPs instead of Hostnames > which causes all tasks to run with RACK_LOCAL locality. > - > > Key: SPARK-16017 > URL: https://issues.apache.org/jira/browse/SPARK-16017 > Project: Spark > Issue Type: Bug > Components: Spark Core >Affects Versions: 1.6.2, 2.0.0 >Reporter: Trystan Leftwich >Priority: Critical > > Since this change: > [SPARK-15395|https://issues.apache.org/jira/browse/SPARK-15395] > When registering new executor backends it registers them as IPs instead of > hostnames. This causes a flow on effect that when the Task manager is trying > to figure out what Locality tasks should run at, no tasks can be run At the > NODE_LOCAL level. > This specific call: > https://github.com/apache/spark/blob/branch-2.0/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala#L886 > [pendingTasksForHost|https://github.com/apache/spark/blob/branch-2.0/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala#L886] > are all hostnames pulled from the DFS locations while > [hasExecutorsAliveOnHost|https://github.com/apache/spark/blob/9b234b55d1b5e4a7c80e482b3e297bfb8b583a56/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala#L560], > uses > [executorsByHost|https://github.com/apache/spark/blob/branch-2.0/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala#L98], > which are all IP's because they are populated from the RpcAddress. > As expected this causes significant performance problems, A simple count > query will take 22 seconds, But if I revert the change from > [SPARK-15395|https://issues.apache.org/jira/browse/SPARK-15395], tasks will > run with NODE_LOCAL locality and the same count will take 3 seconds. -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Commented] (SPARK-16017) YarnClientSchedulerBackend now registers backends as IPs instead of Hostnames which causes all tasks to run with RACK_LOCAL locality.
[ https://issues.apache.org/jira/browse/SPARK-16017?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15337004#comment-15337004 ] Trystan Leftwich commented on SPARK-16017: -- [~zsxwing] I'll run the tests here shortly and get back to you. > YarnClientSchedulerBackend now registers backends as IPs instead of Hostnames > which causes all tasks to run with RACK_LOCAL locality. > - > > Key: SPARK-16017 > URL: https://issues.apache.org/jira/browse/SPARK-16017 > Project: Spark > Issue Type: Bug > Components: Spark Core >Affects Versions: 1.6.2, 2.0.0 >Reporter: Trystan Leftwich >Priority: Critical > > Since this change: > [SPARK-15395|https://issues.apache.org/jira/browse/SPARK-15395] > When registering new executor backends it registers them as IPs instead of > hostnames. This causes a flow on effect that when the Task manager is trying > to figure out what Locality tasks should run at, no tasks can be run At the > NODE_LOCAL level. > This specific call: > https://github.com/apache/spark/blob/branch-2.0/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala#L886 > [pendingTasksForHost|https://github.com/apache/spark/blob/branch-2.0/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala#L886] > are all hostnames pulled from the DFS locations while > [hasExecutorsAliveOnHost|https://github.com/apache/spark/blob/9b234b55d1b5e4a7c80e482b3e297bfb8b583a56/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala#L560], > uses > [executorsByHost|https://github.com/apache/spark/blob/branch-2.0/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala#L98], > which are all IP's because they are populated from the RpcAddress. > As expected this causes significant performance problems, A simple count > query will take 22 seconds, But if I revert the change from > [SPARK-15395|https://issues.apache.org/jira/browse/SPARK-15395], tasks will > run with NODE_LOCAL locality and the same count will take 3 seconds. -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Created] (SPARK-16017) YarnClientSchedulerBackend now registers backends as IPs instead of Hostnames which causes all tasks to run with RACK_LOCAL locality.
Trystan Leftwich created SPARK-16017: Summary: YarnClientSchedulerBackend now registers backends as IPs instead of Hostnames which causes all tasks to run with RACK_LOCAL locality. Key: SPARK-16017 URL: https://issues.apache.org/jira/browse/SPARK-16017 Project: Spark Issue Type: Bug Reporter: Trystan Leftwich Priority: Critical Since this change: [SPARK-15395|https://issues.apache.org/jira/browse/SPARK-15395] When registering new executor backends it registers them as IPs instead of hostnames. This causes a flow on effect that when the Task manager is trying to figure out what Locality tasks should run at, no tasks can be run At the NODE_LOCAL level. This specific call: https://github.com/apache/spark/blob/branch-2.0/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala#L886 [pendingTasksForHost|https://github.com/apache/spark/blob/branch-2.0/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala#L886] are all hostnames pulled from the DFS locations while [hasExecutorsAliveOnHost|https://github.com/apache/spark/blob/9b234b55d1b5e4a7c80e482b3e297bfb8b583a56/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala#L560], uses [executorsByHost|https://github.com/apache/spark/blob/branch-2.0/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala#L98], which are all IP's because they are populated from the RpcAddress. As expected this causes significant performance problems, A simple count query will take 22 seconds, But if I revert the change from [SPARK-15395|https://issues.apache.org/jira/browse/SPARK-15395], tasks will run with NODE_LOCAL locality and the same count will take 3 seconds. -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Commented] (SPARK-15046) When running hive-thriftserver with yarn on a secure cluster the workers fail with java.lang.NumberFormatException
[ https://issues.apache.org/jira/browse/SPARK-15046?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15316546#comment-15316546 ] Trystan Leftwich commented on SPARK-15046: -- It looks like the Spark config is converting any Int value over to a TimeUnit MS value: https://github.com/apache/spark/blob/8b44bd52fa40c0fc7d34798c3654e31533fd3008/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala#L246 So when the configs are written out to be sent to the Worker nodes the values are now TimeUnit ms, which is why this error is happening. > When running hive-thriftserver with yarn on a secure cluster the workers fail > with java.lang.NumberFormatException > -- > > Key: SPARK-15046 > URL: https://issues.apache.org/jira/browse/SPARK-15046 > Project: Spark > Issue Type: Bug >Affects Versions: 2.0.0 >Reporter: Trystan Leftwich > > When running hive-thriftserver with yarn on a secure cluster > (spark.yarn.principal and spark.yarn.keytab are set) the workers fail with > the following error. > {code} > 16/04/30 22:40:50 ERROR yarn.ApplicationMaster: Uncaught exception: > java.lang.NumberFormatException: For input string: "86400079ms" > at > java.lang.NumberFormatException.forInputString(NumberFormatException.java:65) > at java.lang.Long.parseLong(Long.java:441) > at java.lang.Long.parseLong(Long.java:483) > at > scala.collection.immutable.StringLike$class.toLong(StringLike.scala:276) > at scala.collection.immutable.StringOps.toLong(StringOps.scala:29) > at > org.apache.spark.SparkConf$$anonfun$getLong$2.apply(SparkConf.scala:380) > at > org.apache.spark.SparkConf$$anonfun$getLong$2.apply(SparkConf.scala:380) > at scala.Option.map(Option.scala:146) > at org.apache.spark.SparkConf.getLong(SparkConf.scala:380) > at > org.apache.spark.deploy.SparkHadoopUtil.getTimeFromNowToRenewal(SparkHadoopUtil.scala:289) > at > org.apache.spark.deploy.yarn.AMDelegationTokenRenewer.org$apache$spark$deploy$yarn$AMDelegationTokenRenewer$$scheduleRenewal$1(AMDelegationTokenRenewer.scala:89) > at > org.apache.spark.deploy.yarn.AMDelegationTokenRenewer.scheduleLoginFromKeytab(AMDelegationTokenRenewer.scala:121) > at > org.apache.spark.deploy.yarn.ApplicationMaster$$anonfun$run$3.apply(ApplicationMaster.scala:243) > at > org.apache.spark.deploy.yarn.ApplicationMaster$$anonfun$run$3.apply(ApplicationMaster.scala:243) > at scala.Option.foreach(Option.scala:257) > at > org.apache.spark.deploy.yarn.ApplicationMaster.run(ApplicationMaster.scala:243) > at > org.apache.spark.deploy.yarn.ApplicationMaster$$anonfun$main$1.apply$mcV$sp(ApplicationMaster.scala:723) > at > org.apache.spark.deploy.SparkHadoopUtil$$anon$1.run(SparkHadoopUtil.scala:67) > at > org.apache.spark.deploy.SparkHadoopUtil$$anon$1.run(SparkHadoopUtil.scala:66) > 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.spark.deploy.SparkHadoopUtil.runAsSparkUser(SparkHadoopUtil.scala:66) > at > org.apache.spark.deploy.yarn.ApplicationMaster$.main(ApplicationMaster.scala:721) > at > org.apache.spark.deploy.yarn.ExecutorLauncher$.main(ApplicationMaster.scala:748) > at > org.apache.spark.deploy.yarn.ExecutorLauncher.main(ApplicationMaster.scala) > {code} -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Updated] (SPARK-14318) TPCDS query 14 causes Spark SQL to hang
[ https://issues.apache.org/jira/browse/SPARK-14318?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Trystan Leftwich updated SPARK-14318: - Description: TPCDS Q14 parses successfully, and plans created successfully. Spark tries to run (I used only 1GB text file), but "hangs". Tasks are extremely slow to process AND all CPUs are used 100% by the executor JVMs. It is very easy to reproduce: 1. Use the spark-sql CLI to run the query 14 (TPCDS) against a database of 1GB text file (assuming you know how to generate the csv data). My command is like this: {noformat} /TestAutomation/downloads/spark-master/bin/spark-sql --driver-memory 10g --verbose --master yarn-client --packages com.databricks:spark-csv_2.10:1.3.0 --executor-memory 8g --num-executors 4 --executor-cores 4 --conf spark.sql.join.preferSortMergeJoin=true --database hadoopds1g -f $f > q14.o{noformat} The Spark console output: {noformat} 16/03/31 15:45:37 INFO scheduler.TaskSetManager: Starting task 26.0 in stage 17.0 (TID 65, bigaperf138.svl.ibm.com, partition 26,RACK_LOCAL, 4515 bytes) 16/03/31 15:45:37 INFO cluster.YarnClientSchedulerBackend: Launching task 65 on executor id: 4 hostname: bigaperf138.svl.ibm.com. 16/03/31 15:45:37 INFO scheduler.TaskSetManager: Finished task 23.0 in stage 17.0 (TID 62) in 829687 ms on bigaperf138.svl.ibm.com (15/200) 16/03/31 15:45:52 INFO scheduler.TaskSetManager: Starting task 27.0 in stage 17.0 (TID 66, bigaperf138.svl.ibm.com, partition 27,RACK_LOCAL, 4515 bytes) 16/03/31 15:45:52 INFO cluster.YarnClientSchedulerBackend: Launching task 66 on executor id: 4 hostname: bigaperf138.svl.ibm.com. 16/03/31 15:45:52 INFO scheduler.TaskSetManager: Finished task 26.0 in stage 17.0 (TID 65) in 15505 ms on bigaperf138.svl.ibm.com (16/200) 16/03/31 15:46:17 INFO scheduler.TaskSetManager: Starting task 28.0 in stage 17.0 (TID 67, bigaperf138.svl.ibm.com, partition 28,RACK_LOCAL, 4515 bytes) 16/03/31 15:46:17 INFO cluster.YarnClientSchedulerBackend: Launching task 67 on executor id: 4 hostname: bigaperf138.svl.ibm.com. 16/03/31 15:46:17 INFO scheduler.TaskSetManager: Finished task 27.0 in stage 17.0 (TID 66) in 24929 ms on bigaperf138.svl.ibm.com (17/200) 16/03/31 15:51:53 INFO scheduler.TaskSetManager: Starting task 29.0 in stage 17.0 (TID 68, bigaperf137.svl.ibm.com, partition 29,NODE_LOCAL, 4515 bytes) 16/03/31 15:51:53 INFO cluster.YarnClientSchedulerBackend: Launching task 68 on executor id: 2 hostname: bigaperf137.svl.ibm.com. 16/03/31 15:51:53 INFO scheduler.TaskSetManager: Finished task 10.0 in stage 17.0 (TID 47) in 3775585 ms on bigaperf137.svl.ibm.com (18/200) {noformat} Notice that time durations between tasks are unusually long: 2~5 minutes. When looking at the Linux 'perf' tool, two top CPU consumers are: 86.48%java [unknown] 12.41%libjvm.so Using the Java hotspot profiling tools, I am able to show what hotspot methods are (top 5): {noformat} org.apache.spark.storage.DiskBlockObjectWriter.updateBytesWritten() 46.845276 9,654,179 ms (46.8%)9,654,179 ms9,654,179 ms 9,654,179 ms org.apache.spark.unsafe.Platform.copyMemory() 18.631157 3,848,442 ms (18.6%)3,848,442 ms3,848,442 ms3,848,442 ms org.apache.spark.util.collection.CompactBuffer.$plus$eq() 6.8570185 1,418,411 ms (6.9%) 1,418,411 ms1,517,960 ms1,517,960 ms org.apache.spark.sql.execution.UnsafeRowSerializerInstance$$anon$2.writeValue() 4.6126328 955,495 ms (4.6%) 955,495 ms 2,153,910 ms 2,153,910 ms org.apache.spark.shuffle.sort.BypassMergeSortShuffleWriter.write() 4.581077949,930 ms (4.6%) 949,930 ms 19,967,510 ms 19,967,510 ms {noformat} So as you can see, the test has been running for 1.5 hours...with 46% CPU spent in the org.apache.spark.storage.DiskBlockObjectWriter.updateBytesWritten() method. The stacks for top two are: {noformat} Marshalling I java/io/DataOutputStream.writeInt() line 197 org.apache.spark.sql I org/apache/spark/sql/execution/UnsafeRowSerializerInstance$$anon$2.writeValue() line 60 org.apache.spark.storage I org/apache/spark/storage/DiskBlockObjectWriter.write() line 185 org.apache.spark.shuffle I org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.write() line 150 org.apache.spark.scheduler I org/apache/spark/scheduler/ShuffleMapTask.runTask() line 78 I org/apache/spark/scheduler/ShuffleMapTask.runTask() line 46 I org/apache/spark/scheduler/Task.run() line 82 org.apache.spark.executor I org/apache/spark/executor/Executor$TaskRunner.run() line 231 Dispatching Overhead, Standard Library Worker Dispatching I java/util/concurrent/ThreadPoolExecutor.runWorker() line 1142 I java/util/concurrent/ThreadPoolExecutor$Worker.run() line 617 I java/lang/Thread.run() line 745 {noformat} and {noformat} org.apache.spark.unsafe I
[jira] [Created] (SPARK-15048) when running Thriftserver with yarn on a secure cluster it will pass the wrong keytab location.
Trystan Leftwich created SPARK-15048: Summary: when running Thriftserver with yarn on a secure cluster it will pass the wrong keytab location. Key: SPARK-15048 URL: https://issues.apache.org/jira/browse/SPARK-15048 Project: Spark Issue Type: Bug Affects Versions: 2.0.0 Reporter: Trystan Leftwich when running hive-thriftserver with yarn on a secure cluster it will pass the wrong keytab location. {code} 16/05/01 19:33:52 INFO hive.HiveUtils: Initializing HiveMetastoreConnection version 1.2.1 using Spark classes. Exception in thread "main" org.apache.spark.SparkException: Keytab file: test.keytab-e3754e07-c798-4e6a-8745-c5f9d3483507 specified in spark.yarn.keytab does not exist at org.apache.spark.sql.hive.client.HiveClientImpl.(HiveClientImpl.scala:111) at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method) at sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62) at sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45) at java.lang.reflect.Constructor.newInstance(Constructor.java:422) at org.apache.spark.sql.hive.client.IsolatedClientLoader.createClient(IsolatedClientLoader.scala:258) at org.apache.spark.sql.hive.HiveUtils$.newClientForMetadata(HiveUtils.scala:364) at org.apache.spark.sql.hive.HiveUtils$.newClientForMetadata(HiveUtils.scala:268) at org.apache.spark.sql.hive.HiveSharedState.metadataHive$lzycompute(HiveSharedState.scala:39) at org.apache.spark.sql.hive.HiveSharedState.metadataHive(HiveSharedState.scala:38) at org.apache.spark.sql.hive.HiveSessionState.metadataHive$lzycompute(HiveSessionState.scala:45) at org.apache.spark.sql.hive.HiveSessionState.metadataHive(HiveSessionState.scala:45) at org.apache.spark.sql.hive.thriftserver.SparkSQLEnv$.init(SparkSQLEnv.scala:60) at org.apache.spark.sql.hive.thriftserver.HiveThriftServer2$.main(HiveThriftServer2.scala:81) at org.apache.spark.sql.hive.thriftserver.HiveThriftServer2.main(HiveThriftServer2.scala) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:497) at org.apache.spark.deploy.SparkSubmit$.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:726) at org.apache.spark.deploy.SparkSubmit$.doRunMain$1(SparkSubmit.scala:183) at org.apache.spark.deploy.SparkSubmit$.submit(SparkSubmit.scala:208) at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:122) at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala) 16/05/01 19:33:52 INFO spark.SparkContext: Invoking stop() from shutdown hook {code} Note: You will need the patch from SPARK-15046 before you can encounter this bug. It looks like this specific commit caused this issue, https://github.com/apache/spark/commit/8301fadd8d269da11e72870b7a889596e3337839#diff-6fd847124f8eae45ba2de1cf7d6296feL93 Re-adding that one line fixes the bug. Its possible to "Reset" the config before Hive needs it: i.e Adding code to similar to below to the following location: https://github.com/apache/spark/blob/master/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala#L57 {code} diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala index 665a44e..0e32b87 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala @@ -55,6 +55,12 @@ private[hive] object SparkSQLEnv extends Logging { maybeKryoReferenceTracking.getOrElse("false")) sparkContext = new SparkContext(sparkConf) + if (sparkConf.contains("spark.yarn.principal")) { + sparkContext.conf.set("spark.yarn.principal", sparkConf.get("spark.yarn.principal")) + } + if (sparkConf.contains("spark.yarn.keytab")) { + sparkContext.conf.set("spark.yarn.keytab", sparkConf.get("spark.yarn.keytab")) + } sqlContext = SparkSession.withHiveSupport(sparkContext).wrapped val sessionState = sqlContext.sessionState.asInstanceOf[HiveSessionState] sessionState.metadataHive.setOut(new PrintStream(System.out, true, "UTF-8")) {code} -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail:
[jira] [Created] (SPARK-15046) When running hive-thriftserver with yarn on a secure cluster the workers fail with java.lang.NumberFormatException
Trystan Leftwich created SPARK-15046: Summary: When running hive-thriftserver with yarn on a secure cluster the workers fail with java.lang.NumberFormatException Key: SPARK-15046 URL: https://issues.apache.org/jira/browse/SPARK-15046 Project: Spark Issue Type: Bug Affects Versions: 2.0.0 Reporter: Trystan Leftwich When running hive-thriftserver with yarn on a secure cluster (spark.yarn.principal and spark.yarn.keytab are set) the workers fail with the following error. {code} 16/04/30 22:40:50 ERROR yarn.ApplicationMaster: Uncaught exception: java.lang.NumberFormatException: For input string: "86400079ms" at java.lang.NumberFormatException.forInputString(NumberFormatException.java:65) at java.lang.Long.parseLong(Long.java:441) at java.lang.Long.parseLong(Long.java:483) at scala.collection.immutable.StringLike$class.toLong(StringLike.scala:276) at scala.collection.immutable.StringOps.toLong(StringOps.scala:29) at org.apache.spark.SparkConf$$anonfun$getLong$2.apply(SparkConf.scala:380) at org.apache.spark.SparkConf$$anonfun$getLong$2.apply(SparkConf.scala:380) at scala.Option.map(Option.scala:146) at org.apache.spark.SparkConf.getLong(SparkConf.scala:380) at org.apache.spark.deploy.SparkHadoopUtil.getTimeFromNowToRenewal(SparkHadoopUtil.scala:289) at org.apache.spark.deploy.yarn.AMDelegationTokenRenewer.org$apache$spark$deploy$yarn$AMDelegationTokenRenewer$$scheduleRenewal$1(AMDelegationTokenRenewer.scala:89) at org.apache.spark.deploy.yarn.AMDelegationTokenRenewer.scheduleLoginFromKeytab(AMDelegationTokenRenewer.scala:121) at org.apache.spark.deploy.yarn.ApplicationMaster$$anonfun$run$3.apply(ApplicationMaster.scala:243) at org.apache.spark.deploy.yarn.ApplicationMaster$$anonfun$run$3.apply(ApplicationMaster.scala:243) at scala.Option.foreach(Option.scala:257) at org.apache.spark.deploy.yarn.ApplicationMaster.run(ApplicationMaster.scala:243) at org.apache.spark.deploy.yarn.ApplicationMaster$$anonfun$main$1.apply$mcV$sp(ApplicationMaster.scala:723) at org.apache.spark.deploy.SparkHadoopUtil$$anon$1.run(SparkHadoopUtil.scala:67) at org.apache.spark.deploy.SparkHadoopUtil$$anon$1.run(SparkHadoopUtil.scala:66) 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.spark.deploy.SparkHadoopUtil.runAsSparkUser(SparkHadoopUtil.scala:66) at org.apache.spark.deploy.yarn.ApplicationMaster$.main(ApplicationMaster.scala:721) at org.apache.spark.deploy.yarn.ExecutorLauncher$.main(ApplicationMaster.scala:748) at org.apache.spark.deploy.yarn.ExecutorLauncher.main(ApplicationMaster.scala) {code} -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Closed] (SPARK-6043) Error when trying to rename table with alter table after using INSERT OVERWITE to populate the table
[ https://issues.apache.org/jira/browse/SPARK-6043?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Trystan Leftwich closed SPARK-6043. --- Resolution: Fixed Fix Version/s: 1.3.0 > Error when trying to rename table with alter table after using INSERT > OVERWITE to populate the table > > > Key: SPARK-6043 > URL: https://issues.apache.org/jira/browse/SPARK-6043 > Project: Spark > Issue Type: Bug > Components: SQL >Affects Versions: 1.2.1 >Reporter: Trystan Leftwich >Priority: Minor > Fix For: 1.3.0 > > > If you populate a table using INSERT OVERWRITE and then try to rename the > table using alter table it fails with: > {noformat} > Error: org.apache.spark.sql.execution.QueryExecutionException: FAILED: > Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. > Unable to alter table. (state=,code=0) > {noformat} > Using the following SQL statement creates the error: > {code:sql} > CREATE TABLE `tmp_table` (salesamount_c1 DOUBLE); > INSERT OVERWRITE table tmp_table SELECT >MIN(sales_customer.salesamount) salesamount_c1 > FROM > ( > SELECT > SUM(sales.salesamount) salesamount > FROM > internalsales sales > ) sales_customer; > ALTER TABLE tmp_table RENAME to not_tmp; > {code} > But if you change the 'OVERWRITE' to be 'INTO' the SQL statement works. > This is happening on our CDH5.3 cluster with multiple workers, If we use the > CDH5.3 Quickstart VM the SQL does not produce an error. Both cases were spark > 1.2.1 built for hadoop2.4+ -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Updated] (SPARK-6043) Error when trying to rename table with alter table after using INSERT OVERWITE to populate the table
[ https://issues.apache.org/jira/browse/SPARK-6043?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Trystan Leftwich updated SPARK-6043: Fix Version/s: (was: 1.3.0) 1.5.1 > Error when trying to rename table with alter table after using INSERT > OVERWITE to populate the table > > > Key: SPARK-6043 > URL: https://issues.apache.org/jira/browse/SPARK-6043 > Project: Spark > Issue Type: Bug > Components: SQL >Affects Versions: 1.2.1 >Reporter: Trystan Leftwich >Priority: Minor > Fix For: 1.5.1 > > > If you populate a table using INSERT OVERWRITE and then try to rename the > table using alter table it fails with: > {noformat} > Error: org.apache.spark.sql.execution.QueryExecutionException: FAILED: > Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. > Unable to alter table. (state=,code=0) > {noformat} > Using the following SQL statement creates the error: > {code:sql} > CREATE TABLE `tmp_table` (salesamount_c1 DOUBLE); > INSERT OVERWRITE table tmp_table SELECT >MIN(sales_customer.salesamount) salesamount_c1 > FROM > ( > SELECT > SUM(sales.salesamount) salesamount > FROM > internalsales sales > ) sales_customer; > ALTER TABLE tmp_table RENAME to not_tmp; > {code} > But if you change the 'OVERWRITE' to be 'INTO' the SQL statement works. > This is happening on our CDH5.3 cluster with multiple workers, If we use the > CDH5.3 Quickstart VM the SQL does not produce an error. Both cases were spark > 1.2.1 built for hadoop2.4+ -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Created] (SPARK-14539) Fetching delegation tokens in Hive-Thriftserver fails when hive.server2.enable.doAs = True
Trystan Leftwich created SPARK-14539: Summary: Fetching delegation tokens in Hive-Thriftserver fails when hive.server2.enable.doAs = True Key: SPARK-14539 URL: https://issues.apache.org/jira/browse/SPARK-14539 Project: Spark Issue Type: Bug Affects Versions: 1.6.1, 1.6.0 Reporter: Trystan Leftwich Similar to https://issues.apache.org/jira/browse/SPARK-13478 When you are running Hive Thriftserver and have hive.server2.enable.doAs = True you will get {code} javax.security.sasl.SaslException: GSS initiate failed [Caused by GSSException: No valid credentials provided (Mechanism level: Failed to find any Kerberos tgt)] ... {code} -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Commented] (SPARK-12868) ADD JAR via sparkSQL JDBC will fail when using a HDFS URL
[ https://issues.apache.org/jira/browse/SPARK-12868?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15128445#comment-15128445 ] Trystan Leftwich commented on SPARK-12868: -- Added: https://github.com/apache/spark/pull/11026 specifically for branch 1.6 and updated: https://github.com/apache/spark/pull/10797 to be up to date with master. > ADD JAR via sparkSQL JDBC will fail when using a HDFS URL > - > > Key: SPARK-12868 > URL: https://issues.apache.org/jira/browse/SPARK-12868 > Project: Spark > Issue Type: Bug > Components: SQL >Affects Versions: 1.6.0 >Reporter: Trystan Leftwich > Fix For: 1.6.1 > > > When trying to add a jar with a HDFS URI, i.E > {code:sql} > ADD JAR hdfs:///tmp/foo.jar > {code} > Via the spark sql JDBC interface it will fail with: > {code:sql} > java.net.MalformedURLException: unknown protocol: hdfs > at java.net.URL.(URL.java:593) > at java.net.URL.(URL.java:483) > at java.net.URL.(URL.java:432) > at java.net.URI.toURL(URI.java:1089) > at > org.apache.spark.sql.hive.client.ClientWrapper.addJar(ClientWrapper.scala:578) > at org.apache.spark.sql.hive.HiveContext.addJar(HiveContext.scala:652) > at org.apache.spark.sql.hive.execution.AddJar.run(commands.scala:89) > at > org.apache.spark.sql.execution.ExecutedCommand.sideEffectResult$lzycompute(commands.scala:58) > at > org.apache.spark.sql.execution.ExecutedCommand.sideEffectResult(commands.scala:56) > at > org.apache.spark.sql.execution.ExecutedCommand.doExecute(commands.scala:70) > at > org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$5.apply(SparkPlan.scala:132) > at > org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$5.apply(SparkPlan.scala:130) > at > org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:150) > at > org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:130) > at > org.apache.spark.sql.execution.QueryExecution.toRdd$lzycompute(QueryExecution.scala:55) > at > org.apache.spark.sql.execution.QueryExecution.toRdd(QueryExecution.scala:55) > at org.apache.spark.sql.DataFrame.(DataFrame.scala:145) > at org.apache.spark.sql.DataFrame.(DataFrame.scala:130) > at org.apache.spark.sql.DataFrame$.apply(DataFrame.scala:52) > at org.apache.spark.sql.SQLContext.sql(SQLContext.scala:817) > at > org.apache.spark.sql.hive.thriftserver.SparkExecuteStatementOperation.org$apache$spark$sql$hive$thriftserver$SparkExecuteStatementOperation$$execute(SparkExecuteStatementOperation.scala:211) > at > org.apache.spark.sql.hive.thriftserver.SparkExecuteStatementOperation$$anon$1$$anon$2.run(SparkExecuteStatementOperation.scala:154) > at > org.apache.spark.sql.hive.thriftserver.SparkExecuteStatementOperation$$anon$1$$anon$2.run(SparkExecuteStatementOperation.scala:151) > 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:1628) > at > org.apache.spark.sql.hive.thriftserver.SparkExecuteStatementOperation$$anon$1.run(SparkExecuteStatementOperation.scala:164) > at > java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) > at java.util.concurrent.FutureTask.run(FutureTask.java:266) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) > at java.lang.Thread.run(Thread.java:745) > {code} -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Updated] (SPARK-12868) ADD JAR via sparkSQL JDBC will fail when using a HDFS URL
[ https://issues.apache.org/jira/browse/SPARK-12868?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Trystan Leftwich updated SPARK-12868: - Fix Version/s: 1.6.1 Description: When trying to add a jar with a HDFS URI, i.E {code:sql} ADD JAR hdfs:///tmp/foo.jar {code} Via the spark sql JDBC interface it will fail with: {code:sql} java.net.MalformedURLException: unknown protocol: hdfs at java.net.URL.(URL.java:593) at java.net.URL.(URL.java:483) at java.net.URL.(URL.java:432) at java.net.URI.toURL(URI.java:1089) at org.apache.spark.sql.hive.client.ClientWrapper.addJar(ClientWrapper.scala:578) at org.apache.spark.sql.hive.HiveContext.addJar(HiveContext.scala:652) at org.apache.spark.sql.hive.execution.AddJar.run(commands.scala:89) at org.apache.spark.sql.execution.ExecutedCommand.sideEffectResult$lzycompute(commands.scala:58) at org.apache.spark.sql.execution.ExecutedCommand.sideEffectResult(commands.scala:56) at org.apache.spark.sql.execution.ExecutedCommand.doExecute(commands.scala:70) at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$5.apply(SparkPlan.scala:132) at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$5.apply(SparkPlan.scala:130) at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:150) at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:130) at org.apache.spark.sql.execution.QueryExecution.toRdd$lzycompute(QueryExecution.scala:55) at org.apache.spark.sql.execution.QueryExecution.toRdd(QueryExecution.scala:55) at org.apache.spark.sql.DataFrame.(DataFrame.scala:145) at org.apache.spark.sql.DataFrame.(DataFrame.scala:130) at org.apache.spark.sql.DataFrame$.apply(DataFrame.scala:52) at org.apache.spark.sql.SQLContext.sql(SQLContext.scala:817) at org.apache.spark.sql.hive.thriftserver.SparkExecuteStatementOperation.org$apache$spark$sql$hive$thriftserver$SparkExecuteStatementOperation$$execute(SparkExecuteStatementOperation.scala:211) at org.apache.spark.sql.hive.thriftserver.SparkExecuteStatementOperation$$anon$1$$anon$2.run(SparkExecuteStatementOperation.scala:154) at org.apache.spark.sql.hive.thriftserver.SparkExecuteStatementOperation$$anon$1$$anon$2.run(SparkExecuteStatementOperation.scala:151) 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:1628) at org.apache.spark.sql.hive.thriftserver.SparkExecuteStatementOperation$$anon$1.run(SparkExecuteStatementOperation.scala:164) at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) at java.util.concurrent.FutureTask.run(FutureTask.java:266) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) at java.lang.Thread.run(Thread.java:745) {code} was: When trying to add a jar with a HDFS URI, i.E ``` ADD JAR hdfs:///tmp/foo.jar ``` Via the spark sql JDBC interface it will fail with: ``` java.net.MalformedURLException: unknown protocol: hdfs at java.net.URL.(URL.java:593) at java.net.URL.(URL.java:483) at java.net.URL.(URL.java:432) at java.net.URI.toURL(URI.java:1089) at org.apache.spark.sql.hive.client.ClientWrapper.addJar(ClientWrapper.scala:578) at org.apache.spark.sql.hive.HiveContext.addJar(HiveContext.scala:652) at org.apache.spark.sql.hive.execution.AddJar.run(commands.scala:89) at org.apache.spark.sql.execution.ExecutedCommand.sideEffectResult$lzycompute(commands.scala:58) at org.apache.spark.sql.execution.ExecutedCommand.sideEffectResult(commands.scala:56) at org.apache.spark.sql.execution.ExecutedCommand.doExecute(commands.scala:70) at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$5.apply(SparkPlan.scala:132) at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$5.apply(SparkPlan.scala:130) at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:150) at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:130) at org.apache.spark.sql.execution.QueryExecution.toRdd$lzycompute(QueryExecution.scala:55) at org.apache.spark.sql.execution.QueryExecution.toRdd(QueryExecution.scala:55) at org.apache.spark.sql.DataFrame.(DataFrame.scala:145) at org.apache.spark.sql.DataFrame.(DataFrame.scala:130) at org.apache.spark.sql.DataFrame$.apply(DataFrame.scala:52) at org.apache.spark.sql.SQLContext.sql(SQLContext.scala:817) at
[jira] [Commented] (SPARK-13087) Grouping by a complex expression may lead to incorrect AttributeReferences in aggregations
[ https://issues.apache.org/jira/browse/SPARK-13087?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15126980#comment-15126980 ] Trystan Leftwich commented on SPARK-13087: -- I re-built 1.6.1 with commit ddb9633043e82fb2a34c7e0e29b487f635c3c744 this morning and I'm seeing a similar error to above. Similarly we use a custom UDF and the above commit fixes the issue. {code:sql} SELECT concat(t_4.firstname," ",t_4.lastname) customer_name, agg_cust(t_3.customercountestimate1_c2) ctd_customercountestimate1_ok FROM as.sales t_3 JOIN as.customer t_4 ON t_3.key_c1 = t_4.customerkey GROUP BY concat(t_4.firstname," ",t_4.lastname) {code} {code} org.apache.spark.sql.catalyst.errors.package$TreeNodeException: Binding attribute, tree: concat(firstname#329, ,lastname#330)#339 at org.apache.spark.sql.catalyst.errors.package$.attachTree(package.scala:49) at org.apache.spark.sql.catalyst.expressions.BindReferences$$anonfun$bindReference$1.applyOrElse(BoundAttribute.scala:86) at org.apache.spark.sql.catalyst.expressions.BindReferences$$anonfun$bindReference$1.applyOrElse(BoundAttribute.scala:85) at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$3.apply(TreeNode.scala:259) at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$3.apply(TreeNode.scala:259) at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:69) at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:258) at org.apache.spark.sql.catalyst.trees.TreeNode.transform(TreeNode.scala:249) at org.apache.spark.sql.catalyst.expressions.BindReferences$.bindReference(BoundAttribute.scala:85) at org.apache.spark.sql.catalyst.expressions.codegen.GenerateMutableProjection$$anonfun$bind$1.apply(GenerateMutableProjection.scala:39) at org.apache.spark.sql.catalyst.expressions.codegen.GenerateMutableProjection$$anonfun$bind$1.apply(GenerateMutableProjection.scala:39) at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244) at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244) at scala.collection.immutable.List.foreach(List.scala:318) at scala.collection.TraversableLike$class.map(TraversableLike.scala:244) at scala.collection.AbstractTraversable.map(Traversable.scala:105) at org.apache.spark.sql.catalyst.expressions.codegen.GenerateMutableProjection$.bind(GenerateMutableProjection.scala:39) at org.apache.spark.sql.catalyst.expressions.codegen.GenerateMutableProjection$.bind(GenerateMutableProjection.scala:33) at org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator.generate(CodeGenerator.scala:585) at org.apache.spark.sql.execution.SparkPlan.newMutableProjection(SparkPlan.scala:227) at org.apache.spark.sql.execution.Exchange.org$apache$spark$sql$execution$Exchange$$getPartitionKeyExtractor$1(Exchange.scala:197) at org.apache.spark.sql.execution.Exchange$$anonfun$3.apply(Exchange.scala:209) at org.apache.spark.sql.execution.Exchange$$anonfun$3.apply(Exchange.scala:208) at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$21.apply(RDD.scala:728) at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$21.apply(RDD.scala:728) at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38) at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306) at org.apache.spark.rdd.RDD.iterator(RDD.scala:270) at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:73) at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:41) at org.apache.spark.scheduler.Task.run(Task.scala:89) at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:214) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) at java.lang.Thread.run(Thread.java:745) Caused by: java.lang.RuntimeException: Couldn't find concat(firstname#329, ,lastname#330)#339 in [firstname#329,lastname#330,customercountestimate1_c2#326] at scala.sys.package$.error(package.scala:27) at org.apache.spark.sql.catalyst.expressions.BindReferences$$anonfun$bindReference$1$$anonfun$applyOrElse$1.apply(BoundAttribute.scala:92) at org.apache.spark.sql.catalyst.expressions.BindReferences$$anonfun$bindReference$1$$anonfun$applyOrElse$1.apply(BoundAttribute.scala:86) at org.apache.spark.sql.catalyst.errors.package$.attachTree(package.scala:48) ... 34 more {code} > Grouping by a complex expression may lead to incorrect AttributeReferences in > aggregations >
[jira] [Created] (SPARK-12868) ADD JAR via sparkSQL JDBC will fail when using a HDFS URL
Trystan Leftwich created SPARK-12868: Summary: ADD JAR via sparkSQL JDBC will fail when using a HDFS URL Key: SPARK-12868 URL: https://issues.apache.org/jira/browse/SPARK-12868 Project: Spark Issue Type: Bug Affects Versions: 1.6.0 Reporter: Trystan Leftwich When trying to add a jar with a HDFS URI, i.E ``` ADD JAR hdfs:///tmp/foo.jar ``` Via the spark sql JDBC interface it will fail with: ``` java.net.MalformedURLException: unknown protocol: hdfs at java.net.URL.(URL.java:593) at java.net.URL.(URL.java:483) at java.net.URL.(URL.java:432) at java.net.URI.toURL(URI.java:1089) at org.apache.spark.sql.hive.client.ClientWrapper.addJar(ClientWrapper.scala:578) at org.apache.spark.sql.hive.HiveContext.addJar(HiveContext.scala:652) at org.apache.spark.sql.hive.execution.AddJar.run(commands.scala:89) at org.apache.spark.sql.execution.ExecutedCommand.sideEffectResult$lzycompute(commands.scala:58) at org.apache.spark.sql.execution.ExecutedCommand.sideEffectResult(commands.scala:56) at org.apache.spark.sql.execution.ExecutedCommand.doExecute(commands.scala:70) at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$5.apply(SparkPlan.scala:132) at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$5.apply(SparkPlan.scala:130) at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:150) at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:130) at org.apache.spark.sql.execution.QueryExecution.toRdd$lzycompute(QueryExecution.scala:55) at org.apache.spark.sql.execution.QueryExecution.toRdd(QueryExecution.scala:55) at org.apache.spark.sql.DataFrame.(DataFrame.scala:145) at org.apache.spark.sql.DataFrame.(DataFrame.scala:130) at org.apache.spark.sql.DataFrame$.apply(DataFrame.scala:52) at org.apache.spark.sql.SQLContext.sql(SQLContext.scala:817) at org.apache.spark.sql.hive.thriftserver.SparkExecuteStatementOperation.org$apache$spark$sql$hive$thriftserver$SparkExecuteStatementOperation$$execute(SparkExecuteStatementOperation.scala:211) at org.apache.spark.sql.hive.thriftserver.SparkExecuteStatementOperation$$anon$1$$anon$2.run(SparkExecuteStatementOperation.scala:154) at org.apache.spark.sql.hive.thriftserver.SparkExecuteStatementOperation$$anon$1$$anon$2.run(SparkExecuteStatementOperation.scala:151) 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:1628) at org.apache.spark.sql.hive.thriftserver.SparkExecuteStatementOperation$$anon$1.run(SparkExecuteStatementOperation.scala:164) at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) at java.util.concurrent.FutureTask.run(FutureTask.java:266) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) at java.lang.Thread.run(Thread.java:745) ``` -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Created] (SPARK-11248) Spark hivethriftserver is using the wrong user to while getting HDFS permissions
Trystan Leftwich created SPARK-11248: Summary: Spark hivethriftserver is using the wrong user to while getting HDFS permissions Key: SPARK-11248 URL: https://issues.apache.org/jira/browse/SPARK-11248 Project: Spark Issue Type: Bug Affects Versions: 1.5.1, 1.5.0 Reporter: Trystan Leftwich While running spark as a hivethrift-server via Yarn Spark will use the user running the Hivethrift server rather than the user connecting via JDBC to check HDFS perms. i.e. In HDFS the perms are rwx-- 3 testuser testuser /user/testuser/table/testtable And i connect via beeline as user testuser beeline -u 'jdbc:hive2://localhost:10511' -n 'testuser' -p '' If i try to hit that table select count(*) from test_table; I get the following error Error: org.apache.hadoop.hive.ql.metadata.HiveException: Unable to fetch table test_table. java.security.AccessControlException: Permission denied: user=hive, access=READ, inode="/user/testuser/table/testtable":testuser:testuser:drwxr-x--x at org.apache.hadoop.hdfs.server.namenode.FSPermissionChecker.checkFsPermission(FSPermissionChecker.java:271) at org.apache.hadoop.hdfs.server.namenode.FSPermissionChecker.check(FSPermissionChecker.java:257) at org.apache.hadoop.hdfs.server.namenode.FSPermissionChecker.checkPermission(FSPermissionChecker.java:185) at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.checkPermission(FSNamesystem.java:6795) at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.checkPermission(FSNamesystem.java:6777) at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.checkPathAccess(FSNamesystem.java:6702) at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.checkAccess(FSNamesystem.java:9529) at org.apache.hadoop.hdfs.server.namenode.NameNodeRpcServer.checkAccess(NameNodeRpcServer.java:1516) at org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolServerSideTranslatorPB.checkAccess(ClientNamenodeProtocolServerSideTranslatorPB.java:1433) at org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos$ClientNamenodeProtocol$2.callBlockingMethod(ClientNamenodeProtocolProtos.java) at org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:619) at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:962) at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2039) at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2035) 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.ipc.Server$Handler.run(Server.java:2033) (state=,code=0) I have the following in set in hive-site.xml so it should be using the correct user. hive.server2.enable.doAs true hive.metastore.execute.setugi true This works correctly in hive. -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Created] (SPARK-11052) Spaces in the build dir causes failures in the build/mvn script
Trystan Leftwich created SPARK-11052: Summary: Spaces in the build dir causes failures in the build/mvn script Key: SPARK-11052 URL: https://issues.apache.org/jira/browse/SPARK-11052 Project: Spark Issue Type: Bug Components: Build Affects Versions: 1.5.1, 1.5.0 Reporter: Trystan Leftwich If you are running make-distribution in a path that contains a space in it the build/mvn script will fail: {code} mkdir /tmp/test\ spaces cd /tmp/test\ spaces git clone https://github.com/apache/spark.git cd spark ./make-distribution.sh --name spark-1.5-test4 --tgz -Pyarn -Phive-thriftserver -Phive {code} You will get the following errors {code} /tmp/test spaces/spark/build/mvn: line 107: cd: /../lib: No such file or directory usage: dirname path /tmp/test spaces/spark/build/mvn: line 108: cd: /../lib: No such file or directory /tmp/test spaces/spark/build/mvn: line 138: /tmp/test: No such file or directory /tmp/test spaces/spark/build/mvn: line 140: /tmp/test: No such file or directory {code} -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Created] (SPARK-6196) Add MAPR 4.0.2 support to the build
Trystan Leftwich created SPARK-6196: --- Summary: Add MAPR 4.0.2 support to the build Key: SPARK-6196 URL: https://issues.apache.org/jira/browse/SPARK-6196 Project: Spark Issue Type: Improvement Reporter: Trystan Leftwich Mapr 4.0.2 upgraded to use hadoop 2.5.1 and the current mapr build doesn't support building for 4.0.2 http://doc.mapr.com/display/RelNotes/Version+4.0.2+Release+Notes -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Created] (SPARK-6144) When in cluster mode using ADD JAR with a hdfs:// sourced jar will fail
Trystan Leftwich created SPARK-6144: --- Summary: When in cluster mode using ADD JAR with a hdfs:// sourced jar will fail Key: SPARK-6144 URL: https://issues.apache.org/jira/browse/SPARK-6144 Project: Spark Issue Type: Bug Affects Versions: 1.3.0 Reporter: Trystan Leftwich While in cluster mode if you use ADD JAR with a HDFS sourced jar it will fail trying to source that jar on the worker nodes with the following error: {code} 15/03/03 04:56:50 ERROR executor.Executor: Exception in task 0.0 in stage 0.0 (TID 0) java.io.FileNotFoundException: /yarn/nm/usercache/vagrant/appcache/application_1425166832391_0027/-19222735701425358546704_cache (No such file or directory) at java.io.FileInputStream.open(Native Method) at java.io.FileInputStream.init(FileInputStream.java:146) {code} PR -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Updated] (SPARK-6144) When in cluster mode using ADD JAR with a hdfs:// sourced jar will fail
[ https://issues.apache.org/jira/browse/SPARK-6144?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Trystan Leftwich updated SPARK-6144: Description: While in cluster mode if you use ADD JAR with a HDFS sourced jar it will fail trying to source that jar on the worker nodes with the following error: {code} 15/03/03 04:56:50 ERROR executor.Executor: Exception in task 0.0 in stage 0.0 (TID 0) java.io.FileNotFoundException: /yarn/nm/usercache/vagrant/appcache/application_1425166832391_0027/-19222735701425358546704_cache (No such file or directory) at java.io.FileInputStream.open(Native Method) at java.io.FileInputStream.init(FileInputStream.java:146) {code} PR https://github.com/apache/spark/pull/4880 was: While in cluster mode if you use ADD JAR with a HDFS sourced jar it will fail trying to source that jar on the worker nodes with the following error: {code} 15/03/03 04:56:50 ERROR executor.Executor: Exception in task 0.0 in stage 0.0 (TID 0) java.io.FileNotFoundException: /yarn/nm/usercache/vagrant/appcache/application_1425166832391_0027/-19222735701425358546704_cache (No such file or directory) at java.io.FileInputStream.open(Native Method) at java.io.FileInputStream.init(FileInputStream.java:146) {code} PR When in cluster mode using ADD JAR with a hdfs:// sourced jar will fail --- Key: SPARK-6144 URL: https://issues.apache.org/jira/browse/SPARK-6144 Project: Spark Issue Type: Bug Affects Versions: 1.3.0 Reporter: Trystan Leftwich While in cluster mode if you use ADD JAR with a HDFS sourced jar it will fail trying to source that jar on the worker nodes with the following error: {code} 15/03/03 04:56:50 ERROR executor.Executor: Exception in task 0.0 in stage 0.0 (TID 0) java.io.FileNotFoundException: /yarn/nm/usercache/vagrant/appcache/application_1425166832391_0027/-19222735701425358546704_cache (No such file or directory) at java.io.FileInputStream.open(Native Method) at java.io.FileInputStream.init(FileInputStream.java:146) {code} PR https://github.com/apache/spark/pull/4880 -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Updated] (SPARK-6043) Error when trying to rename table with alter table after using INSERT OVERWITE to populate the table
[ https://issues.apache.org/jira/browse/SPARK-6043?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Trystan Leftwich updated SPARK-6043: Description: If you populate a table using INSERT OVERWRITE and then try to rename the table using alter table it fails with: {noformat} Error: org.apache.spark.sql.execution.QueryExecutionException: FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. Unable to alter table. (state=,code=0) {noformat} Using the following SQL statement creates the error: {code:sql} CREATE TABLE `tmp_table` (salesamount_c1 DOUBLE); INSERT OVERWRITE table tmp_table SELECT MIN(sales_customer.salesamount) salesamount_c1 FROM ( SELECT SUM(sales.salesamount) salesamount FROM internalsales sales ) sales_customer; ALTER TABLE tmp_table RENAME to not_tmp; {code} But if you change the 'OVERWRITE' to be 'INTO' the SQL statement works. This is happening on our CDH5.3 cluster with multiple workers, If we use the CDH5.3 Quickstart VM the SQL does not produce an error. Both cases were spark 1.2.1 built for hadoop2.4+ was: If you populate a table using INSERT OVERWRITE and then try to rename the table using alter table it fails with: {noformat} Error: org.apache.spark.sql.execution.QueryExecutionException: FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. Unable to alter table. (state=,code=0) {noformat} Using the following SQL statement creates the error: {code:sql} CREATE TABLE `tmp_table` (salesamount_c1 DOUBLE); INSERT OVERWRITE table tmp_table SELECT MIN(sales_customer.salesamount) salesamount_c1 FROM ( SELECT SUM(sales.salesamount) salesamount FROM factinternetsales sales ) sales_customer; ALTER TABLE tmp_table RENAME to not_tmp; {code} But if you change the 'OVERWRITE' to be 'INTO' the SQL statement works. This is happening on our CDH5.3 cluster with multiple workers, If we use the CDH5.3 Quickstart VM the SQL does not produce an error. Both cases were spark 1.2.1 built for hadoop2.4+ Error when trying to rename table with alter table after using INSERT OVERWITE to populate the table Key: SPARK-6043 URL: https://issues.apache.org/jira/browse/SPARK-6043 Project: Spark Issue Type: Bug Affects Versions: 1.2.1 Reporter: Trystan Leftwich Priority: Minor If you populate a table using INSERT OVERWRITE and then try to rename the table using alter table it fails with: {noformat} Error: org.apache.spark.sql.execution.QueryExecutionException: FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. Unable to alter table. (state=,code=0) {noformat} Using the following SQL statement creates the error: {code:sql} CREATE TABLE `tmp_table` (salesamount_c1 DOUBLE); INSERT OVERWRITE table tmp_table SELECT MIN(sales_customer.salesamount) salesamount_c1 FROM ( SELECT SUM(sales.salesamount) salesamount FROM internalsales sales ) sales_customer; ALTER TABLE tmp_table RENAME to not_tmp; {code} But if you change the 'OVERWRITE' to be 'INTO' the SQL statement works. This is happening on our CDH5.3 cluster with multiple workers, If we use the CDH5.3 Quickstart VM the SQL does not produce an error. Both cases were spark 1.2.1 built for hadoop2.4+ -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Created] (SPARK-6043) Error when trying to rename table with alter table after using INSERT OVERWITE to populate the table
Trystan Leftwich created SPARK-6043: --- Summary: Error when trying to rename table with alter table after using INSERT OVERWITE to populate the table Key: SPARK-6043 URL: https://issues.apache.org/jira/browse/SPARK-6043 Project: Spark Issue Type: Bug Affects Versions: 1.2.1 Reporter: Trystan Leftwich If you populate a table using INSERT OVERWRITE and then try to rename the table using alter table it fails with: {noformat} Error: org.apache.spark.sql.execution.QueryExecutionException: FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. Unable to alter table. (state=,code=0) {noformat} Using the following SQL statement creates the error: {code:sql} CREATE TABLE `tmp_table` (salesamount_c1 DOUBLE); INSERT OVERWRITE table tmp_table SELECT MIN(sales_customer.salesamount) salesamount_c1 FROM ( SELECT SUM(sales.salesamount) salesamount FROM factinternetsales sales ) sales_customer; ALTER TABLE tmp_table RENAME to not_tmp; {code} But if you change the 'OVERWRITE' to be 'INTO' the SQL statement works. This is happening on our CDH5.3 cluster with multiple workers, If we use the CDH5.3 Quickstart VM the SQL does not produce an error. Both cases were spark 1.2.1 built for hadoop2.4+ -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Updated] (SPARK-6043) Error when trying to rename table with alter table after using INSERT OVERWITE to populate the table
[ https://issues.apache.org/jira/browse/SPARK-6043?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Trystan Leftwich updated SPARK-6043: Priority: Minor (was: Major) Error when trying to rename table with alter table after using INSERT OVERWITE to populate the table Key: SPARK-6043 URL: https://issues.apache.org/jira/browse/SPARK-6043 Project: Spark Issue Type: Bug Affects Versions: 1.2.1 Reporter: Trystan Leftwich Priority: Minor If you populate a table using INSERT OVERWRITE and then try to rename the table using alter table it fails with: {noformat} Error: org.apache.spark.sql.execution.QueryExecutionException: FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. Unable to alter table. (state=,code=0) {noformat} Using the following SQL statement creates the error: {code:sql} CREATE TABLE `tmp_table` (salesamount_c1 DOUBLE); INSERT OVERWRITE table tmp_table SELECT MIN(sales_customer.salesamount) salesamount_c1 FROM ( SELECT SUM(sales.salesamount) salesamount FROM factinternetsales sales ) sales_customer; ALTER TABLE tmp_table RENAME to not_tmp; {code} But if you change the 'OVERWRITE' to be 'INTO' the SQL statement works. This is happening on our CDH5.3 cluster with multiple workers, If we use the CDH5.3 Quickstart VM the SQL does not produce an error. Both cases were spark 1.2.1 built for hadoop2.4+ -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org