yihua commented on issue #5485:
URL: https://github.com/apache/hudi/issues/5485#issuecomment-1115491470
@leobiscassi Thanks for providing the detailed steps. There are three
things:
(1) When using Deltastreamer, the `s3a://` prefix should be used, to avoid
`_$folder$` suffix.
(2) Hudi 0.9.0-amzn-1 does not support `date`-typed partition field. The
support is only added recently #5432. However, you can still using
String-typed partition field.
(3) the parquet files you generated do not have the `date` field in the
schema, i.e., when each individual parquet file is directly read, the `date`
field is not there. `ParquetDFSSource` directly reads each parquet file and
does not recover the partition path from the `sample-data` path. That's why
you see `1970-01-01` as the partition path since the `date` is not found and
the default value is that.
Below is an example that can actually achieve what you need:
```
from pyspark.sql import SparkSession
from datetime import date
data = [
{'date': date(2022, 1, 5), 'date2': '2022-01-05', 'ts':
'2022-04-10T09:47:54+00:00', 'name': 'Fake Name 1', 'email':
'[email protected]'},
{'date': date(2022, 1, 4), 'date2': '2022-01-04', 'ts':
'2022-04-10T09:47:54+00:00', 'name': 'Fake Name 2', 'email':
'[email protected]'},
{'date': date(2022, 1, 3), 'date2': '2022-01-03', 'ts':
'2022-04-10T09:47:54+00:00', 'name': 'Fake Name 3', 'email':
'[email protected]'},
{'date': date(2022, 2, 5), 'date2': '2022-02-05', 'ts':
'2022-04-10T09:47:54+00:00', 'name': 'Fake Name 4', 'email':
'[email protected]'},
{'date': date(2022, 3, 5), 'date2': '2022-03-05', 'ts':
'2022-04-10T09:47:54+00:00', 'name': 'Fake Name 5', 'email':
'[email protected]'},
{'date': date(2022, 5, 10), 'date2': '2022-05-10', 'ts':
'2022-04-10T09:47:54+00:00', 'name': 'Fake Name 6', 'email':
'[email protected]'},
{'date': date(2022, 5, 1), 'date2': '2022-05-01', 'ts':
'2022-04-10T09:47:54+00:00', 'name': 'Fake Name 7', 'email':
'[email protected]'},
]
spark = SparkSession.builder.getOrCreate()
df = spark.createDataFrame(data)
df.write.partitionBy('date').parquet('sample-data')
```
The above data is written to `s3a://hudi-testing-tmp/sample-data-hudi/`.
```
spark-submit --class
org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer \
--jars /usr/lib/spark/external/lib/spark-avro.jar \
--master yarn \
--deploy-mode client \
--conf spark.sql.hive.convertMetastoreParquet=false
/usr/lib/hudi/hudi-utilities-bundle.jar \
--table-type COPY_ON_WRITE \
--source-ordering-field ts \
--source-class
org.apache.hudi.utilities.sources.ParquetDFSSource \
--target-table sample_data_custom \
--target-base-path s3a://hudi-testing-tmp/sample-data-hudi/ \
--hoodie-conf
hoodie.deltastreamer.source.dfs.root=s3a://hudi-testing-tmp/sample-data3/ \
--hoodie-conf hoodie.datasource.write.recordkey.field=ts,email \
--hoodie-conf
hoodie.datasource.write.hive_style_partitioning=true \
--op UPSERT \
--hoodie-conf
hoodie.datasource.write.partitionpath.field=date2:timestamp \
--hoodie-conf
hoodie.deltastreamer.keygen.timebased.timestamp.type=DATE_STRING \
--hoodie-conf
hoodie.deltastreamer.keygen.timebased.input.dateformat="yyyy-MM-dd" \
--hoodie-conf
hoodie.deltastreamer.keygen.timebased.output.dateformat="yyyy-MM-dd" \
--hoodie-conf
hoodie.datasource.write.keygenerator.class=org.apache.hudi.keygen.CustomKeyGenerator
```
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]