-
Notifications
You must be signed in to change notification settings - Fork 2.6k
Flink : migrate hive table to iceberg table #2217
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Conversation
@openinx @stevenzwu could you help me review it ? thanks |
|
||
@Test | ||
public void testMigrateUnpartition() throws IOException, TableNotExistException { | ||
getTableEnv().getConfig().setSqlDialect(SqlDialect.HIVE); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
iceberg orc use orc-core-nohive ,flink use orc-core. They will have some conflicts, so I can not use sql to write data into hive tables
94f6f7c
to
5b86272
Compare
// hive source table | ||
ObjectPath tableSource = new ObjectPath(hiveSourceDbName, hiveSourceTableName); | ||
org.apache.hadoop.hive.metastore.api.Table hiveTable; | ||
flinkHiveCatalog.open(); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
we need to close the catalog in the end
Schema icebergSchema = HiveSchemaUtil.convert(fieldSchemaList); | ||
PartitionSpec spec = toPartitionSpec(partitionList, icebergSchema); | ||
|
||
String hiveFormat = hiveTable.getSd().getInputFormat(); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
this method is getting very long. maybe extract this fileFormat logic to a util class/method?
return files.size(); | ||
} | ||
|
||
private Namespace toNamespace(Namespace namespace) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
to be consistent, I would suggest we also pass in the icebergDbName
as arg.
private final String hiveSourceTableName; | ||
private final Catalog icebergCatalog; | ||
private final Namespace baseNamespace; | ||
private final String icebergDbName; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
nit: it seems that Iceberg convention is to have more full name. like Db
-> Database
. sam thing for the hiveSourceDbName
above
} | ||
|
||
private Namespace toNamespace(Namespace namespace) { | ||
if (namespace.isEmpty()) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
one probably slightly simpler approach is to use List to avoid the if-else. First add the levels from baseNamespace
. Then add the icebergDbName
. Then toArray
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Actually, I found a simpler one-liner.
Namespace.of(ArrayUtils.concat(baseNamespace.levels(), new String[] {DATABASE}))
} | ||
|
||
AppendFiles append = icebergTable.newAppend(); | ||
files.forEach(append::appendFile); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I am concerned about how this works for huge tables with millions of files.
- collecting all of them into a single list can cause memory pressure.
- writing all of them into a single manifest file may also hurt read performance for partitioned tables, as we can't filter manifest files by partition value ranges in manifest list file.
Maybe we should commit the files by batches. Each Hive partition is a batch and all data files under one Hive partitions get written into a single manifest file. We can use AppendFiles#appendManifest
API too.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
thanks for your suggesion,I will refactor this logic
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
- collecting all of them into a single list can cause memory pressure.
I looked at the code again, the listStatus
method of hadoop filesystem use ArrayList
to construct the FileStatus
array.
public FileStatus[] listStatus(Path f, PathFilter filter)
throws FileNotFoundException, IOException {
ArrayList<FileStatus> results = new ArrayList<FileStatus>();
listStatus(results, f, filter);
return results.toArray(new FileStatus[results.size()]);
}
although there is the org.apache.hadoop.fs.FileSystem#listStatusIterator
method, the listStatus
method is still called in the internal of FileSystem#listStatusIterator
method, so when we get the file list of hive, even if we use Iterator, I think it is still useless.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
After the refactoring of emitting ManifestFile per partition, we are only listing the files under one partition. Then I won't be concerned for the list. Previously, we are collecting all files under a table.
|
||
List<DataFile> files = null; | ||
if (spec.isUnpartitioned()) { | ||
MigrateMap migrateMap = new MigrateMap(spec, nameMapping, fileFormat, metricsConfig); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
extract this to a separate method migrateUnpartitionedTable
?
|
||
DataStream<PartitionAndLocation> dataStream = env.fromCollection(inputs); | ||
MigrateMap migrateMap = new MigrateMap(spec, nameMapping, fileFormat, metricsConfig); | ||
DataStream<List<DataFile>> ds = dataStream.map(migrateMap).setParallelism(parallelism); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
As mentioned in the another comment, we can parallelize the processing of each Hive partition and generate a manifest file for each partition on taskmanagers. That is probably the more expensive part of migration. Then on the jobmanager, we can just keep appendManifest
and commit
once.
.flatMap(Collection::stream).collect(Collectors.toList()); | ||
} | ||
|
||
private static class MigrateMap implements MapFunction<PartitionAndLocation, List<DataFile>> { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
nit: Map -> Mapper? Map
gives the impression of Java Collection.
Here is the Flink DataStream
map API
map(MapFunction<T, R> mapper)
Configuration conf = HadoopUtils.getHadoopConfiguration(GlobalConfiguration.loadConfiguration()); | ||
NameMapping nameMapping = nameMappingString != null ? NameMappingParser.fromJson(nameMappingString) : null; | ||
List<DataFile> files; | ||
switch (fileFormat) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
could a table have mixed file formats?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I lookup the hive doc, when we create a hive table , specify the table format by STORED AS file_format
, and the table format is table level properties. I think we cannot create different formats for the same table.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
One can change the InputFormat/OutputFormat, but that will result in a "dead" table. So it is ok to expect only a single file format.
|
||
return DataFiles.builder(partitionSpec) | ||
.withPath(stat.getPath().toString()) | ||
.withFormat(FileFormat.ORC.name()) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
nit: there is a withFormat
overload that takes the enum directly. we don't have to call .name()
private String location; | ||
private Map<String, String> map; | ||
|
||
public PartitionAndLocation(String location, Map<String, String> map) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
nit: for better readability, should we rename map
to partitionSpec
?
@BeforeClass | ||
public static void createHiveDB() { | ||
try { | ||
TestMigrateAction.metastoreClient = new HiveMetaStoreClient(hiveConf); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
nit: drop the TestMigrateAction.
} | ||
|
||
@AfterClass | ||
public static void dropHiveDB() { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
we need to close HiveMetaStoreClient
, right?
exec("USE CATALOG %s", catalogName); | ||
exec("CREATE DATABASE IF NOT EXISTS %s", DATABASE); | ||
|
||
flinkHiveCatalog = new HiveCatalog(SOURCE_HIVE_CATALOG_NAME, SOURCE_HIVE_DB_NAME, hiveConf, "2.3.6"); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
should we create HiveCatalog
once in beforeClass
?
} | ||
|
||
@Override | ||
protected StreamTableEnvironment getTableEnv() { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
why override it? what is the reason to enableCheckpointing
and set parallelism to 1?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
because we use StreamExecutionEnvironment
in the MigrateAction, so I use StreamExecutionEnvironment
to construct the StreamTableEnvironment
.
the parallelism
and enableCheckpointing
is my mistake ,I remove it
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Can you elaborate a little why do we need StreamExecutionEnvironment
?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
in the future, we will use StreamExecutionEnvironment
and DataStream
to do streaming and batch jobs. the DataSet
will be deprecated, (here)
This means that, in the long run, the DataSet API will be deprecated and subsumed by the DataStream API and the Table API/SQL (FLIP-131).
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I am aware of the DataSet deprecation in the future. Let me ask in a diff way. Why FlinkTestBase#getTableEnv
doesn't work for this case? TableEnvironment
interface is a higher abstraction than DataSet/DataStream
.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
because the FlinkTestBase#getTableEnv
only provide a TableEnvironment
, but in the MigrateAction
,we need a StreamExecutionEnvironment
to get DataStream
by StreamExecutionEnvironment#fromElements
.
Simply , we need to do the migrate by flink api instead of flink sql
sql("CREATE TABLE %s (id INT, data STRING) PARTITIONED BY (p STRING) STORED AS %s", SOURCE_HIVE_TABLE_NAME, | ||
format.name()); | ||
|
||
Partition hivePartition = createHivePartition(format); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
can we test the case with multiple partitions?
int migrateFileCount = | ||
Actions.migrateHive2Iceberg(env, flinkHiveCatalog, SOURCE_HIVE_DB_NAME, SOURCE_HIVE_TABLE_NAME, | ||
validationCatalog, baseNamespace, DATABASE, TABLE_NAME_UNPARTITIONED).execute(); | ||
Assert.assertEquals("Should produce the expected file count.", 1, migrateFileCount); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
if we are going to create 1 manifest per partition, then we can also add an assertion on allManifests
5b86272
to
b80f906
Compare
@stevenzwu thanks for you review,I updated all |
d61151b
to
0ce2d1b
Compare
} | ||
} catch (Exception e) { | ||
deleteManifests(icebergTable.io(), manifestFiles); | ||
throw new RuntimeException("Migrate", e); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
nit: "Failed to migrate"?
private void deleteManifests(FileIO io, List<ManifestFile> manifests) { | ||
Tasks.foreach(manifests) | ||
.noRetry() | ||
.suppressFailureWhenFinished() |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
nit: add some logging onFailure
throw new RuntimeException("Migrate", e); | ||
} | ||
|
||
AppendFiles append = icebergTable.newAppend(); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
nit: move this into the try block?
Found #2210 for Spark which tries to archive the same. We might want to synchronize our efforts. |
Also adding @RussellSpitzer who did the snapshot and migrate actions. Russell, could you take a look to make sure the migration path is similar for both query engines? |
String metadataLocation = getMetadataLocation(icebergTable); | ||
|
||
String nameMapping = | ||
PropertyUtil.propertyAsString(icebergTable.properties(), TableProperties.DEFAULT_NAME_MAPPING, null); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
in the Spark path we default to adding a mapping if none is specified. I think this is probably a good idea here as well.
iceberg/spark3/src/main/java/org/apache/iceberg/actions/Spark3CreateAction.java
Lines 174 to 180 in 04f4a30
protected void ensureNameMappingPresent(Table table) { | |
if (!table.properties().containsKey(TableProperties.DEFAULT_NAME_MAPPING)) { | |
NameMapping nameMapping = MappingUtil.create(table.schema()); | |
String nameMappingJson = NameMappingParser.toJson(nameMapping); | |
table.updateProperties().set(TableProperties.DEFAULT_NAME_MAPPING, nameMappingJson).commit(); | |
} | |
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I'm not sure if this is an issue in Hive, but in Spark there can be differences between file layout and schema of the table which lets files possibly not match the Iceberg schema. Adding in the name mapping protects against these cases
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
yes, I add the name mapping.
.getOrDefault(TableProperties.WRITE_METADATA_LOCATION, table.location() + "/" + ICEBERG_METADATA_FOLDER); | ||
} | ||
|
||
private List<ManifestFile> migrateUnpartitionedTable(PartitionSpec spec, FileFormat fileFormat, String hiveLocation, |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
One thing that we worried about here, and I'm not sure you have the same issue here, is what happens if a Hive writer is attempting to write while these migrations are occurring. We don't want to end up with dead files that a writer thinks are committed. To avoid this in the Spark Action, we move the catalog entry to a backup identifier so that all modifications will fail until the migration is complete. Basically making the old entry unavailable until we are sure it is either a Iceberg Table or we have aborted and we move it back as it's original config.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I read the code of Spark3MigrateAction
, but I am not familiar with spark, I don’t know if I understand it correctly. In Spark3MigrateAction
, source catalog and destination catalog, source table name and destination table name are the same, so after we had migrated a non-iceberg table to the iceberg table, the read and write operations using spark may not require any changes.
But for hive table and iceberg table, some changes will be needed. For example, we are using flink to read and write hive table, the sql like this: SELECT * FROM hivecatalog.hivedb.hivetable;
After the migration is completed, the sql needs to be modified to select * from iceberg_catalog.iceberg_db.iceberg_table
, so when users migrate the hive table to iceberg, they need to stop the writing hive job first, then migrate, and finally modify the logic to reading and writing iceberg table. So I think even if we renamed the source hive table, it does not make much sence to the user, because the user still needs to manually stop and modify the job of writing hive to write iceberg.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The issue is that, say the user is writing to hivetable and while a write is in progress you start the migrate operation. The write finishes and commits, but the migrate command will not pick up these new files since they were created after the migrate scanned the partitions for existing datafiles.
So you can have a successful write to the table which doesn't get migrated, I'm not sure if this is ok or not.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
So my idea is that in order to prevent this case, we recommend that users should stop writing hive job first before migrating, then do the migration, and finally modify the writing and reading job of hive.
if migrate failed, we will do the clean work on catch block, If unfortunately the clean failed, the user may need to manually clean the iceberg table and manifests.
I will add some comments on MigrateAction
,what do you think ?
@@ -346,7 +346,7 @@ public void testPartitionTypes() throws Exception { | |||
assertRecords(run(), records, typesSchema); | |||
} | |||
|
|||
static void assertRecords(List<Row> results, List<Record> expectedRecords, Schema schema) { | |||
public static void assertRecords(List<Row> results, List<Record> expectedRecords, Schema schema) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Maybe move this into a utility class now that it's public static?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
yes, I move it to SimpleDataUtil
,this method can be used for the assertion of unsorted list,I have encountered similar needs in other PR
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
We have something like it in the Hive tests as well, and I would guess Spark has one too 😄
Do we have some other place even more accessible?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@pvary , Could you give me some cases in the Hive module? Let me see if we can move them to data or core module
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@zhangjun0x01: We have 2 flavors for checking the contents of the table:
- HiveIcebergTestUtils.validateData(Table table, List expected, int sortBy)
- HiveIcebergTestUtils.validateData(List expected, List actual, int sortBy)
Thanks,
Peter
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@pvary I search the spark code , It does have a similar required, but I think we should extract the function in a new PR later ,what do you think?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I am fine with moving this refactoring to another PR.
Thanks, Peter
Really exciting to see more functionality being added here! |
aba5af7
to
509ff74
Compare
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@zhangjun0x01 I think this is a great propose that we should get this in, would you like to rebase this PR when you have time ?
public static MigrateAction migrateHive2Iceberg(StreamExecutionEnvironment env, HiveCatalog flinkHiveCatalog, | ||
String hiveSourceDatabaseName, String hiveSourceTableName, | ||
Catalog icebergCatalog, Namespace baseNamespace, | ||
String icebergDbName, String icebergTableName) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Here we pass the namespace
, database
, table
, then how about just use a TableIdentifier
?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I think the database
,table
may be more clear for new iceberg user ,because the TableIdentifier
is iceberg internal class,the user need to know how to construct the TableIdentifier
,what do you think?
} | ||
|
||
public static MigrateAction migrateHive2Iceberg(StreamExecutionEnvironment env, HiveCatalog flinkHiveCatalog, | ||
String hiveSourceDatabaseName, String hiveSourceTableName, |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Those database
& table
could just be simplified by a TableIdentifier
.
|
||
FileFormat fileFormat = getHiveFileFormat(hiveTable); | ||
|
||
Namespace namespace = Namespace.of(ArrayUtils.concat(baseNamespace.levels(), new String[] {icebergDatabaseName})); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The ArrayUtils
is a Internal
API for flink, I will suggest not to use it because it may be changed in the next release.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
yes ,I use the ObjectArrays
in google guava to replace the ArrayUtils
} | ||
|
||
Table icebergTable; | ||
if (icebergCatalog instanceof HadoopCatalog) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@zhangjun0x01 Is possible to use the FlinkCatalog
to create the table ? Then we don't have to use different code paths to create the iceberg table, I'm concerning that there will be more Catalog that will be introduced in future.
509ff74
to
a078800
Compare
@zhangjun0x01 hello,How does this example work FlinkSQL? Can you give an example?thanks |
Has this PR gone dormant due to acceptance of #2210? I'm using a Trino environment where it's easy to write Hive parquet files from pandas, but I haven't found the fast path from pandas to Iceberg. (Row-by-row insertion is too slow.) I'm here because I thought table migration might be an answer... |
We implemented Hive(external) to Iceberg migration in the Hive repo: https://issues.apache.org/jira/browse/HIVE-25008 Not sure it helps you, but just for the record is someone stumbles to this issue again |
This pull request has been marked as stale due to 30 days of inactivity. It will be closed in 1 week if no further activity occurs. If you think that’s incorrect or this pull request requires a review, please simply write any comment. If closed, you can revive the PR at any time and @mention a reviewer or discuss it on the dev@iceberg.apache.org list. Thank you for your contributions. |
This pull request has been closed due to lack of activity. This is not a judgement on the merit of the PR in any way. It is just a way of keeping the PR queue manageable. If you think that is incorrect, or the pull request requires review, you can revive the PR at any time. |
related to #2162