[
https://issues.apache.org/jira/browse/HIVE-20911?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
anishek updated HIVE-20911:
---------------------------
Attachment: HIVE-20911.04.patch
> External Table Replication for Hive
> -----------------------------------
>
> Key: HIVE-20911
> URL: https://issues.apache.org/jira/browse/HIVE-20911
> Project: Hive
> Issue Type: Bug
> Components: HiveServer2
> Affects Versions: 4.0.0
> Reporter: anishek
> Assignee: anishek
> Priority: Critical
> Labels: pull-request-available
> Fix For: 4.0.0
>
> Attachments: HIVE-20911.01.patch, HIVE-20911.02.patch,
> HIVE-20911.03.patch, HIVE-20911.04.patch
>
>
> External tables are not replicated currently as part of hive replication. As
> part of this jira we want to enable that.
> Approach:
> * Target cluster will have a top level base directory config that will be
> used to copy all data relevant to external tables. This will be provided via
> the *with* clause in the *repl load* command. This base path will be prefixed
> to the path of the same external table on source cluster.
> * Since changes to directories on the external table can happen without hive
> knowing it, hence we cant capture the relevant events when ever new data is
> added or removed, we will have to copy the data from the source path to
> target path for external tables every time we run incremental replication.
> ** this will require incremental *repl dump* to now create an additional
> file *\_external\_tables\_info* with data in the following form
> {code}
> tableName,base64Encoded(tableDataLocation)
> {code}
> In case there are different partitions in the table pointing to different
> locations there will be multiple entries in the file for the same table name
> with location pointing to different partition locations. For partitions
> created in a table without specifying the _set location_ command will be
> within the same table Data location and hence there will not be different
> entries in the file above
> ** *repl load* will read the *\_external\_tables\_info* to identify what
> locations are to be copied from source to target and create corresponding
> tasks for them.
> * New External tables will be created with metadata only with no data copied
> as part of regular tasks while incremental load/bootstrap load.
> * Bootstrap dump will also create *\_external\_tables\_info* which will be
> used to copy data from source to target as part of boostrap load.
> * Bootstrap load will create a DAG, that can use parallelism in the execution
> phase, the hdfs copy related tasks are created, once the bootstrap phase is
> complete.
> * Since incremental load results in a DAG with only sequential execution (
> events applied in sequence ) to effectively use the parallelism capability in
> execution mode, we create tasks for hdfs copy along with the incremental DAG.
> This requires a few basic calculations to approximately meet the configured
> value in "hive.repl.approx.max.load.tasks"
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)