If you are writing to an existing hive table, our insert into operator
follows hive's requirement, which is
"*the dynamic partition columns must be specified last among the columns in
the SELECT statement and in the same order** in which they appear in the
PARTITION() clause*."

You can find requirement in
https://cwiki.apache.org/confluence/display/Hive/DynamicPartitions.

If you use select to reorder columns, I think it should work. Also, since
the table is an existing hive table, you do not need to specify the format
because we will use the format of existing table.

btw, please feel free to open a jira about removing this requirement for
inserting into an existing hive table.

Thanks,

Yin

On Thu, Jun 18, 2015 at 9:39 PM, Yin Huai <yh...@databricks.com> wrote:

> Are you writing to an existing hive orc table?
>
> On Wed, Jun 17, 2015 at 3:25 PM, Cheng Lian <lian.cs....@gmail.com> wrote:
>
>> Thanks for reporting this. Would you mind to help creating a JIRA for
>> this?
>>
>>
>> On 6/16/15 2:25 AM, patcharee wrote:
>>
>>> I found if I move the partitioned columns in schemaString and in Row to
>>> the end of the sequence, then it works correctly...
>>>
>>> On 16. juni 2015 11:14, patcharee wrote:
>>>
>>>> Hi,
>>>>
>>>> I am using spark 1.4 and HiveContext to append data into a partitioned
>>>> hive table. I found that the data insert into the table is correct, but the
>>>> partition(folder) created is totally wrong.
>>>> Below is my code snippet>>
>>>>
>>>> -------------------------------------------------------------------------------------------------------------------------------------------------------------------
>>>>
>>>> val schemaString = "zone z year month date hh x y height u v w ph phb t
>>>> p pb qvapor qgraup qnice qnrain tke_pbl el_pbl"
>>>>     val schema =
>>>>       StructType(
>>>>         schemaString.split(" ").map(fieldName =>
>>>>           if (fieldName.equals("zone") || fieldName.equals("z") ||
>>>> fieldName.equals("year") || fieldName.equals("month") ||
>>>>               fieldName.equals("date") || fieldName.equals("hh") ||
>>>> fieldName.equals("x") || fieldName.equals("y"))
>>>>             StructField(fieldName, IntegerType, true)
>>>>           else
>>>>             StructField(fieldName, FloatType, true)
>>>>         ))
>>>>
>>>> val pairVarRDD =
>>>> sc.parallelize(Seq((Row(2,42,2009,3,1,0,218,365,9989.497.floatValue(),29.627113.floatValue(),19.071793.floatValue(),0.11982734.floatValue(),3174.6812.floatValue(),
>>>>
>>>> 97735.2.floatValue(),16.389032.floatValue(),-96.62891.floatValue(),25135.365.floatValue(),2.6476808E-5.floatValue(),0.0.floatValue(),13195.351.floatValue(),
>>>>
>>>>         0.0.floatValue(),0.1.floatValue(),0.0.floatValue()))
>>>>     ))
>>>>
>>>> val partitionedTestDF2 = sqlContext.createDataFrame(pairVarRDD, schema)
>>>>
>>>> partitionedTestDF2.write.format("org.apache.spark.sql.hive.orc.DefaultSource")
>>>>
>>>> .mode(org.apache.spark.sql.SaveMode.Append).partitionBy("zone","z","year","month").saveAsTable("test4DimBySpark")
>>>>
>>>>
>>>> -------------------------------------------------------------------------------------------------------------------------------------------------------------------
>>>>
>>>>
>>>> The table contains 23 columns (longer than Tuple maximum length), so I
>>>> use Row Object to store raw data, not Tuple.
>>>> Here is some message from spark when it saved data>>
>>>>
>>>> 15/06/16 10:39:22 INFO metadata.Hive: Renaming
>>>> src:hdfs://service-10-0.local:8020/tmp/hive-patcharee/hive_2015-06-16_10-39-21_205_8768669104487548472-1/-ext-10000/zone=13195/z=0/year=0/month=0/part-00001;dest:
>>>> hdfs://service-10-0.local:8020/apps/hive/warehouse/test4dimBySpark/zone=13195/z=0/year=0/month=0/part-00001;Status:true
>>>>
>>>> 15/06/16 10:39:22 INFO metadata.Hive: New loading path =
>>>> hdfs://service-10-0.local:8020/tmp/hive-patcharee/hive_2015-06-16_10-39-21_205_8768669104487548472-1/-ext-10000/zone=13195/z=0/year=0/month=0
>>>> with partSpec {zone=13195, z=0, year=0, month=0}
>>>>
>>>> From the raw data (pairVarRDD) zone = 2, z = 42, year = 2009, month =
>>>> 3. But spark created a partition {zone=13195, z=0, year=0, month=0}.
>>>>
>>>> When I queried from hive>>
>>>>
>>>> hive> select * from test4dimBySpark;
>>>> OK
>>>> 2    42    2009    3    1.0    0.0    218.0    365.0    9989.497
>>>> 29.627113    19.071793    0.11982734    -3174.6812    97735.2 16.389032
>>>> -96.62891    25135.365    2.6476808E-5    0.0 13195    0    0    0
>>>> hive> select zone, z, year, month from test4dimBySpark;
>>>> OK
>>>> 13195    0    0    0
>>>> hive> dfs -ls /apps/hive/warehouse/test4dimBySpark/*/*/*/*;
>>>> Found 2 items
>>>> -rw-r--r--   3 patcharee hdfs       1411 2015-06-16 10:39
>>>> /apps/hive/warehouse/test4dimBySpark/zone=13195/z=0/year=0/month=0/part-00001
>>>>
>>>> The data stored in the table is correct zone = 2, z = 42, year = 2009,
>>>> month = 3, but the partition created was wrong
>>>> "zone=13195/z=0/year=0/month=0"
>>>>
>>>> Is this a bug or what could be wrong? Any suggestion is appreciated.
>>>>
>>>> BR,
>>>> Patcharee
>>>>
>>>>
>>>>
>>>>
>>>>
>>>>
>>>>
>>>> ---------------------------------------------------------------------
>>>> To unsubscribe, e-mail: user-unsubscr...@spark.apache.org
>>>> For additional commands, e-mail: user-h...@spark.apache.org
>>>>
>>>>
>>>
>>> ---------------------------------------------------------------------
>>> To unsubscribe, e-mail: user-unsubscr...@spark.apache.org
>>> For additional commands, e-mail: user-h...@spark.apache.org
>>>
>>>
>>>
>>
>> ---------------------------------------------------------------------
>> To unsubscribe, e-mail: user-unsubscr...@spark.apache.org
>> For additional commands, e-mail: user-h...@spark.apache.org
>>
>>
>

Reply via email to