Re: Problems with connecting Spark to Hive

2014-06-03 Thread Yin Huai
Hello Lars, Can you check the value of hive.security.authenticator.manager in hive-site.xml? I guess the value is org.apache.hadoop.hive.ql.security.ProxyUserAuthenticator. This class was introduced in hive 0.13, but Spark SQL is based on hive 0.12 right now. Can you change the value of

Re: jsonFile function in SQLContext does not work

2014-06-25 Thread Yin Huai
Hi Durin, I guess that blank lines caused the problem (like Aaron said). Right now, jsonFile does not skip faulty lines. Can you first use sc.textfile to load the file as RDD[String] and then use filter to filter out those blank lines (code snippet can be found below)? val sqlContext = new

Re: Spark SQL : Join throws exception

2014-07-01 Thread Yin Huai
Seems it is a bug. I have opened https://issues.apache.org/jira/browse/SPARK-2339 to track it. Thank you for reporting it. Yin On Tue, Jul 1, 2014 at 12:06 PM, Subacini B subac...@gmail.com wrote: Hi All, Running this join query sql(SELECT * FROM A_TABLE A JOIN B_TABLE B WHERE

Re: Spark SQL : Join throws exception

2014-07-07 Thread Yin Huai
Hi Subacini, Just want to follow up on this issue. SPARK-2339 has been merged into the master and 1.0 branch. Thanks, Yin On Tue, Jul 1, 2014 at 2:00 PM, Yin Huai huaiyin@gmail.com wrote: Seems it is a bug. I have opened https://issues.apache.org/jira/browse/SPARK-2339 to track

Re: spark1.0.1 catalyst transform filter not push down

2014-07-14 Thread Yin Huai
Hi, queryPlan.baseLogicalPlan is not the plan used to execution. Actually, the baseLogicalPlan of a SchemaRDD (queryPlan in your case) is just the parsed plan (the parsed plan will be analyzed, and then optimized. Finally, a physical plan will be created). The plan shows up after you execute val

Re: Spark Streaming Json file groupby function

2014-07-16 Thread Yin Huai
Hi Srinivas, Seems the query you used is val results =sqlContext.sql(select type from table1). However, table1 does not have a field called type. The schema of table1 is defined as the class definition of your case class Record (i.e. ID, name, score, and school are fields of your table1). Can you

Re: Spark 1.0.1 SQL on 160 G parquet file (snappy compressed, made by cloudera impala), 23 core and 60G mem / node, yarn-client mode, always failed

2014-07-19 Thread Yin Huai
Can you attach your code? Thanks, Yin On Sat, Jul 19, 2014 at 4:10 PM, chutium teng@gmail.com wrote: 160G parquet files (ca. 30 files, snappy compressed, made by cloudera impala) ca. 30 full table scan, took 3-5 columns out, then some normal scala operations like substring, groupby,

Re: spark1.0.1 spark sql error java.lang.NoClassDefFoundError: Could not initialize class $line11.$read$

2014-07-21 Thread Yin Huai
Hi Victor, Instead of importing sqlContext.createSchemaRDD, can you explicitly call sqlContext.createSchemaRDD(rdd) to create a SchemaRDD? For example, You have a case class Record. case class Record(data_date: String, mobile: String, create_time: String) Then, you create a RDD[Record] and

Re: Spark 1.0.1 SQL on 160 G parquet file (snappy compressed, made by cloudera impala), 23 core and 60G mem / node, yarn-client mode, always failed

