Hi,

I found that after I converted to Hudi managed dataset, there are added several 
columns:

_hoodie_commit_time, _hoodie_commit_seqno, _hoodie_record_key, 
_hoodie_partition_path, _hoodie_file_name

Does these columns added into table forever or temporary? Thanks.

Best,
Qian
On Oct 11, 2019, 3:39 PM -0700, Qian Wang <qwang1...@gmail.com>, wrote:
> Hi,
>
> I have successfully converted the parquet data into Hudi managed dataset. 
> However, I found that the previous data size is about 44G, after converted by 
> Hudi, the data size is about 88G. Why the data size increased almost twice?
>
> Best,
> Qian
> On Oct 11, 2019, 1:57 PM -0700, Qian Wang <qwang1...@gmail.com>, wrote:
> > Hi Kabeer,
> >
> > Thanks for your detailed explanation. I will try it again. Will update you 
> > the result.
> >
> > Best,
> > Qian
> > On Oct 11, 2019, 1:49 PM -0700, Kabeer Ahmed <kab...@linuxmail.org>, wrote:
> > > Hi Qian,
> > >
> > > If there are no nulls in the data, then most likey it is issue with the 
> > > data types being stored. I have seen this issue again and again and in 
> > > the recent one it was due to me storing double value when I had actually 
> > > declared the schema as IntegerType. I can reproduce this with an example 
> > > to prove the point. But I think you should look into your data.
> > > If possible I would recommend you run something like: 
> > > https://stackoverflow.com/questions/33270907/how-to-validate-contents-of-spark-dataframe
> > >  
> > > (https://link.getmailspring.com/link/1a222369-02ff-464b-9e5e-48022a443...@getmailspring.com/0?redirect=https%3A%2F%2Fstackoverflow.com%2Fquestions%2F33270907%2Fhow-to-validate-contents-of-spark-dataframe&recipient=ZGV2QGh1ZGkuYXBhY2hlLm9yZw%3D%3D).
> > >  This will show you if there is any value in any column that is against 
> > > the declared schema type. And when you fix that, the errors will go away.
> > > Keep us posted on how you get along with this.
> > > Thanks
> > > Kabeer.
> > >
> > > On Oct 9 2019, at 12:24 am, nishith agarwal <n3.nas...@gmail.com> wrote:
> > > > Hmm, AVRO is case-sensitive but I've not had issues reading fields from
> > > > GenericRecords with lower or upper so I'm not 100% confident on what the
> > > > resolution for a lower vs upper case is. Have you tried using the
> > > > partitionpath field names in upper case (in case your schema field is 
> > > > also
> > > > upper case) ?
> > > >
> > > > -Nishith
> > > > On Tue, Oct 8, 2019 at 4:00 PM Qian Wang <qwang1...@gmail.com> wrote:
> > > > > Hi Nishith,
> > > > > I have checked the data, there is no null in that field. Does there 
> > > > > has
> > > > > other possibility about this error?
> > > > >
> > > > > Thanks,
> > > > > Qian
> > > > > On Oct 8, 2019, 10:55 AM -0700, Qian Wang <qwang1...@gmail.com>, 
> > > > > wrote:
> > > > > > Hi Nishith,
> > > > > >
> > > > > > Thanks for your response.
> > > > > > The session_date is one field in my original dataset. I have some
> > > > >
> > > > > questions about the schema parameter:
> > > > > >
> > > > > > 1. Do I need create the target table?
> > > > > > 2. My source data is Parquet format, why the tool need the schema 
> > > > > > file
> > > > >
> > > > > as the parameter?
> > > > > > 3. Can I use the schema file of Avro format?
> > > > > >
> > > > > > The schema is looks like:
> > > > > > {"type":"record","name":"PathExtractData","doc":"Path event extract 
> > > > > > fact
> > > > > data”,”fields”:[
> > > > > > {“name”:”SESSION_DATE”,”type”:”string”},
> > > > > > {“name”:”SITE_ID”,”type”:”int”},
> > > > > > {“name”:”GUID”,”type”:”string”},
> > > > > > {“name”:”SESSION_KEY”,”type”:”long”},
> > > > > > {“name”:”USER_ID”,”type”:”string”},
> > > > > > {“name”:”STEP”,”type”:”int”},
> > > > > > {“name”:”PAGE_ID”,”type”:”int”}
> > > > > > ]}
> > > > > >
> > > > > > Thanks.
> > > > > > Best,
> > > > > > Qian
> > > > > > On Oct 8, 2019, 10:47 AM -0700, nishith agarwal 
> > > > > > <n3.nas...@gmail.com>,
> > > > >
> > > > > wrote:
> > > > > > > Qian,
> > > > > > >
> > > > > > > It looks like the partitionPathField that you specified 
> > > > > > > (session_date)
> > > > > is
> > > > > > > missing or the code is unable to grab it from your payload. Is 
> > > > > > > this
> > > > > >
> > > > >
> > > > > field a
> > > > > > > top-level field or a nested field in your schema ?
> > > > > > > ( Currently, the HDFSImporterTool looks for your 
> > > > > > > partitionPathField
> > > > > >
> > > > >
> > > > > only at
> > > > > > > the top-level, for example genericRecord.get("session_date") )
> > > > > > >
> > > > > > > Thanks,
> > > > > > > Nishith
> > > > > > >
> > > > > > >
> > > > > > > On Tue, Oct 8, 2019 at 10:12 AM Qian Wang <qwang1...@gmail.com> 
> > > > > > > wrote:
> > > > > > > > Hi,
> > > > > > > > Thanks for your response.
> > > > > > > > Now I tried to convert existing dataset to Hudi managed dataset 
> > > > > > > > and
> > > > > I used
> > > > > > > > the hdfsparquestimport in hud-cli. I encountered following 
> > > > > > > > error:
> > > > > > > >
> > > > > > > > 19/10/08 09:50:59 INFO DAGScheduler: Job 1 failed: countByKey at
> > > > > > > > HoodieBloomIndex.java:148, took 2.913761 s
> > > > > > > > 19/10/08 09:50:59 ERROR HDFSParquetImporter: Error occurred.
> > > > > > > > org.apache.hudi.exception.HoodieUpsertException: Failed to 
> > > > > > > > upsert for
> > > > > > > > commit time 20191008095056
> > > > > > > >
> > > > > > > > Caused by: org.apache.hudi.exception.HoodieIOException: 
> > > > > > > > partition
> > > > > key is
> > > > > > > > missing. :session_date
> > > > > > > >
> > > > > > > > My command in hud-cli as following:
> > > > > > > > hdfsparquetimport --upsert false --srcPath /path/to/source
> > > > > > >
> > > > > >
> > > > >
> > > > > --targetPath
> > > > > > > > /path/to/target --tableName xxx --tableType COPY_ON_WRITE
> > > > > > >
> > > > > >
> > > > >
> > > > > --rowKeyField
> > > > > > > > _row_key --partitionPathField session_date --parallelism 1500
> > > > > > > > --schemaFilePath /path/to/avro/schema --format parquet 
> > > > > > > > --sparkMemory
> > > > > > >
> > > > > >
> > > > >
> > > > > 6g
> > > > > > > > --retry 2
> > > > > > > >
> > > > > > > > Could you please tell me how to solve this problem? Thanks.
> > > > > > > > Best,
> > > > > > > > Qian
> > > > > > > > On Oct 6, 2019, 9:15 AM -0700, Qian Wang <qwang1...@gmail.com>,
> > > > > > >
> > > > > >
> > > > >
> > > > > wrote:
> > > > > > > > > Hi,
> > > > > > > > >
> > > > > > > > > I have some questions when I try to use Hudi in my company’s 
> > > > > > > > > prod
> > > > > env:
> > > > > > > > >
> > > > > > > > > 1. When I migrate the history table in HDFS, I tried use 
> > > > > > > > > hudi-cli
> > > > > and
> > > > > > > > HDFSParquetImporter tool. How can I specify Spark parameters in 
> > > > > > > > this
> > > > > > >
> > > > > >
> > > > >
> > > > > tool,
> > > > > > > > such as Yarn queue, etc?
> > > > > > > > > 2. Hudi needs to write metadata to Hive and it uses
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > > > HiveMetastoreClient
> > > > > > > > and HiveJDBC. How can I do if the Hive has Kerberos 
> > > > > > > > Authentication?
> > > > > > > > >
> > > > > > > > > Thanks.
> > > > > > > > > Best,
> > > > > > > > > Qian
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > > >
> > >

Reply via email to