Skip to content
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

Parquet schema evolution on non-primitive type #71

Closed
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
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,9 @@
import static com.facebook.presto.hive.HiveType.HIVE_LONG;
import static com.facebook.presto.hive.HiveType.HIVE_SHORT;
import static java.util.Objects.requireNonNull;
import static org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector.Category.LIST;
import static org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector.Category.MAP;
import static org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector.Category.STRUCT;

public class HiveCoercionPolicy
implements CoercionPolicy
Expand All @@ -40,6 +43,10 @@ public HiveCoercionPolicy(TypeManager typeManager)
@Override
public boolean canCoerce(HiveType fromHiveType, HiveType toHiveType)
{
if (fromHiveType.getCategory() == MAP || fromHiveType.getCategory() == LIST || fromHiveType.getCategory() == STRUCT) {
return fromHiveType.getCategory().equals(toHiveType.getCategory());
}

Type fromType = typeManager.getType(fromHiveType.getTypeSignature());
Type toType = typeManager.getType(toHiveType.getTypeSignature());
if (fromType instanceof VarcharType) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,7 @@
package com.facebook.presto.hive;

import com.facebook.presto.hive.HivePageSourceProvider.ColumnMapping;
import com.facebook.presto.hive.parquet.ParquetHiveRecordCursor;
import com.facebook.presto.spi.PrestoException;
import com.facebook.presto.spi.RecordCursor;
import com.facebook.presto.spi.type.Type;
Expand Down Expand Up @@ -64,6 +65,10 @@ public HiveCoercionRecordCursor(
ColumnMapping columnMapping = columnMappings.get(columnIndex);

if (columnMapping.getCoercionFrom().isPresent()) {
// skip non-primitive coercion for Parquet
if (delegate instanceof ParquetHiveRecordCursor && !columnMapping.isFromPrimitiveType()) {
continue;
}
coercers[columnIndex] = createCoercer(typeManager, columnMapping.getCoercionFrom().get(), columnMapping.getHiveColumnHandle().getHiveType());
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -13,6 +13,8 @@
*/
package com.facebook.presto.hive;

import com.facebook.presto.hive.parquet.ParquetPageSourceFactory;
import com.facebook.presto.hive.parquet.ParquetRecordCursorProvider;
import com.facebook.presto.spi.ColumnHandle;
import com.facebook.presto.spi.ConnectorPageSource;
import com.facebook.presto.spi.ConnectorSession;
Expand Down Expand Up @@ -51,6 +53,7 @@
import static com.google.common.collect.Maps.uniqueIndex;
import static java.util.Objects.requireNonNull;
import static java.util.stream.Collectors.toList;
import static org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector.Category.PRIMITIVE;

public class HivePageSourceProvider
implements ConnectorPageSourceProvider
Expand Down Expand Up @@ -149,14 +152,17 @@ public static Optional<ConnectorPageSource> createHivePageSource(
List<ColumnMapping> regularColumnMappings = ColumnMapping.extractRegularColumnMappings(columnMappings);

for (HivePageSourceFactory pageSourceFactory : pageSourceFactories) {
// only Parquet support flexible non-primitive converter in its cursor
boolean doCoercionOnNonPrimitive = !(pageSourceFactory instanceof ParquetPageSourceFactory);

Optional<? extends ConnectorPageSource> pageSource = pageSourceFactory.createPageSource(
configuration,
session,
path,
start,
length,
schema,
extractRegularColumnHandles(regularColumnMappings, true),
extractRegularColumnHandles(regularColumnMappings, true, doCoercionOnNonPrimitive),
effectivePredicate,
hiveStorageTimeZone
);
Expand All @@ -173,6 +179,8 @@ public static Optional<ConnectorPageSource> createHivePageSource(
for (HiveRecordCursorProvider provider : cursorProviders) {
// GenericHiveRecordCursor will automatically do the coercion without HiveCoercionRecordCursor
boolean doCoercion = !(provider instanceof GenericHiveRecordCursorProvider);
// only Parquet support non-primitive coercion without HiveCoercionRecordCursor
boolean doCoercionOnNonPrimitive = !(provider instanceof ParquetRecordCursorProvider);

Optional<RecordCursor> cursor = provider.createRecordCursor(
clientId,
Expand All @@ -182,7 +190,7 @@ public static Optional<ConnectorPageSource> createHivePageSource(
start,
length,
schema,
extractRegularColumnHandles(regularColumnMappings, doCoercion),
extractRegularColumnHandles(regularColumnMappings, doCoercion, doCoercionOnNonPrimitive),
effectivePredicate,
hiveStorageTimeZone,
typeManager);
Expand Down Expand Up @@ -261,6 +269,12 @@ public Optional<HiveType> getCoercionFrom()
return coercionFrom;
}

public boolean isFromPrimitiveType()
{
checkState(coercionFrom.isPresent(), "coercionFrom is not present");
return coercionFrom.get().getCategory() == PRIMITIVE;
}

private static boolean isPrefilled(HiveColumnHandle hiveColumnHandle)
{
return hiveColumnHandle.getColumnType() != REGULAR;
Expand Down Expand Up @@ -305,14 +319,17 @@ public static List<ColumnMapping> extractRegularColumnMappings(List<ColumnMappin
.collect(toList());
}

public static List<HiveColumnHandle> extractRegularColumnHandles(List<ColumnMapping> regularColumnMappings, boolean doCoercion)
public static List<HiveColumnHandle> extractRegularColumnHandles(List<ColumnMapping> regularColumnMappings, boolean doCoercion, boolean doCoercionOnNonPrimitive)
{
return regularColumnMappings.stream()
.map(columnMapping -> {
HiveColumnHandle columnHandle = columnMapping.getHiveColumnHandle();
if (!doCoercion || !columnMapping.getCoercionFrom().isPresent()) {
return columnHandle;
}
if (!columnMapping.isFromPrimitiveType() && !doCoercionOnNonPrimitive) {
return columnHandle;
}
return new HiveColumnHandle(columnHandle.getClientId(),
columnHandle.getName(),
columnMapping.getCoercionFrom().get(),
Expand Down
Loading