Thats awesome! and you are very very brave to attempt this :) Do you have plans to eventually upgrade Hive? Would really love to support your use case on an official release :)
On Thu, Oct 24, 2019 at 4:58 AM Gurudatt Kulkarni <[email protected]> wrote: > Hi Vinoth, > > I tried your second suggestion and it worked > > >>> B) Try to shade/bundle more jars to try and make master work. Current > bundles don't include parquet - either in utilities-bundle for hive syncing > or mr-bundle for queries. This amounts to including jars > > https://github.com/apache/incubator-hudi/blob/master/packaging/hudi-hadoop-mr-bundle/pom.xml#L66 > here. > > Just putting down points that I followed > > 1. Changed to Hive 1.1 and modified relevant classes, here's the diff [1] > 2. But even after modifying the relevant classes and building, I was > hitting into the same error, then I stumbled into this PR [2]. I followed > the same and commented out the relocation patterns, and it worked. > Just couldn't understand how shading could cause ClassCastExceptions. > > [1] > https://github.com/apache/incubator-hudi/compare/master...Guru107:hive11 > [2] https://github.com/apache/incubator-hudi/pull/625 > > > Regards, > Gurudatt > > On Mon, Oct 21, 2019 at 9:29 PM Gurudatt Kulkarni <[email protected]> > wrote: > > > Hi Vinoth, > > > > Thank you for going back in time to figure out a way :) > > Will try out your suggestions. > > > > Regards, > > Gurudatt > > > > On Monday, October 21, 2019, Vinoth Chandar <[email protected]> wrote: > > > After looking into some diffs. Main issue with Hive 1.x is that its > still > > > on com.twitter:parquet and our code is now (rightfully) moved to > > > org.apache.parquet. > > > > > > > > > https://github.com/apache/incubator-hudi/blob/hoodie-0.4.7/hoodie-hadoop-mr/pom.xml#L87 > > > > > > https://github.com/apache/incubator-hudi/blob/hoodie-0.4.7/packaging/hoodie-hadoop-mr-bundle/pom.xml#L183 > > > > > > > > > A) We could either go back to previous release of Hudi that supported > > 1.x. > > > you would need to build using > > > > > > https://github.com/apache/incubator-hudi/commit/e042aea837412b461b02f8062d8a11e8029823bf#diff-018e220b50f876168d6d1d4dbf52ce84 > > > *mvn clean install -DskipTests -DskipITs -Dhive11* > > > > > > B) Try to shade/bundle more jars to try and make master work. Current > > > bundles don't include parquet - either in utilities-bundle for hive > > syncing > > > or mr-bundle for queries. This amounts to including jars > > > > > > https://github.com/apache/incubator-hudi/blob/master/packaging/hudi-hadoop-mr-bundle/pom.xml#L66 > > > here. > > > > > > On Fri, Oct 18, 2019 at 8:50 AM Vinoth Chandar <[email protected]> > > wrote: > > > > > >> Looks like this is similar to the Hive 1.x issue faced in another > > thread. > > >> > > >> Let me think through this and get back to you. We need to reverse > trace > > >> change we did to drop hive 1.x support and pom redesign changes > > >> > > >> On Wed, Oct 16, 2019 at 6:53 AM Gurudatt Kulkarni < > [email protected]> > > >> wrote: > > >> > > >>> Hi Nishith, > > >>> > > >>> I tried the approach that you suggested, no luck. The writes were > > >>> successful but the hive-sync fails. > > >>> > > >>> Here's the stack trace that I am getting. Its mostly version mismatch > > >>> issue > > >>> from what I searched. > > >>> > > >>> ``` > > >>> 19/10/16 18:31:42 ERROR jdbc.HiveConnection: Error opening session > > >>> org.apache.thrift.TApplicationException: Required field > > 'client_protocol' > > >>> is unset! Struct:TOpenSessionReq(client_protocol:null, > > >>> > > >>> > > > configuration:{set:hiveconf:hive.server2.thrift.resultset.default.fetch.size=1000, > > >>> use:database=default}) > > >>> at > > >>> > > >>> > > > org.apache.thrift.TApplicationException.read(TApplicationException.java:111) > > >>> at > org.apache.thrift.TServiceClient.receiveBase(TServiceClient.java:79) > > >>> at > > >>> > > >>> > > > org.apache.hudi.org.apache.hive.service.rpc.thrift.TCLIService$Client.recv_OpenSession(TCLIService.java:168) > > >>> at > > >>> > > >>> > > > org.apache.hudi.org.apache.hive.service.rpc.thrift.TCLIService$Client.OpenSession(TCLIService.java:155) > > >>> at > > >>> > > >>> > > > org.apache.hudi.org.apache.hive.jdbc.HiveConnection.openSession(HiveConnection.java:680) > > >>> at > > >>> > > >>> > > > org.apache.hudi.org.apache.hive.jdbc.HiveConnection.<init>(HiveConnection.java:200) > > >>> at > > >>> > > >>> > > > org.apache.hudi.org.apache.hive.jdbc.HiveDriver.connect(HiveDriver.java:107) > > >>> at java.sql.DriverManager.getConnection(DriverManager.java:664) > > >>> at java.sql.DriverManager.getConnection(DriverManager.java:247) > > >>> at > > >>> > > >>> > > > org.apache.hudi.hive.HoodieHiveClient.createHiveConnection(HoodieHiveClient.java:544) > > >>> at > > org.apache.hudi.hive.HoodieHiveClient.<init>(HoodieHiveClient.java:106) > > >>> at org.apache.hudi.hive.HiveSyncTool.<init>(HiveSyncTool.java:60) > > >>> at > > >>> > > >>> > > > org.apache.hudi.utilities.deltastreamer.DeltaSync.syncHive(DeltaSync.java:440) > > >>> at > > >>> > > >>> > > > org.apache.hudi.utilities.deltastreamer.DeltaSync.writeToSink(DeltaSync.java:382) > > >>> at > > >>> > > >>> > > > org.apache.hudi.utilities.deltastreamer.DeltaSync.syncOnce(DeltaSync.java:226) > > >>> at > > >>> > > >>> > > > org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer.sync(HoodieDeltaStreamer.java:120) > > >>> at > > >>> > > >>> > > > org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer.main(HoodieDeltaStreamer.java:292) > > >>> 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:498) > > >>> at > > >>> > > >>> > > > org.apache.spark.deploy.yarn.ApplicationMaster$$anon$2.run(ApplicationMaster.scala:688) > > >>> ``` > > >>> > > >>> Regards, > > >>> Gurudatt > > >>> > > >>> On Tue, Oct 15, 2019 at 9:50 PM nishith agarwal <[email protected] > > > > >>> wrote: > > >>> > > >>> > Gurudatt, > > >>> > > > >>> > Hudi master moved away from Hive 1.x to Hive 2.x. However, can you > > try > > >>> to > > >>> > build master and provide a connection URL to the Hive 1.x > > metastore/hive > > >>> > server ? > > >>> > > > >>> > Thanks, > > >>> > Nishith > > >>> > > > >>> > > > >>> > > > >>> > On Tue, Oct 15, 2019 at 9:11 AM Vinoth Chandar <[email protected]> > > >>> wrote: > > >>> > > > >>> > > Ouch.. We dropped support for Hive 1.x recently. But Hive 1.2.x > > might > > >>> > still > > >>> > > work. Is there a possibility of going one minor version up? > > >>> > > > > >>> > > Balaji and Nishith know the gory details, and possible deal with. > > Hive > > >>> > 1.x > > >>> > > now and then. > > >>> > > Folks, Is there any chance to make master work with Hive 1.x with > > some > > >>> > > custom changes? > > >>> > > > > >>> > > On Mon, Oct 14, 2019 at 10:11 PM Gurudatt Kulkarni < > > >>> [email protected]> > > >>> > > wrote: > > >>> > > > > >>> > > > Hi Vinoth, > > >>> > > > > > >>> > > > Thank you for the quick response, but using the master branch > > would > > >>> > mean > > >>> > > > building for Hive 2.X, but we are still working on Hive 1.1.0 > :( > > >>> > > > > > >>> > > > > > >>> > > > On Mon, Oct 14, 2019 at 7:57 PM Vinoth Chandar < > > [email protected]> > > >>> > > wrote: > > >>> > > > > > >>> > > > > Hi Gurudatt, > > >>> > > > > > > >>> > > > > Thanks for reporting this. This seems like a class mismatch > > issue > > >>> > (the > > >>> > > > > particular stack trace). master and the next org.apache.hudi > > >>> release > > >>> > > has > > >>> > > > > tons of fixes around this. Could you give master branch a > shot > > by > > >>> > > > building > > >>> > > > > it yourself? > > >>> > > > > < > > >>> > > > > > > >>> > > > > > >>> > > > > >>> > > > >>> > > > http://mail-archives.apache.org/mod_mbox/hudi-dev/201906.mbox/%3CCADTZSaV9GO=3uymyzy6vidjo_va-_98tqalokmmoakbcc1g...@mail.gmail.com%3E > > >>> > > > > > > > >>> > > > > > > >>> > > > > To achieve what you are trying to do, please see this old > > thread. > > >>> > > > > > > >>> > > > > > > >>> > > > > > >>> > > > > >>> > > > >>> > > > http://mail-archives.apache.org/mod_mbox/hudi-dev/201906.mbox/%3CCADTZSaV9GO=3uymyzy6vidjo_va-_98tqalokmmoakbcc1g...@mail.gmail.com%3E > > >>> > > > > > > >>> > > > > You also need to set the following properties. > > >>> > > > > > > >>> > > > > > > >>> > > > > > > >>> > > > > > >>> > > > > >>> > > > >>> > > > hoodie.datasource.write.keygenerator.class=org.apache.hudi.NonpartitionedKeyGenerator > > >>> > > > > > > >>> > > > > > > >>> > > > > > > >>> > > > > > >>> > > > > >>> > > > >>> > > > hoodie.datasource.hive_sync.partition_extractor_class=org.apache.hudi.hive.NonPartitionedExtractor > > >>> > > > > > > >>> > > > > > > >>> > > > > Balaji ,Can we faq your answer on that thread, since this an > > often > > >>> > > > > asked question? > > >>> > > > > > > >>> > > > > > > >>> > > > > Thanks > > >>> > > > > > > >>> > > > > vinoth > > >>> > > > > > > >>> > > > > > > >>> > > > > > > >>> > > > > On Mon, Oct 14, 2019 at 4:58 AM Gurudatt Kulkarni < > > >>> > [email protected] > > >>> > > > > > >>> > > > > wrote: > > >>> > > > > > > >>> > > > > > Hi All, > > >>> > > > > > > > >>> > > > > > I am using HoodieDeltaStreamer (hoodie-0.4.7) to migrate a > > small > > >>> > > table. > > >>> > > > > The > > >>> > > > > > data is being written successfully in parquet format but > the > > >>> hive > > >>> > > sync > > >>> > > > > > fails. > > >>> > > > > > > > >>> > > > > > Here's the Stacktrace. > > >>> > > > > > > > >>> > > > > > 19/10/14 17:02:12 INFO metastore.ObjectStore: Setting > > MetaStore > > >>> > > object > > >>> > > > > > pin classes with > > >>> > > > > > > > >>> > > > > > > > >>> > > > > > > >>> > > > > > >>> > > > > >>> > > > >>> > > > hive.metastore.cache.pinobjtypes="Table,StorageDescriptor,SerDeInfo,Partition,Database,Type,FieldSchema,Order" > > >>> > > > > > 19/10/14 17:02:12 ERROR yarn.ApplicationMaster: User class > > threw > > >>> > > > > > exception: java.lang.ClassCastException: > > >>> > > > > > > > >>> > > > > >>> > org.apache.hadoop.hive.ql.optimizer.ppr.PartitionExpressionForMetastore > > >>> > > > > > cannot be cast to > > >>> > > > > > > > >>> > > > > > >>> > > > >>> > > com.uber.hoodie.org.apache.hadoop_hive.metastore.PartitionExpressionProxy > > >>> > > > > > java.lang.ClassCastException: > > >>> > > > > > > > >>> > > > > >>> > org.apache.hadoop.hive.ql.optimizer.ppr.PartitionExpressionForMetastore > > >>> > > > > > cannot be cast to > > >>> > > > > > > > >>> > > > > > >>> > > > >>> > > com.uber.hoodie.org.apache.hadoop_hive.metastore.PartitionExpressionProxy > > >>> > > > > > at > > >>> > > > > > > > >>> > > > > > > >>> > > > > > >>> > > > > >>> > > > >>> > > > com.uber.hoodie.org.apache.hadoop_hive.metastore.ObjectStore.createExpressionProxy(ObjectStore.java:367) > > >>> > > > > > at > > >>> > > > > > > > >>> > > > > > > >>> > > > > > >>> > > > > >>> > > > >>> > > > com.uber.hoodie.org.apache.hadoop_hive.metastore.ObjectStore.initialize(ObjectStore.java:345) > > >>> > > > > > at > > >>> > > > > > > > >>> > > > > > > >>> > > > > > >>> > > > > >>> > > > >>> > > > com.uber.hoodie.org.apache.hadoop_hive.metastore.ObjectStore.setConf(ObjectStore.java:298) > > >>> > > > > > at > > >>> > > > > > > > >>> > > > > >>> > org.apache.hadoop.util.ReflectionUtils.setConf(ReflectionUtils.java:73) > > >>> > > > > > at > > >>> > > > > > > > >>> > > > > > > >>> > > > > > >>> > > > > >>> > > > >>> > > > org.apache.hadoop.util.ReflectionUtils.newInstance(ReflectionUtils.java:133) > > >>> > > > > > at > > >>> > > > > > > > >>> > > > > > > >>> > > > > > >>> > > > > >>> > > > >>> > > > com.uber.hoodie.org.apache.hadoop_hive.metastore.RawStoreProxy.<init>(RawStoreProxy.java:60) > > >>> > > > > > at > > >>> > > > > > > > >>> > > > > > > >>> > > > > > >>> > > > > >>> > > > >>> > > > com.uber.hoodie.org.apache.hadoop_hive.metastore.RawStoreProxy.getProxy(RawStoreProxy.java:69) > > >>> > > > > > at > > >>> > > > > > > > >>> > > > > > > >>> > > > > > >>> > > > > >>> > > > >>> > > > com.uber.hoodie.org.apache.hadoop_hive.metastore.HiveMetaStore$HMSHandler.newRawStore(HiveMetaStore.java:682) > > >>> > > > > > at > > >>> > > > > > > > >>> > > > > > > >>> > > > > > >>> > > > > >>> > > > >>> > > > com.uber.hoodie.org.apache.hadoop_hive.metastore.HiveMetaStore$HMSHandler.getMS(HiveMetaStore.java:660) > > >>> > > > > > at > > >>> > > > > > > > >>> > > > > > > >>> > > > > > >>> > > > > >>> > > > >>> > > > com.uber.hoodie.org.apache.hadoop_hive.metastore.HiveMetaStore$HMSHandler.createDefaultDB(HiveMetaStore.java:709) > > >>> > > > > > at > > >>> > > > > > > > >>> > > > > > > >>> > > > > > >>> > > > > >>> > > > >>> > > > com.uber.hoodie.org.apache.hadoop_hive.metastore.HiveMetaStore$HMSHandler.init(HiveMetaStore.java:508) > > >>> > > > > > at > > >>> > > > > > > > >>> > > > > > > >>> > > > > > >>> > > > > >>> > > > >>> > > > com.uber.hoodie.org.apache.hadoop_hive.metastore.RetryingHMSHandler.<init>(RetryingHMSHandler.java:78) > > >>> > > > > > at > > >>> > > > > > > > >>> > > > > > > >>> > > > > > >>> > > > > >>> > > > >>> > > > com.uber.hoodie.org.apache.hadoop_hive.metastore.RetryingHMSHandler.getProxy(RetryingHMSHandler.java:84) > > >>> > > > > > at > > >>> > > > > > > > >>> > > > > > > >>> > > > > > >>> > > > > >>> > > > >>> > > > com.uber.hoodie.org.apache.hadoop_hive.metastore.HiveMetaStore.newRetryingHMSHandler(HiveMetaStore.java:6481) > > >>> > > > > > at > > >>> > > > > > > > >>> > > > > > > >>> > > > > > >>> > > > > >>> > > > >>> > > > com.uber.hoodie.org.apache.hadoop_hive.metastore.HiveMetaStoreClient.<init>(HiveMetaStoreClient.java:207) > > >>> > > > > > at > > >>> > > > > > > > >>> > > > > > > >>> > > > > > >>> > > > > >>> > > > >>> > > > com.uber.hoodie.org.apache.hadoop_hive.metastore.HiveMetaStoreClient.<init>(HiveMetaStoreClient.java:187) > > >>> > > > > > at > > >>> > > > > > > > >>> > > > > >>> > com.uber.hoodie.hive.HoodieHiveClient.<init>(HoodieHiveClient.java:102) > > >>> > > > > > at > > >>> > > > com.uber.hoodie.hive.HiveSyncTool.<init>(HiveSyncTool.java:61) > > >>> > > > > > at > > >>> > > > > > > > >>> > > > > > > >>> > > > > > >>> > > > > >>> > > > >>> > > > com.uber.hoodie.utilities.deltastreamer.HoodieDeltaStreamer.syncHive(HoodieDeltaStreamer.java:328) > > >>> > > > > > at > > >>> > > > > > > > >>> > > > > > > >>> > > > > > >>> > > > > >>> > > > >>> > > > com.uber.hoodie.utilities.deltastreamer.HoodieDeltaStreamer.sync(HoodieDeltaStreamer.java:298) > > >>> > > > > > at > > >>> > > > > > > > >>> > > > > > > >>> > > > > > >>> > > > > >>> > > > >>> > > > com.uber.hoodie.utilities.deltastreamer.HoodieDeltaStreamer.main(HoodieDeltaStreamer.java:469) > > >>> > > > > > 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:498) > > >>> > > > > > at > > >>> > > > > > > > >>> > > > > > > >>> > > > > > >>> > > > > >>> > > > >>> > > > org.apache.spark.deploy.yarn.ApplicationMaster$$anon$2.run(ApplicationMaster.scala:688) > > >>> > > > > > > > >>> > > > > > > > >>> > > > > > Here's are the properties that I am using, > > >>> > > > > > > > >>> > > > > > > > >>> > > > > > ``` > > >>> > > > > > > > >>> > > > > > hoodie.upsert.shuffle.parallelism=2 > > >>> > > > > > hoodie.insert.shuffle.parallelism=2 > > >>> > > > > > hoodie.bulkinsert.shuffle.parallelism=2 > > >>> > > > > > > > >>> > > > > > # Key fields, for kafka example > > >>> > > > > > hoodie.datasource.write.recordkey.field=<primary_key> > > >>> > > > > > *hoodie.datasource.write.partitionpath.field=* > > >>> > > > > > # schema provider configs > > >>> > > > > > hoodie.deltastreamer.schemaprovider.registry.url= > > >>> > > > > > http://localhost:8081/subjects/schema_name/versions/latest > > >>> > > > > > # Kafka Source > > >>> > > > > > hoodie.datasource.hive_sync.database=default > > >>> > > > > > hoodie.datasource.hive_sync.table=table_name > > >>> > > > > > > > hoodie.datasource.hive_sync.jdbcurl=jdbc:hive2://localhost:10000 > > >>> > > > > > *hoodie.datasource.hive_sync.partition_fields=* > > >>> > > > > > > > >>> > > > > > hoodie.deltastreamer.source.kafka.topic=topic_name > > >>> > > > > > #Kafka props > > >>> > > > > > metadata.broker.list=localhost:9092 > > >>> > > > > > auto.offset.reset=smallest > > >>> > > > > > schema.registry.url=http://localhost:8081 > > >>> > > > > > > > >>> > > > > > ``` > > >>> > > > > > > > >>> > > > > > > > >>> > > > > > The table does not have partitons, hence i have kept > > >>> > > > > > *hoodie.datasource.write.partitionpath.field *blank, > > >>> > > > > > > > >>> > > > > > so it is writing it to `default` directory. > > >>> > > > > > > > >>> > > > > > Also, *hoodie.datasource.hive_sync.partition_fields > > *property is > > >>> > left > > >>> > > > > > blank for the same reason. > > >>> > > > > > > > >>> > > > > > > > >>> > > > > > Regards, > > >>> > > > > > > > >>> > > > > > Gurudatt > > >>> > > > > > > > >>> > > > > > > >>> > > > > > >>> > > > > >>> > > > >>> > > >> > > > >
