Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
39 changes: 39 additions & 0 deletions data/conf/iceberg/llap/hive-site.xml
Original file line number Diff line number Diff line change
Expand Up @@ -397,4 +397,43 @@
<name>hive.lock.sleep.between.retries</name>
<value>2</value>
</property>

<!-- -- Iceberg REST catalog specific properties -->
<property>
<name>connector.name</name>
<value>iceberg</value>
</property>
<property>
<name>iceberg.rest-catalog.type</name>
<value>rest</value>
</property>
<property>
<name>iceberg.rest-catalog.uri</name>
<value>{restServerUrl}</value>
</property>
<!-- For OAUTH2 authentication add this as well -->
<property>
<name>iceberg.rest-catalog.security</name>
<value>OAUTH2</value>
</property>
<property>
<name>iceberg.rest-catalog.credential</name>
<value>{user:pass}</value>
</property>
<property>
<name>iceberg.rest-catalog.scope</name>
<value>PRINCIPAL_ROLE:ALL</value>
</property>
<property>
<name>iceberg.rest-catalog.oauth2.credential</name>
<value>{user:pass}</value>
</property>
<property>
<name>iceberg.rest-catalog.oauth2.scope</name>
<value>PRINCIPAL_ROLE:ALL</value>
</property>
<property>
<name>iceberg.rest-catalog.warehouse</name>
<value>{catalogName}</value>
</property>
</configuration>
1 change: 0 additions & 1 deletion iceberg/iceberg-catalog/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -69,7 +69,6 @@
<dependency>
<groupId>org.apache.hive</groupId>
<artifactId>hive-exec</artifactId>
<scope>test</scope>
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

why did you remove the test scope?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I added import in HMSTablePropertyHelper which needs this dependency.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

you shouldn't add hive-exec as dependency to iceberg-catalog. catalog can be used by Spark and Impala that doesn't have/need hive-exec dependency

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Are you sure it is used in Spark or Impala?

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

yes, there is no iceberg runtime post iceberg 1.8

Copy link

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, It is definitely used by Impala.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I have unknowns around this area. Why do we have org.apache.iceberg.hive.HiveCatalog while Apache Iceberg also has it? I recall we discussed ML before, but I'm still unsure about some points. When I started being involved with Hive + Iceberg, we already had ported files.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@okumin historical reasons :)
Was already like that when i joined the project. The main reason was a struggle to get anything into iceberg repo.
Not 100% sure, but I think Iceberg folks still haven't upgraded to Hive-4.x, so there could be some conflicts.
However, I agree on a long run we should drop it from Hive and use iceberg dependency

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I checked and there is 0 reference from Impala and 0 reference from Spark for hive-iceberg-catalog.

Copy link
Member

@deniskuzZ deniskuzZ Jul 16, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

iceberg-catalog shouldn't have any dependency on runtime (hive-exec) !!

This is a temporary copy of iceberg-hive-metastore, a direct dependency of iceberg-spark-runtime.

With the removal of iceberg-hive-runtime in Iceberg 1.8, Impala needs to adopt the Hive Iceberg runtime jar.
https://github.com/apache/impala/blob/41b6652fbf612a44676fed1cd9bd0da45133a04d/fe/pom.xml#L437-L441

PS: Only resolve comments after reaching consensus!

</dependency>
<dependency>
<groupId>org.immutables</groupId>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,29 +19,40 @@

package org.apache.iceberg.hive;

import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.Optional;
import java.util.Properties;
import java.util.Set;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hive.common.StatsSetupConst;
import org.apache.hadoop.hive.metastore.api.SerDeInfo;
import org.apache.hadoop.hive.metastore.api.Table;
import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
import org.apache.hadoop.hive.ql.parse.TransformSpec;
import org.apache.hadoop.hive.ql.session.SessionStateUtil;
import org.apache.hive.iceberg.com.fasterxml.jackson.core.JsonProcessingException;
import org.apache.iceberg.BaseMetastoreTableOperations;
import org.apache.iceberg.PartitionSpec;
import org.apache.iceberg.PartitionSpecParser;
import org.apache.iceberg.Schema;
import org.apache.iceberg.SchemaParser;
import org.apache.iceberg.Snapshot;
import org.apache.iceberg.SnapshotSummary;
import org.apache.iceberg.SortOrder;
import org.apache.iceberg.SortOrderParser;
import org.apache.iceberg.TableMetadata;
import org.apache.iceberg.TableProperties;
import org.apache.iceberg.catalog.TableIdentifier;
import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
import org.apache.iceberg.relocated.com.google.common.collect.BiMap;
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableBiMap;
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
import org.apache.iceberg.relocated.com.google.common.collect.Maps;
import org.apache.iceberg.util.JsonUtil;
import org.apache.parquet.Strings;
import org.apache.parquet.hadoop.ParquetOutputFormat;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
Expand All @@ -58,6 +69,15 @@ public class HMSTablePropertyHelper {
GC_ENABLED, "external.table.purge", TableProperties.PARQUET_COMPRESSION, ParquetOutputFormat.COMPRESSION,
TableProperties.PARQUET_ROW_GROUP_SIZE_BYTES, ParquetOutputFormat.BLOCK_SIZE);

