|
22 | 22 | import org.apache.hudi.client.SparkRDDWriteClient; |
23 | 23 | import org.apache.hudi.client.common.HoodieSparkEngineContext; |
24 | 24 | import org.apache.hudi.common.config.TypedProperties; |
| 25 | +import org.apache.hudi.common.model.HoodieTimelineTimeZone; |
25 | 26 | import org.apache.hudi.common.table.HoodieTableConfig; |
26 | 27 | import org.apache.hudi.common.table.HoodieTableMetaClient; |
27 | 28 | import org.apache.hudi.common.util.Option; |
| 29 | +import org.apache.hudi.common.util.StringUtils; |
28 | 30 | import org.apache.hudi.common.util.ValidationUtils; |
29 | 31 | import org.apache.hudi.config.HoodieCompactionConfig; |
30 | 32 | import org.apache.hudi.config.HoodieIndexConfig; |
|
33 | 35 | import org.apache.hudi.hive.HiveSyncConfig; |
34 | 36 | import org.apache.hudi.hive.HiveSyncTool; |
35 | 37 | import org.apache.hudi.index.HoodieIndex; |
| 38 | +import org.apache.hudi.keygen.NonpartitionedKeyGenerator; |
| 39 | +import org.apache.hudi.keygen.SimpleKeyGenerator; |
| 40 | +import org.apache.hudi.util.SparkKeyGenUtils; |
36 | 41 | import org.apache.hudi.utilities.UtilHelpers; |
37 | 42 | import org.apache.hudi.utilities.schema.SchemaProvider; |
38 | 43 |
|
|
48 | 53 | import java.util.HashMap; |
49 | 54 |
|
50 | 55 | import static org.apache.hudi.common.table.HoodieTableConfig.ARCHIVELOG_FOLDER; |
| 56 | +import static org.apache.hudi.common.table.HoodieTableConfig.PARTITION_METAFILE_USE_BASE_FORMAT; |
| 57 | +import static org.apache.hudi.common.table.HoodieTableConfig.POPULATE_META_FIELDS; |
| 58 | +import static org.apache.hudi.common.table.HoodieTableConfig.TIMELINE_TIMEZONE; |
| 59 | +import static org.apache.hudi.config.HoodieBootstrapConfig.KEYGEN_CLASS_NAME; |
| 60 | +import static org.apache.hudi.config.HoodieWriteConfig.PRECOMBINE_FIELD_NAME; |
51 | 61 | import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_SYNC_BUCKET_SYNC; |
52 | 62 | import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_SYNC_BUCKET_SYNC_SPEC; |
| 63 | +import static org.apache.hudi.keygen.constant.KeyGeneratorOptions.HIVE_STYLE_PARTITIONING_ENABLE; |
| 64 | +import static org.apache.hudi.keygen.constant.KeyGeneratorOptions.RECORDKEY_FIELD_NAME; |
| 65 | +import static org.apache.hudi.keygen.constant.KeyGeneratorOptions.URL_ENCODE_PARTITIONING; |
53 | 66 | import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_BASE_FILE_FORMAT; |
54 | 67 | import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_BASE_PATH; |
55 | 68 |
|
@@ -187,16 +200,44 @@ private void initializeTable() throws IOException { |
187 | 200 | + ". Cannot bootstrap data on top of an existing table"); |
188 | 201 | } |
189 | 202 | } |
190 | | - HoodieTableMetaClient.withPropertyBuilder() |
| 203 | + |
| 204 | + HoodieTableMetaClient.PropertyBuilder builder = HoodieTableMetaClient.withPropertyBuilder() |
191 | 205 | .fromProperties(props) |
192 | 206 | .setTableType(cfg.tableType) |
193 | 207 | .setTableName(cfg.targetTableName) |
194 | | - .setArchiveLogFolder(ARCHIVELOG_FOLDER.defaultValue()) |
| 208 | + .setRecordKeyFields(props.getString(RECORDKEY_FIELD_NAME.key())) |
| 209 | + .setPreCombineField(props.getString( |
| 210 | + PRECOMBINE_FIELD_NAME.key(), PRECOMBINE_FIELD_NAME.defaultValue())) |
| 211 | + .setPopulateMetaFields(props.getBoolean( |
| 212 | + POPULATE_META_FIELDS.key(), POPULATE_META_FIELDS.defaultValue())) |
| 213 | + .setArchiveLogFolder(props.getString( |
| 214 | + ARCHIVELOG_FOLDER.key(), ARCHIVELOG_FOLDER.defaultValue())) |
195 | 215 | .setPayloadClassName(cfg.payloadClassName) |
196 | 216 | .setBaseFileFormat(cfg.baseFileFormat) |
197 | 217 | .setBootstrapIndexClass(cfg.bootstrapIndexClass) |
198 | 218 | .setBootstrapBasePath(bootstrapBasePath) |
199 | | - .initTable(new Configuration(jssc.hadoopConfiguration()), cfg.targetBasePath); |
| 219 | + .setHiveStylePartitioningEnable(props.getBoolean( |
| 220 | + HIVE_STYLE_PARTITIONING_ENABLE.key(), |
| 221 | + Boolean.parseBoolean(HIVE_STYLE_PARTITIONING_ENABLE.defaultValue()) |
| 222 | + )) |
| 223 | + .setUrlEncodePartitioning(props.getBoolean( |
| 224 | + URL_ENCODE_PARTITIONING.key(), |
| 225 | + Boolean.parseBoolean(URL_ENCODE_PARTITIONING.defaultValue()))) |
| 226 | + .setCommitTimezone(HoodieTimelineTimeZone.valueOf( |
| 227 | + props.getString( |
| 228 | + TIMELINE_TIMEZONE.key(), |
| 229 | + String.valueOf(TIMELINE_TIMEZONE.defaultValue())))) |
| 230 | + .setPartitionMetafileUseBaseFormat(props.getBoolean( |
| 231 | + PARTITION_METAFILE_USE_BASE_FORMAT.key(), |
| 232 | + PARTITION_METAFILE_USE_BASE_FORMAT.defaultValue())); |
| 233 | + String partitionColumns = SparkKeyGenUtils.getPartitionColumns(props); |
| 234 | + if (!StringUtils.isNullOrEmpty(partitionColumns)) { |
| 235 | + builder.setPartitionFields(partitionColumns).setKeyGeneratorClassProp(props.getString(KEYGEN_CLASS_NAME.key(), SimpleKeyGenerator.class.getName())); |
| 236 | + } else { |
| 237 | + builder.setKeyGeneratorClassProp(props.getString(KEYGEN_CLASS_NAME.key(), NonpartitionedKeyGenerator.class.getName())); |
| 238 | + } |
| 239 | + |
| 240 | + builder.initTable(new Configuration(jssc.hadoopConfiguration()), cfg.targetBasePath); |
200 | 241 | } |
201 | 242 |
|
202 | 243 | public HoodieWriteConfig getBootstrapConfig() { |
|
0 commit comments