-
Notifications
You must be signed in to change notification settings - Fork 4.7k
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
HIVE-28793: Expire query history snapshots #5666
base: master
Are you sure you want to change the base?
Changes from all commits
e00e260
c1165fa
6577e5a
546ba2d
1509056
d26064f
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -33,6 +33,7 @@ | |
import org.apache.hadoop.hive.ql.metadata.HiveException; | ||
import org.apache.hadoop.hive.ql.metadata.HiveStorageHandler; | ||
import org.apache.hadoop.hive.ql.metadata.Table; | ||
import org.apache.hadoop.hive.ql.parse.AlterTableExecuteSpec; | ||
import org.apache.hadoop.hive.ql.plan.TableDesc; | ||
import org.apache.hadoop.hive.ql.queryhistory.schema.Record; | ||
import org.apache.hadoop.hive.ql.queryhistory.schema.Schema; | ||
|
@@ -57,6 +58,10 @@ | |
import java.util.UUID; | ||
import java.util.concurrent.ExecutorService; | ||
import java.util.concurrent.Executors; | ||
import java.util.concurrent.ScheduledExecutorService; | ||
import java.util.concurrent.TimeUnit; | ||
|
||
import static org.apache.hadoop.hive.ql.parse.AlterTableExecuteSpec.ExecuteOperationType.EXPIRE_SNAPSHOT; | ||
|
||
public class IcebergRepository extends AbstractRepository implements QueryHistoryRepository { | ||
private static final Logger LOG = LoggerFactory.getLogger(IcebergRepository.class); | ||
|
@@ -70,12 +75,17 @@ public class IcebergRepository extends AbstractRepository implements QueryHistor | |
@VisibleForTesting | ||
TableDesc tableDesc; | ||
private ExecutorService icebergExecutor; | ||
private final ScheduledExecutorService snapshotExpiryExecutor = Executors.newScheduledThreadPool(1, | ||
new ThreadFactoryBuilder().setNameFormat("IcebergRepository periodic snapshot expiry").setDaemon(true).build()); | ||
|
||
@Override | ||
public void init(HiveConf conf, Schema schema) { | ||
super.init(conf, schema); | ||
icebergExecutor = Executors.newFixedThreadPool(2, | ||
(new ThreadFactoryBuilder()).setDaemon(true).setNameFormat(ICEBERG_WORKER_THREAD_NAME_FORMAT).build()); | ||
int expiryInterval = HiveConf.getIntVar(conf, | ||
HiveConf.ConfVars.HIVE_QUERY_HISTORY_ICEBERG_SNAPSHOT_EXPIRY_INVERVAL_SECONDS); | ||
snapshotExpiryExecutor.scheduleAtFixedRate(this::expireSnapshots, 0, expiryInterval, TimeUnit.SECONDS); | ||
} | ||
|
||
@Override | ||
|
@@ -89,6 +99,7 @@ protected Table createTable(Hive hive, Database db) throws HiveException { | |
ICEBERG_STORAGE_HANDLER); | ||
table.setProperty("table_type", "ICEBERG"); | ||
table.setProperty("write.format.default", "orc"); | ||
table.setProperty("history.expire.max-snapshot-age-ms", Integer.toString(getSnapshotMaxAge())); | ||
table.setProperty(hive_metastoreConstants.META_TABLE_NAME, QUERY_HISTORY_DB_TABLE_NAME); | ||
|
||
table.setFields(schema.getFields()); | ||
|
@@ -101,8 +112,14 @@ protected Table createTable(Hive hive, Database db) throws HiveException { | |
return table; | ||
} | ||
|
||
@VisibleForTesting | ||
int getSnapshotMaxAge() { | ||
// expire/delete snapshots older than 1 day | ||
return 24 * 60 * 60 * 1000; | ||
deniskuzZ marked this conversation as resolved.
Show resolved
Hide resolved
|
||
} | ||
|
||
@Override | ||
protected void postInitTable(Table table) throws Exception { | ||
protected void postInitTable() throws Exception { | ||
this.tableDesc = Utilities.getTableDesc(table); | ||
|
||
Map<String, String> map = new HashMap<>(); | ||
|
@@ -176,4 +193,13 @@ private void prepareConfForWrite() { | |
SessionStateUtil.addCommitInfo(SessionState.getSessionConf(), tableDesc.getTableName(), jobId, 1, | ||
Maps.fromProperties(tableDesc.getProperties())); | ||
} | ||
|
||
private void expireSnapshots() { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. We have an You can define the expiry policy for the There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. yeah, theoretically, handling them in a centralized place in metastore instead of multiple HS2s looks better from system design point of view There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. no need for a "NEW" service, add it to the list of take a look at
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. took a look, my concern about this whole service is exactly the doTheMagic part: I remember we faced lots of problems with e.g. the PartitionManagementTask because it couldn't have been run in a lightweight fashion, and the common thing is that we need to iterate on all tables of all DBs to achieve what we want here: the best case scenario is that we implement a filter on ICEBERG tables (like: listTableNamesByFilter), kinda pushing down the filter to backend db Focusing on the motivation, which is to handle the query history table—fully loaded and maintained by Hive—we owe it to users to optimize it as much as possible. However, the question is whether we should take on the additional burden in this PR to handle all Iceberg tables visible in the metastore, or keep the scope focused for now. Or maybe a similar db + table name filter (that's used in the part mgmt task), but we default it to: "sys" + "query_history" in MetastoreConf and give the user the power to set it even to There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
|
||
LOG.debug("Attempting to expire snapshots for table: {}", table.getFullTableName()); | ||
try { | ||
storageHandler.executeOperation(table, new AlterTableExecuteSpec<>(EXPIRE_SNAPSHOT, null)); | ||
} catch (Exception e) { | ||
LOG.warn("Failed to expire snapshots", 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.
could we trigger
postInitTable
within theinitTable
?