public static final Set<String> PROPERTIES_TO_REMOVE = ImmutableSet
// We don't want to push down the metadata location props to Iceberg from HMS,
// since the snapshot pointer in HMS would always be one step ahead
.of(BaseMetastoreTableOperations.METADATA_LOCATION_PROP,
BaseMetastoreTableOperations.PREVIOUS_METADATA_LOCATION_PROP);

public static final String NAME = "name";
public static final String LOCATION = "location";

private HMSTablePropertyHelper() {
}

Expand Down Expand Up @@ -131,6 +151,103 @@ public static void updateHmsTableForIcebergTable(
tbl.setParameters(parameters);
}

/**
* Create {@link PartitionSpec} based on the partition information stored in
* {@link TransformSpec}.
* @param configuration a Hadoop configuration
* @param schema iceberg table schema
* @return iceberg partition spec, always non-null
*/
public static PartitionSpec createPartitionSpec(Configuration configuration, Schema schema) {
List<TransformSpec> partitionTransformSpecList = SessionStateUtil
.getResource(configuration, hive_metastoreConstants.PARTITION_TRANSFORM_SPEC)
.map(o -> (List<TransformSpec>) o)
.orElse(null);

if (partitionTransformSpecList == null) {
LOG.warn("Iceberg partition transform spec is not found in QueryState.");
return null;
}
PartitionSpec.Builder builder = PartitionSpec.builderFor(schema);
partitionTransformSpecList.forEach(spec -> {
switch (spec.getTransformType()) {
case IDENTITY:
builder.identity(spec.getColumnName().toLowerCase());
break;
case YEAR:
builder.year(spec.getColumnName());
break;
case MONTH:
builder.month(spec.getColumnName());
break;
case DAY:
builder.day(spec.getColumnName());
break;
case HOUR:
builder.hour(spec.getColumnName());
break;
case TRUNCATE:
builder.truncate(spec.getColumnName(), spec.getTransformParam().get());
break;
case BUCKET:
builder.bucket(spec.getColumnName(), spec.getTransformParam().get());
break;
}
});
return builder.build();
}

public static SortOrder getSortOrder(Properties props, Schema schema) {
String sortOrderJsonString = props.getProperty(TableProperties.DEFAULT_SORT_ORDER);
return Strings.isNullOrEmpty(sortOrderJsonString) ? SortOrder.unsorted() : SortOrderParser.fromJson(schema,
sortOrderJsonString);
}

/**
* Calculates the properties we would like to send to the catalog.
* <ul>
* <li>The base of the properties is the properties stored at the Hive Metastore for the given table
* <li>We add the {@link HiveIcebergRESTCatalogClientAdapter#LOCATION} as the table location
* <li>We add the {@link HiveIcebergRESTCatalogClientAdapter#NAME} as
* TableIdentifier defined by the database name and table name
* <li>We add the serdeProperties of the HMS table
* <li>We remove some parameters that we don't want to push down to the Iceberg table props
* </ul>
* @param hmsTable Table for which we are calculating the properties
* @return The properties we can provide for Iceberg functions
*/
public static Properties getCatalogProperties(org.apache.hadoop.hive.metastore.api.Table hmsTable) {
Properties properties = new Properties();

hmsTable.getParameters().entrySet().stream().filter(e -> e.getKey() != null && e.getValue() != null).forEach(e -> {
// translate key names between HMS and Iceberg where needed
String icebergKey = HMSTablePropertyHelper.translateToIcebergProp(e.getKey());
properties.put(icebergKey, e.getValue());
});

if (properties.get(LOCATION) == null &&
hmsTable.getSd() != null && hmsTable.getSd().getLocation() != null) {
properties.put(LOCATION, hmsTable.getSd().getLocation());
}

if (properties.get(NAME) == null) {
properties.put(NAME, TableIdentifier.of(hmsTable.getDbName(), hmsTable.getTableName()).toString());
}

SerDeInfo serdeInfo = hmsTable.getSd().getSerdeInfo();
if (serdeInfo != null) {
serdeInfo.getParameters().entrySet().stream()
.filter(e -> e.getKey() != null && e.getValue() != null).forEach(e -> {
String icebergKey = HMSTablePropertyHelper.translateToIcebergProp(e.getKey());
properties.put(icebergKey, e.getValue());
});
}

// Remove HMS table parameters we don't want to propagate to Iceberg
PROPERTIES_TO_REMOVE.forEach(properties::remove);

return properties;
}
private static void setCommonParameters(
String newMetadataLocation,
String uuid,
Expand All @@ -143,8 +260,9 @@ private static void setCommonParameters(
if (uuid != null) {
parameters.put(TableProperties.UUID, uuid);
}

obsoleteProps.forEach(parameters::remove);
if (obsoleteProps != null) {
obsoleteProps.forEach(parameters::remove);
}

parameters.put(BaseMetastoreTableOperations.TABLE_TYPE_PROP, tableType);
parameters.put(BaseMetastoreTableOperations.METADATA_LOCATION_PROP, newMetadataLocation);
Expand Down
Loading
Loading