branch-2.1: [Fix](Iceberg-hadoop-catalog)Fix Kerberos-authenticated HadoopCatalog insert failures due to missing kerberos credentials #51245 (#51337)

Cherry-picked from #51245

---------

Co-authored-by: Calvin Kirs <guoqiang@selectdb.com>
This commit is contained in:
github-actions[bot]
2025-06-20 14:12:25 +08:00
committed by GitHub
parent 2e8ec1850f
commit f58b3204ca
3 changed files with 124 additions and 38 deletions

View File

@ -73,10 +73,18 @@ public class IcebergTransaction implements Transaction {
}
}
public void beginInsert(SimpleTableInfo tableInfo) {
this.tableInfo = tableInfo;
this.table = getNativeTable(tableInfo);
this.transaction = table.newTransaction();
public void beginInsert(SimpleTableInfo tableInfo) throws UserException {
try {
ops.getPreExecutionAuthenticator().execute(() -> {
// create and start the iceberg transaction
this.tableInfo = tableInfo;
this.table = getNativeTable(tableInfo);
this.transaction = table.newTransaction();
});
} catch (Exception e) {
throw new UserException("Failed to begin insert for iceberg table " + tableInfo, e);
}
}
public void finishInsert(SimpleTableInfo tableInfo, Optional<InsertCommandContext> insertCtx) {