2014-07-21 Thread Yin Huai
Instead of using union, can you try sqlContext.parquetFile(/user/ hive/warehouse/xxx_parquet.db).registerAsTable(parquetTable)? Then, var all = sql(select some_id, some_type, some_time from parquetTable).map(line = (line(0), (line(1).toString, line(2).toString.substring(0, 19 Thanks, Yin

Re: spark1.0.1 spark sql error java.lang.NoClassDefFoundError: Could not initialize class $line11.$read$

2014-07-22 Thread Yin Huai
On Tue, Jul 22, 2014 at 12:53 AM, Victor Sheng victorsheng...@gmail.com wrote: Hi, Yin Huai I test again with your snippet code. It works well in spark-1.0.1 Here is my code: val sqlContext = new org.apache.spark.sql.SQLContext(sc) case class Record(data_date: String, mobile

Re: spark1.0.1 spark sql error java.lang.NoClassDefFoundError: Could not initialize class $line11.$read$

2014-07-23 Thread Yin Huai
Yes, https://issues.apache.org/jira/browse/SPARK-2576 is used to track it. On Wed, Jul 23, 2014 at 9:11 AM, Nicholas Chammas nicholas.cham...@gmail.com wrote: Do we have a JIRA issue to track this? I think I've run into a similar issue. On Wed, Jul 23, 2014 at 1:12 AM, Yin Huai yh

Re: Simple record matching using Spark SQL

2014-07-24 Thread Yin Huai
Hi Sarath, I will try to reproduce the problem. Thanks, Yin On Wed, Jul 23, 2014 at 11:32 PM, Sarath Chandra sarathchandra.jos...@algofusiontech.com wrote: Hi Michael, Sorry for the delayed response. I'm using Spark 1.0.1 (pre-built version for hadoop 1). I'm running spark programs on

Re: Simple record matching using Spark SQL

2014-07-24 Thread Yin Huai
Hi Sarath, Have you tried the current branch 1.0? If not, can you give it a try and see if the problem can be resolved? Thanks, Yin On Thu, Jul 24, 2014 at 11:17 AM, Yin Huai yh...@databricks.com wrote: Hi Sarath, I will try to reproduce the problem. Thanks, Yin On Wed, Jul 23

Re: Inconsistent Spark SQL behavior when column names contain dots

2014-07-31 Thread Yin Huai
I have created https://issues.apache.org/jira/browse/SPARK-2775 to track it. On Thu, Jul 31, 2014 at 11:47 AM, Budde, Adam bu...@amazon.com wrote: I still see the same “Unresolved attributes” error when using hql + backticks. Here’s a code snippet that replicates this behavior: val

Re: pyspark inferSchema

2014-08-05 Thread Yin Huai
Yes, 2376 has been fixed in master. Can you give it a try? Also, for inferSchema, because Python is dynamically typed, I agree with Davies to provide a way to scan a subset (or entire) of the dataset to figure out the proper schema. We will take a look it. Thanks, Yin On Tue, Aug 5, 2014 at

Re: trouble with jsonRDD and jsonFile in pyspark

2014-08-05 Thread Yin Huai
I tried jsonRDD(...).printSchema() and it worked. Seems the problem is when we take the data back to the Python side, SchemaRDD#javaToPython failed on your cases. I have created https://issues.apache.org/jira/browse/SPARK-2875 to track it. Thanks, Yin On Tue, Aug 5, 2014 at 9:20 PM, Brad

Re: trouble with saveAsParquetFile

2014-08-07 Thread Yin Huai
Hi Brad, It is a bug. I have filed https://issues.apache.org/jira/browse/SPARK-2908 to track it. It will be fixed soon. Thanks, Yin On Thu, Aug 7, 2014 at 10:55 AM, Brad Miller bmill...@eecs.berkeley.edu wrote: Hi All, I'm having a bit of trouble with nested data structures in pyspark

Re: trouble with saveAsParquetFile

2014-08-07 Thread Yin Huai
will have a better story to handle NullType columns ( https://issues.apache.org/jira/browse/SPARK-2695). But, we still will not expose NullType to users. On Thu, Aug 7, 2014 at 1:41 PM, Brad Miller bmill...@eecs.berkeley.edu wrote: Thanks Yin! best, -Brad On Thu, Aug 7, 2014 at 1:39 PM, Yin

Re: trouble with saveAsParquetFile

2014-08-07 Thread Yin Huai
The PR is https://github.com/apache/spark/pull/1840. On Thu, Aug 7, 2014 at 1:48 PM, Yin Huai yh...@databricks.com wrote: Actually, the issue is if values of a field are always null (or this field is missing), we cannot figure out the data type. So, we use NullType (it is an internal data

Re: CDH5, HiveContext, Parquet

2014-08-10 Thread Yin Huai
If the link to PR/1819 is broken. Here is the one https://github.com/apache/spark/pull/1819. On Sun, Aug 10, 2014 at 5:56 PM, Eric Friedman eric.d.fried...@gmail.com wrote: Thanks Michael, I can try that too. I know you guys aren't in sales/marketing (thank G-d), but given all the hoopla

Re: Spark sql failed in yarn-cluster mode when connecting to non-default hive database

2014-08-11 Thread Yin Huai
Hi Jenny, How's your metastore configured for both Hive and Spark SQL? Which metastore mode are you using (based on https://cwiki.apache.org/confluence/display/Hive/AdminManual+MetastoreAdmin )? Thanks, Yin On Mon, Aug 11, 2014 at 6:15 PM, Jenny Zhao linlin200...@gmail.com wrote: you can

Re: Spark sql failed in yarn-cluster mode when connecting to non-default hive database

2014-08-12 Thread Yin Huai
.svl.ibm.com:8080/value /property property namehive.security.authorization.enabled/name valuetrue/value /property property namehive.security.authorization.createtable.owner.grants/name valueALL/value /property /configuration On Mon, Aug 11, 2014 at 4:29 PM, Yin Huai huaiyin

Re: Spark sql failed in yarn-cluster mode when connecting to non-default hive database

2014-08-13 Thread Yin Huai
not able to switch to a database other than the default one, for Yarn-client mode, it works fine. Thanks! Jenny On Tue, Aug 12, 2014 at 12:53 PM, Yin Huai huaiyin@gmail.com wrote: Hi Jenny, Have you copied hive-site.xml to spark/conf directory? If not, can you put it in conf/ and try

Re: SparkSQL Hive partitioning support

2014-08-13 Thread Yin Huai
Hi Silvio, You can insert into a static partition via SQL statement. Dynamic partitioning is not supported at the moment. Thanks, Yin On Wed, Aug 13, 2014 at 2:03 PM, Michael Armbrust mich...@databricks.com wrote: This is not supported at the moment. There are no concrete plans at the

Re: Spark RuntimeException due to Unsupported datatype NullType

2014-08-19 Thread Yin Huai
Hi Rafeeq, I think the following part triggered the bug https://issues.apache.org/jira/browse/SPARK-2908. [{*href:null*,rel:me}] It has been fixed. Can you try spark master and see if the error get resolved? Thanks, Yin On Mon, Aug 11, 2014 at 3:53 AM, rafeeq s rafeeq.ec...@gmail.com wrote:

Re: NullPointerException when connecting from Spark to a Hive table backed by HBase

2014-08-19 Thread Yin Huai
Seems https://issues.apache.org/jira/browse/SPARK-2846 is the jira tracking this issue. On Mon, Aug 18, 2014 at 6:26 PM, cesararevalo ce...@zephyrhealthinc.com wrote: Thanks, Zhan for the follow up. But, do you know how I am supposed to set that table name on the jobConf? I don't have

Re: spark error when distinct on more than one cloume

2014-08-19 Thread Yin Huai
Hi, The SQLParser used by SQLContext is pretty limited. Instead, can you try HiveContext? Thanks, Yin On Tue, Aug 19, 2014 at 7:57 AM, wan...@testbird.com wan...@testbird.com wrote: sql:SELECT app_id,COUNT(DISTINCT app_id, macaddr) cut from object group by app_id *Error Log* 14/08/19

Re: Got NotSerializableException when access broadcast variable

2014-08-20 Thread Yin Huai
If you want to filter the table name, you can use hc.sql(show tables).filter(row = !test.equals(row.getString(0 Seems making functionRegistry transient can fix the error. On Wed, Aug 20, 2014 at 8:53 PM, Vida Ha v...@databricks.com wrote: Hi, I doubt the the broadcast variable is your

RE: Got NotSerializableException when access broadcast variable

2014-08-20 Thread Yin Huai
PR is https://github.com/apache/spark/pull/2074. -- From: Yin Huai huaiyin@gmail.com Sent: ‎8/‎20/‎2014 10:56 PM To: Vida Ha v...@databricks.com Cc: tianyi tia...@asiainfo.com; Fengyun RAO raofeng...@gmail.com; user@spark.apache.org Subject: Re: Got

Re: Spark SQL: Caching nested structures extremely slow

2014-08-21 Thread Yin Huai
I have not profiled this part. But, I think one possible cause is allocating an array for every inner struct for every row (every struct value is represented by a Spark SQL row). I will play with it later and see what I find. On Tue, Aug 19, 2014 at 9:01 PM, Evan Chan velvia.git...@gmail.com

Re: Spark SQL Parser error

2014-08-22 Thread Yin Huai
Hi Sankar, You need to create an external table in order to specify the location of data (i.e. using CREATE EXTERNAL TABLE user1 LOCATION). You can take a look at this page https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DDL#LanguageManualDDL-Create/Drop/TruncateTable for

Re: Spark SQL Parser error

2014-08-22 Thread Yin Huai
the create external table command as well. I get the same error. Please help me to find the root cause. Thanks and Regards, Sankar S. On Friday, 22 August 2014, 22:43, Yin Huai huaiyin@gmail.com wrote: Hi Sankar, You need to create an external table in order to specify

Re: Spark SQL Parser error

2014-08-26 Thread Yin Huai
, In all three option when I try to create temporary funtion i get the classNotFoundException. What would be the issue here? Thanks and Regards, Sankar S. On Saturday, 23 August 2014, 0:53, Yin Huai huaiyin@gmail.com wrote: Hello Sankar, Add JAR in SQL is not supported at the moment

Re: unable to instantiate HiveMetaStoreClient on LocalHiveContext

2014-08-26 Thread Yin Huai
Hello Du, Can you check if there is a dir metastore in the place you launching your program. If so, can you delete it and try again? Also, can you try HiveContext? LocalHiveContext is deprecated. Thanks, Yin On Mon, Aug 25, 2014 at 6:33 PM, Du Li l...@yahoo-inc.com.invalid wrote: Hi, I

Re: Problem Accessing Hive Table from hiveContext

2014-09-01 Thread Yin Huai
Hello Igor, Although Decimal is supported, Hive 0.12 does not support user definable precision and scale (it was introduced in Hive 0.13). Thanks, Yin On Sat, Aug 30, 2014 at 1:50 AM, Zitser, Igor igor.zit...@citi.com wrote: Hi All, New to spark and using Spark 1.0.2 and hive 0.12. If

Re: spark sql - create new_table as select * from table

2014-09-11 Thread Yin Huai
What is the schema of table? On Thu, Sep 11, 2014 at 4:30 PM, jamborta jambo...@gmail.com wrote: thanks. this was actually using hivecontext. -- View this message in context:

Re: Re: Spark SQL -- more than two tables for join

2014-09-11 Thread Yin Huai
1.0.1 does not have the support on outer joins (added in 1.1). Can you try 1.1 branch? On Wed, Sep 10, 2014 at 9:28 PM, boyingk...@163.com boyingk...@163.com wrote: Hi,michael : I think Arthur.hk.chan arthur.hk.c...@gmail.com isn't here now,I Can Show something: 1)my spark version is 1.0.1

Re: compiling spark source code

2014-09-13 Thread Yin Huai
Can you try sbt/sbt clean first? On Sat, Sep 13, 2014 at 4:29 PM, Ted Yu yuzhih...@gmail.com wrote: bq. [error] File name too long It is not clear which file(s) loadfiles was loading. Is the filename in earlier part of the output ? Cheers On Sat, Sep 13, 2014 at 10:58 AM, kkptninja

Re: About SparkSQL 1.1.0 join between more than two table

2014-09-15 Thread Yin Huai
1.0.1 does not have the support on outer joins (added in 1.1). Your query should be fine in 1.1. On Mon, Sep 15, 2014 at 5:35 AM, Yanbo Liang yanboha...@gmail.com wrote: Spark SQL can support SQL and HiveSQL which used SQLContext and HiveContext separate. As far as I know, SQLContext of Spark

Re: SparkSQL 1.1 hang when DROP or LOAD

2014-09-16 Thread Yin Huai
Seems https://issues.apache.org/jira/browse/HIVE-5474 is related? On Tue, Sep 16, 2014 at 4:49 AM, Cheng, Hao hao.ch...@intel.com wrote: Thank you for pasting the steps, I will look at this, hopefully come out with a solution soon. -Original Message- From: linkpatrickliu

Re: SparkSQL 1.1 hang when DROP or LOAD

2014-09-16 Thread Yin Huai
I meant it may be a Hive bug since we also call Hive's drop table internally. On Tue, Sep 16, 2014 at 1:44 PM, Yin Huai huaiyin@gmail.com wrote: Seems https://issues.apache.org/jira/browse/HIVE-5474 is related? On Tue, Sep 16, 2014 at 4:49 AM, Cheng, Hao hao.ch...@intel.com wrote: Thank

Re: spark-1.1.0-bin-hadoop2.4 java.lang.NoClassDefFoundError: org/codehaus/jackson/annotate/JsonClass

2014-09-18 Thread Yin Huai
Hello Andy, Will our JSON support in Spark SQL help your case? If your JSON files store one JSON object per line, you can use SQLContext.jsonFile to load it. If you want to do pre-process these files, once you have an RDD[String] (one JSON object per String), you can use SQLContext.jsonRDD. In

Re: Spark SQL CLI

2014-09-22 Thread Yin Huai
Hi Gaurav, Can you put hive-site.xml in conf/ and try again? Thanks, Yin On Mon, Sep 22, 2014 at 4:02 PM, gtinside gtins...@gmail.com wrote: Hi , I have been using spark shell to execute all SQLs. I am connecting to Cassandra , converting the data in JSON and then running queries on it,

Re: Unresolved attributes: SparkSQL on the schemaRDD

2014-09-29 Thread Yin Huai
What version of Spark did you use? Can you try the master branch? On Mon, Sep 29, 2014 at 1:52 PM, vdiwakar.malladi vdiwakar.mall...@gmail.com wrote: Thanks for your prompt response. Still on further note, I'm getting the exception while executing the query. SELECT data[0].name FROM people

Re: Unresolved attributes: SparkSQL on the schemaRDD

2014-09-30 Thread Yin Huai
I think this problem has been fixed after the 1.1 release. Can you try the master branch? On Mon, Sep 29, 2014 at 10:06 PM, vdiwakar.malladi vdiwakar.mall...@gmail.com wrote: I'm using the latest version i.e. Spark 1.1.0 Thanks. -- View this message in context:

Re: partition size for initial read

2014-10-02 Thread Yin Huai
Hi Tamas, Can you try to set mapred.map.tasks and see if it works? Thanks, Yin On Thu, Oct 2, 2014 at 10:33 AM, Tamas Jambor jambo...@gmail.com wrote: That would work - I normally use hive queries through spark sql, I have not seen something like that there. On Thu, Oct 2, 2014 at 3:13

Re: How To Implement More Than One Subquery in Scala/Spark

2014-10-13 Thread Yin Huai
Question 1: Please check http://spark.apache.org/docs/1.1.0/sql-programming-guide.html#hive-tables. Question 2: One workaround is to re-write it. You can use LEFT SEMI JOIN to implement the subquery with EXISTS and use LEFT OUTER JOIN + IS NULL to implement the subquery with NOT EXISTS. SELECT

Re: Is Array Of Struct supported in json RDDs? is it possible to query this?

2014-10-13 Thread Yin Huai
If you are using HiveContext, it should work in 1.1. Thanks, Yin On Mon, Oct 13, 2014 at 5:08 AM, shahab shahab.mok...@gmail.com wrote: Hello, Given the following structure, is it possible to query, e.g. session[0].id ? In general, is it possible to query Array Of Struct in json RDDs?

Re: Nested Query using SparkSQL 1.1.0

2014-10-13 Thread Yin Huai
Hi Shahab, Can you try to use HiveContext? Its should work in 1.1. For SQLContext, this issues was not fixed in 1.1 and you need to use master branch at the moment. Thanks, Yin On Sun, Oct 12, 2014 at 5:20 PM, shahab shahab.mok...@gmail.com wrote: Hi, Apparently is it is possible to query

Re: Spark SQL parser bug?

2014-10-13 Thread Yin Huai
Seems the reason that you got wrong results was caused by timezone. The time in java.sql.Timestamp(long time) means milliseconds since January 1, 1970, 00:00:00 *GMT*. A negative number is the number of milliseconds before January 1, 1970, 00:00:00 *GMT*. However, in ts='1970-01-01 00:00:00',

Re: Nested Query using SparkSQL 1.1.0

2014-10-13 Thread Yin Huai
on youtube Easy JSON Data Manipulation in Spark), is it possible to perform aggregation kind queries, for example counting number of attributes (considering that attributes in schema is presented as array), or any other type of aggregation? best, /Shahab On Mon, Oct 13, 2014 at 4:01 PM, Yin Huai

Re: Spark SQL parser bug?

2014-10-13 Thread Yin Huai
,ts#3], MapPartitionsRDD[22] at mapPartitions at basicOperators.scala:208 scala s.collect res5: Array[org.apache.spark.sql.Row] = Array() Mohammed *From:* Yin Huai [mailto:huaiyin@gmail.com] *Sent:* Monday, October 13, 2014 7:19 AM *To:* Mohammed Guller *Cc:* Cheng, Hao

Re: SparkSQL IndexOutOfBoundsException when reading from Parquet

2014-10-14 Thread Yin Huai
Hello Terry, How many columns does pqt_rdt_snappy have? Thanks, Yin On Tue, Oct 14, 2014 at 11:52 AM, Terry Siu terry@smartfocus.com wrote: Hi Michael, That worked for me. At least I’m now further than I was. Thanks for the tip! -Terry From: Michael Armbrust

Re: SparkSQL IndexOutOfBoundsException when reading from Parquet

2014-10-16 Thread Yin Huai
by the 2 partition columns, coll_def_id and seg_def_id. Output shows 29 rows, but that looks like it’s just counting the rows in the console output. Let me know if you need more information. Thanks -Terry From: Yin Huai huaiyin@gmail.com Date: Tuesday, October 14, 2014 at 6:29 PM

Re: spark sql: timestamp in json - fails

2014-10-20 Thread Yin Huai
Hi Tridib, For the second approach, can you attach the complete stack trace? Thanks, Yin On Mon, Oct 20, 2014 at 8:24 PM, Michael Armbrust mich...@databricks.com wrote: I think you are running into a bug that will be fixed by this PR: https://github.com/apache/spark/pull/2850 On Mon, Oct

Re: spark sql: timestamp in json - fails

2014-10-20 Thread Yin Huai
/jira/browse/SPARK-4003 You can check PR https://github.com/apache/spark/pull/2850 . Thanks, Daoyuan *From:* Yin Huai [mailto:huaiyin@gmail.com] *Sent:* Tuesday, October 21, 2014 10:00 AM *To:* Michael Armbrust *Cc:* tridib; u...@spark.incubator.apache.org *Subject:* Re: spark

Re: spark sql: sqlContext.jsonFile date type detection and perforormance

2014-10-21 Thread Yin Huai
Is there any specific issues you are facing? Thanks, Yin On Tue, Oct 21, 2014 at 4:00 PM, tridib tridib.sama...@live.com wrote: Any help? or comments? -- View this message in context:

Re: Spark SQL : sqlContext.jsonFile date type detection and perforormance

2014-10-21 Thread Yin Huai
that may help. val hiveContext = new org.apache.spark.sql.hive.HiveContext(sc) val schemaRDD = hiveContext.jsonFile(...) schemaRDD.registerTempTable(jsonTable) hiveContext.sql(SELECT CAST(columnName as DATE) FROM jsonTable) Thanks, Yin On Tue, Oct 21, 2014 at 8:00 PM, Yin Huai huaiyin

Re: SchemaRDD Convert

2014-10-22 Thread Yin Huai
The implicit conversion function mentioned by Hao is createSchemaRDD in SQLContext/HiveContext. You can import it by doing val sqlContext = new org.apache.spark.sql.SQLContext(sc) // Or new org.apache.spark.sql.hive.HiveContext(sc) for HiveContext import sqlContext.createSchemaRDD On Wed, Oct

Re: Aggregation Error: org.apache.spark.sql.catalyst.errors.package$TreeNodeException:

2014-10-23 Thread Yin Huai
Hello Arthur, You can use do aggregations in SQL. How did you create LINEITEM? Thanks, Yin On Thu, Oct 23, 2014 at 8:54 AM, arthur.hk.c...@gmail.com arthur.hk.c...@gmail.com wrote: Hi, I got $TreeNodeException, few questions: Q1) How should I do aggregation in SparK? Can I use

Re: spark sql create nested schema

2014-11-04 Thread Yin Huai
Hello Tridib, For you case, you can use StructType(StructField(ParentInfo, parentInfo, true) :: StructField(ChildInfo, childInfo, true) :: Nil) to create the StructType representing the schema (parentInfo and childInfo are two existing StructTypes). You can take a look at our docs (

Re: [SQL] PERCENTILE is not working

2014-11-05 Thread Yin Huai
Hello Kevin, https://issues.apache.org/jira/browse/SPARK-3891 will fix this bug. Thanks, Yin On Wed, Nov 5, 2014 at 8:06 PM, Cheng, Hao hao.ch...@intel.com wrote: Which version are you using? I can reproduce that in the latest code, but with different exception. I've filed an bug

Re: jsonRdd and MapType

2014-11-07 Thread Yin Huai
Hello Brian, Right now, MapType is not supported in the StructType provided to jsonRDD/jsonFile. We will add the support. I have created https://issues.apache.org/jira/browse/SPARK-4302 to track this issue. Thanks, Yin On Fri, Nov 7, 2014 at 3:41 PM, boclair bocl...@gmail.com wrote: I'm

Re: Converting a json struct to map

2014-11-19 Thread Yin Huai
Oh, actually, we do not support MapType provided by the schema given to jsonRDD at the moment (my bad..). Daniel, you need to wait for the patch of 4476 (I should have one soon). Thanks, Yin On Wed, Nov 19, 2014 at 2:32 PM, Daniel Haviv danielru...@gmail.com wrote: Thank you Michael I will

Re: How to deal with BigInt in my case class for RDD = SchemaRDD convertion

2014-11-21 Thread Yin Huai
Hello Jianshi, The reason of that error is that we do not have a Spark SQL data type for Scala BigInt. You can use Decimal for your case. Thanks, Yin On Fri, Nov 21, 2014 at 5:11 AM, Jianshi Huang jianshi.hu...@gmail.com wrote: Hi, I got an error during rdd.registerTempTable(...) saying

Re: Spark SQL Join returns less rows that expected

2014-11-25 Thread Yin Huai
I guess you want to use split(\\|) instead of split(|). On Tue, Nov 25, 2014 at 4:51 AM, Cheng Lian lian.cs@gmail.com wrote: Which version are you using? Or if you are using the most recent master or branch-1.2, which commit are you using? On 11/25/14 4:08 PM, david wrote: Hi, I

Re: SchemaRDD.saveAsTable() when schema contains arrays and was loaded from a JSON file using schema auto-detection

2014-11-26 Thread Yin Huai
Hello Jonathan, There was a bug regarding casting data types before inserting into a Hive table. Hive does not have the notion of containsNull for array values. So, for a Hive table, the containsNull will be always true for an array and we should ignore this field for Hive. This issue has been

Re: can't get smallint field from hive on spark

2014-11-26 Thread Yin Huai
For hive on spark, did you mean the thrift server of Spark SQL or https://issues.apache.org/jira/browse/HIVE-7292? If you meant the latter one, I think Hive's mailing list will be a good place to ask (see https://hive.apache.org/mailing_lists.html). Thanks, Yin On Wed, Nov 26, 2014 at 10:49 PM,

Re: Convert RDD[Map[String, Any]] to SchemaRDD

2014-12-08 Thread Yin Huai
Hello Jianshi, You meant you want to convert a Map to a Struct, right? We can extract some useful functions from JsonRDD.scala, so others can access them. Thanks, Yin On Mon, Dec 8, 2014 at 1:29 AM, Jianshi Huang jianshi.hu...@gmail.com wrote: I checked the source code for inferSchema. Looks

Re: [SQL] Using HashPartitioner to distribute by column

2015-01-21 Thread Yin Huai
Hello Michael, In Spark SQL, we have our internal concepts of Output Partitioning (representing the partitioning scheme of an operator's output) and Required Child Distribution (representing the requirement of input data distribution of an operator) for a physical operator. Let's say we have two

Re: spark sql registerFunction with 1.2.1

2015-02-11 Thread Yin Huai
Regarding backticks: Right. You need backticks to quote the column name timestamp because timestamp is a reserved keyword in our parser. On Tue, Feb 10, 2015 at 3:02 PM, Mohnish Kodnani mohnish.kodn...@gmail.com wrote: actually i tried in spark shell , got same error and then for some reason i

Re: Can we execute create table and load data commands against Hive inside HiveContext?

2015-02-10 Thread Yin Huai
org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdConfOnlyAuthorizerFactory was introduced in Hive 0.14 and Spark SQL only supports Hive 0.12 and 0.13.1. Can you change the setting of hive.security.authorization.manager to someone accepted by 0.12 or 0.13.1? On Thu, Feb 5, 2015

Re: Spark SQL - Column name including a colon in a SELECT clause

2015-02-10 Thread Yin Huai
Can you try using backticks to quote the field name? Like `f:price`. On Tue, Feb 10, 2015 at 5:47 AM, presence2001 neil.andra...@thefilter.com wrote: Hi list, I have some data with a field name of f:price (it's actually part of a JSON structure loaded from ElasticSearch via

Re: SQLContext.applySchema strictness

2015-02-13 Thread Yin Huai
Hi Justin, It is expected. We do not check if the provided schema matches rows since all rows need to be scanned to give a correct answer. Thanks, Yin On Fri, Feb 13, 2015 at 1:33 PM, Justin Pihony justin.pih...@gmail.com wrote: Per the documentation: It is important to make sure that

Re: Spark SQL weird exception after upgrading from 1.1.1 to 1.2.x

2015-03-18 Thread Yin Huai
Hi Roberto, For now, if the timestamp is a top level column (not a field in a struct), you can use use backticks to quote the column name like `timestamp `. Thanks, Yin On Wed, Mar 18, 2015 at 12:10 PM, Roberto Coluccio roberto.coluc...@gmail.com wrote: Hey Cheng, thank you so much for your

Re: HiveContext can't find registered function

2015-03-17 Thread Yin Huai
it mean 'resolved attribute'? On Mar 17, 2015 8:14 PM, Yin Huai yh...@databricks.com wrote: The number is an id we used internally to identify an resolved Attribute. Looks like basic_null_diluted_d was not resolved since there is no id associated with it. On Tue, Mar 17, 2015 at 2:08 PM

Re: Date and decimal datatype not working

2015-03-17 Thread Yin Huai
p(0) is a String. So, you need to explicitly convert it to a Long. e.g. p(0).trim.toLong. You also need to do it for p(2). For those BigDecimals value, you need to create BigDecimal objects from your String values. On Tue, Mar 17, 2015 at 5:55 PM, BASAK, ANANDA ab9...@att.com wrote: Hi All,

Re: HiveContext can't find registered function

2015-03-17 Thread Yin Huai
. I will check it soon and update. Can you elaborate what does it mean the # and the number? Is that a reference to the field in the rdd? Thank you, Ophir On Mar 17, 2015 7:06 PM, Yin Huai yh...@databricks.com wrote: Seems basic_null_diluted_d was not resolved? Can you check

Re: Spark SQL filter DataFrame by date?

2015-03-19 Thread Yin Huai
Can you add your code snippet? Seems it's missing in the original email. Thanks, Yin On Thu, Mar 19, 2015 at 3:22 PM, kamatsuoka ken...@gmail.com wrote: I'm trying to filter a DataFrame by a date column, with no luck so far. Here's what I'm doing: When I run reqs_day.count() I get zero,

Re: DataFrame operation on parquet: GC overhead limit exceeded

2015-03-20 Thread Yin Huai
Yin, thanks a lot for that! Will give it a shot and let you know. On 19 March 2015 at 16:30, Yin Huai yh...@databricks.com wrote: Was the OOM thrown during the execution of first stage (map) or the second stage (reduce)? If it was the second stage, can you increase the value

Re: HiveContext can't find registered function

2015-03-17 Thread Yin Huai
Seems basic_null_diluted_d was not resolved? Can you check if basic_null_diluted_d is in you table? On Tue, Mar 17, 2015 at 9:34 AM, Ophir Cohen oph...@gmail.com wrote: Hi Guys, I'm registering a function using: sqlc.registerFunction(makeEstEntry,ReutersDataFunctions.makeEstEntry _) Then I

Re: Spark-SQL and Hive - is Hive required?

2015-03-06 Thread Yin Huai
Hi Edmon, No, you do not need to install Hive to use Spark SQL. Thanks, Yin On Fri, Mar 6, 2015 at 6:31 AM, Edmon Begoli ebeg...@gmail.com wrote: Does Spark-SQL require installation of Hive for it to run correctly or not? I could not tell from this statement:

Re: Loading in json with spark sql

2015-03-13 Thread Yin Huai
Seems you want to use array for the field of providers, like providers:[{id: ...}, {id:...}] instead of providers:{{id: ...}, {id:...}} On Fri, Mar 13, 2015 at 7:45 PM, kpeng1 kpe...@gmail.com wrote: Hi All, I was noodling around with loading in a json file into spark sql's hive context and

Re: Spark SQL. Cast to Bigint

2015-03-13 Thread Yin Huai
Are you using SQLContext? Right now, the parser in the SQLContext is quite limited on the data type keywords that it handles (see here https://github.com/apache/spark/blob/master/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala#L391) and unfortunately bigint is not handled

Re: Spark 1.3 SQL Type Parser Changes?

2015-03-10 Thread Yin Huai
Hi Nitay, Can you try using backticks to quote the column name? Like org.apache.spark.sql.hive.HiveMetastoreTypes.toDataType( struct`int`:bigint)? Thanks, Yin On Tue, Mar 10, 2015 at 2:43 PM, Michael Armbrust mich...@databricks.com wrote: Thanks for reporting. This was a result of a change

Re: Supporting Hive features in Spark SQL Thrift JDBC server

2015-03-03 Thread Yin Huai
Regarding current_date, I think it is not in either Hive 0.12.0 or 0.13.1 (versions that we support). Seems https://issues.apache.org/jira/browse/HIVE-5472 added it Hive recently. On Tue, Mar 3, 2015 at 6:03 AM, Cheng, Hao hao.ch...@intel.com wrote: The temp table in metastore can not be

Re: Issues reading in Json file with spark sql

2015-03-02 Thread Yin Huai
Is the string of the above JSON object in the same line? jsonFile requires that every line is a JSON object or an array of JSON objects. On Mon, Mar 2, 2015 at 11:28 AM, kpeng1 kpe...@gmail.com wrote: Hi All, I am currently having issues reading in a json file using spark sql's api. Here is

Re: Supporting Hive features in Spark SQL Thrift JDBC server

2015-03-03 Thread Yin Huai
@Shahab, based on https://issues.apache.org/jira/browse/HIVE-5472, current_date was added in Hive *1.2.0 (not 0.12.0)*. For my previous email, I meant current_date is not in neither Hive 0.12.0 nor Hive 0.13.1 (Spark SQL currently supports these two Hive versions). On Tue, Mar 3, 2015 at 8:55 AM,

Re: [SparkSQL, Spark 1.2] UDFs in group by broken?

2015-02-26 Thread Yin Huai
Seems you hit https://issues.apache.org/jira/browse/SPARK-4296. It has been fixed in 1.2.1 and 1.3. On Thu, Feb 26, 2015 at 1:22 PM, Yana Kadiyska yana.kadiy...@gmail.com wrote: Can someone confirm if they can run UDFs in group by in spark1.2? I have two builds running -- one from a custom

Re: saveAsTable broken in v1.3 DataFrames?

2015-03-19 Thread Yin Huai
created https://issues.apache.org/jira/browse/SPARK-6413 to track the improvement on the output of DESCRIBE statement. On Thu, Mar 19, 2015 at 12:11 PM, Yin Huai yh...@databricks.com wrote: Hi Christian, Your table is stored correctly in Parquet format. For saveAsTable, the table created

Re: saveAsTable broken in v1.3 DataFrames?

2015-03-19 Thread Yin Huai
Hi Christian, Your table is stored correctly in Parquet format. For saveAsTable, the table created is *not* a Hive table, but a Spark SQL data source table ( http://spark.apache.org/docs/1.3.0/sql-programming-guide.html#data-sources). We are only using Hive's metastore to store the metadata (to

Re: DataFrame operation on parquet: GC overhead limit exceeded

2015-03-19 Thread Yin Huai
: Hi Yin, Thanks for your feedback. I have 1700 parquet files, sized 100MB each. The number of tasks launched is equal to the number of parquet files. Do you have any idea on how to deal with this situation? Thanks a lot On 18 Mar 2015 17:35, Yin Huai yh...@databricks.com wrote: Seems

Re: rdd.toDF().saveAsParquetFile(tachyon://host:19998/test)

2015-03-28 Thread Yin Huai
You are hitting https://issues.apache.org/jira/browse/SPARK-6330. It has been fixed in 1.3.1, which will be released soon. On Fri, Mar 27, 2015 at 10:42 PM, sud_self 852677...@qq.com wrote: spark version is 1.3.0 with tanhyon-0.6.1 QUESTION DESCRIPTION:

Re: Hive UDAF percentile_approx says This UDAF does not support the deprecated getEvaluator() method.

2015-01-13 Thread Yin Huai
Yeah, it's a bug. It has been fixed by https://issues.apache.org/jira/browse/SPARK-3891 in master. On Tue, Jan 13, 2015 at 2:41 PM, Ted Yu yuzhih...@gmail.com wrote: Looking at the source code for AbstractGenericUDAFResolver, the following (non-deprecated) method should be called: public

Re: Use pig load function in spark

2015-03-23 Thread Yin Huai
Hello Kevin, You can take a look at our generic load function https://spark.apache.org/docs/1.3.0/sql-programming-guide.html#generic-loadsave-functions . For example, you can use val df = sqlContext.load(/myData, parquet) To load a parquet dataset stored in /myData as a DataFrame

Re: Date and decimal datatype not working

2015-03-23 Thread Yin Huai
--- Ananda Basak Ph: 425-213-7092 *From:* BASAK, ANANDA *Sent:* Tuesday, March 17, 2015 3:08 PM *To:* Yin Huai *Cc:* user@spark.apache.org *Subject:* RE: Date and decimal datatype not working Ok, thanks for the suggestions. Let me try and will confirm all. Regards Ananda

Re: [SQL] Simple DataFrame questions

2015-04-02 Thread Yin Huai
For cast, you can use selectExpr method. For example, df.selectExpr(cast(col1 as int) as col1, cast(col2 as bigint) as col2). Or, df.select(df(colA).cast(int), ...) On Thu, Apr 2, 2015 at 8:33 PM, Michael Armbrust mich...@databricks.com wrote: val df = Seq((test, 1)).toDF(col1, col2) You can

Re: Bug? Can't reference to the column by name after join two DataFrame on a same name key

2015-04-23 Thread Yin Huai
Hi Shuai, You can use as to create a table alias. For example, df1.as(df1). Then you can use $df1.col to refer it. Thanks, Yin On Thu, Apr 23, 2015 at 11:14 AM, Shuai Zheng szheng.c...@gmail.com wrote: Hi All, I use 1.3.1 When I have two DF and join them on a same name key, after

Re: Parquet Hive table become very slow on 1.3?

2015-04-22 Thread Yin Huai
Xudong and Rex, Can you try 1.3.1? With PR 5339 http://github.com/apache/spark/pull/5339 , after we get a hive parquet from metastore and convert it to our native parquet code path, we will cache the converted relation. For now, the first access to that hive parquet table reads all of the footers

Re: Parquet error reading data that contains array of structs

2015-04-24 Thread Yin Huai
, 2015 at 11:00 AM, Yin Huai yh...@databricks.com wrote: The exception looks like the one mentioned in https://issues.apache.org/jira/browse/SPARK-4520. What is the version of Spark? On Fri, Apr 24, 2015 at 2:40 AM, Jianshi Huang jianshi.hu...@gmail.com wrote: Hi, My data looks like

  1   2   >