Skip to content
Merged
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
29 changes: 28 additions & 1 deletion presto-docs/src/main/sphinx/connector/hive.rst
Original file line number Diff line number Diff line change
Expand Up @@ -291,6 +291,33 @@ Add the ``metastore.storage.schema.reader.impl`` property to ``hive-site.xml`` w

You must restart the metastore service for this configuration to take effect. This setting allows the metastore to read storage schemas for Avro tables and avoids ``Storage schema reading not supported`` errors.

Textfile Configuration Properties
---------------------------------

Table Properties
^^^^^^^^^^^^^^^^

These properties can be used when creating TEXTFILE tables in Presto:

======================================================== ============================================================================== =============================
Property Name Description Default
======================================================== ============================================================================== =============================
``textfile_field_delim`` A custom single-character delimiter to separate fields. NONE

``textfile_escape_delim`` A custom single-character delimiter to escape characters. NONE

``textfile_collection_delim`` A custom single-character delimiter to separate collection elements. NONE

``textfile_mapkey_delim`` A custom single-character delimiter to separate map keys. NONE

======================================================== ============================================================================== =============================

.. note::
These properties are mapped to the corresponding properties in Hive ``LazySerDeParameters`` during serialization and
follow the same behaviors with ``LazySimpleSerDe``.
If they are not defined, the Hive defaults are used, which are typically ``\001`` for field delimiter, ``\002`` for
collection delimiter, ``\003`` for map key delimiter, and escape character is disabled.

Metastore Configuration Properties
----------------------------------

Expand Down Expand Up @@ -1308,4 +1335,4 @@ Example::
CAST(id AS BIGINT) AS id,
CAST(value AS INT) AS value,
CAST(date_col AS DATE) AS date_col
FROM hive.csv.csv_data;
FROM hive.csv.csv_data;
Original file line number Diff line number Diff line change
Expand Up @@ -261,9 +261,12 @@
import static com.facebook.presto.hive.HiveTableProperties.SKIP_HEADER_LINE_COUNT;
import static com.facebook.presto.hive.HiveTableProperties.SORTED_BY_PROPERTY;
import static com.facebook.presto.hive.HiveTableProperties.STORAGE_FORMAT_PROPERTY;
import static com.facebook.presto.hive.HiveTableProperties.TEXTFILE_COLLECTION_DELIM;
import static com.facebook.presto.hive.HiveTableProperties.TEXTFILE_ESCAPE_DELIM;
import static com.facebook.presto.hive.HiveTableProperties.TEXTFILE_FIELD_DELIM;
import static com.facebook.presto.hive.HiveTableProperties.TEXTFILE_MAPKEY_DELIM;
import static com.facebook.presto.hive.HiveTableProperties.getAvroSchemaUrl;
import static com.facebook.presto.hive.HiveTableProperties.getBucketProperty;
import static com.facebook.presto.hive.HiveTableProperties.getCsvProperty;
import static com.facebook.presto.hive.HiveTableProperties.getDwrfEncryptionAlgorithm;
import static com.facebook.presto.hive.HiveTableProperties.getDwrfEncryptionProvider;
import static com.facebook.presto.hive.HiveTableProperties.getEncryptColumns;
Expand All @@ -276,6 +279,7 @@
import static com.facebook.presto.hive.HiveTableProperties.getOrcBloomFilterFpp;
import static com.facebook.presto.hive.HiveTableProperties.getPartitionedBy;
import static com.facebook.presto.hive.HiveTableProperties.getPreferredOrderingColumns;
import static com.facebook.presto.hive.HiveTableProperties.getSingleCharacterProperty;
import static com.facebook.presto.hive.HiveTableProperties.isExternalTable;
import static com.facebook.presto.hive.HiveType.HIVE_BINARY;
import static com.facebook.presto.hive.HiveType.toHiveType;
Expand Down Expand Up @@ -418,6 +422,17 @@ public class HiveMetadata
private static final String CSV_QUOTE_KEY = OpenCSVSerde.QUOTECHAR;
private static final String CSV_ESCAPE_KEY = OpenCSVSerde.ESCAPECHAR;

private static final String TEXTFILE_FIELD_DELIM_KEY = "field.delim";
private static final String TEXTFILE_ESCAPE_DELIM_KEY = "escape.delim";
private static final String TEXTFILE_COLLECTION_DELIM_KEY = "collection.delim";
private static final String TEXTFILE_MAPKEY_DELIM_KEY = "mapkey.delim";

private static final Set<String> TEXTFILE_SERDE_KEYS = ImmutableSet.of(
TEXTFILE_FIELD_DELIM_KEY,
TEXTFILE_ESCAPE_DELIM_KEY,
TEXTFILE_COLLECTION_DELIM_KEY,
TEXTFILE_MAPKEY_DELIM_KEY);

public static final String SKIP_HEADER_COUNT_KEY = "skip.header.line.count";
public static final String SKIP_FOOTER_COUNT_KEY = "skip.footer.line.count";

Expand Down Expand Up @@ -768,6 +783,16 @@ private ConnectorTableMetadata getTableMetadata(Optional<Table> table, SchemaTab
getSerdeProperty(table.get(), SKIP_FOOTER_COUNT_KEY)
.ifPresent(skipFooterCount -> properties.put(SKIP_FOOTER_LINE_COUNT, Integer.valueOf(skipFooterCount)));

// Textfile specific properties
getSerdeProperty(table.get(), TEXTFILE_FIELD_DELIM_KEY)
.ifPresent(fieldDelim -> properties.put(TEXTFILE_FIELD_DELIM, fieldDelim));
getSerdeProperty(table.get(), TEXTFILE_ESCAPE_DELIM_KEY)
.ifPresent(escapeDelim -> properties.put(TEXTFILE_ESCAPE_DELIM, escapeDelim));
getSerdeProperty(table.get(), TEXTFILE_COLLECTION_DELIM_KEY)
.ifPresent(textCollectionDelim -> properties.put(TEXTFILE_COLLECTION_DELIM, textCollectionDelim));
getSerdeProperty(table.get(), TEXTFILE_MAPKEY_DELIM_KEY)
.ifPresent(textMapKeyDelim -> properties.put(TEXTFILE_MAPKEY_DELIM, textMapKeyDelim));

// CSV specific property
getCsvSerdeProperty(table.get(), CSV_SEPARATOR_KEY)
.ifPresent(csvSeparator -> properties.put(CSV_SEPARATOR, csvSeparator));
Expand Down Expand Up @@ -1335,22 +1360,45 @@ private Map<String, String> getEmptyTableProperties(
});

// CSV specific properties
getCsvProperty(tableMetadata.getProperties(), CSV_ESCAPE)
getSingleCharacterProperty(tableMetadata.getProperties(), CSV_ESCAPE)
.ifPresent(escape -> {
checkFormatForProperty(hiveStorageFormat, CSV, CSV_ESCAPE);
tableProperties.put(CSV_ESCAPE_KEY, escape.toString());
});
getCsvProperty(tableMetadata.getProperties(), CSV_QUOTE)
getSingleCharacterProperty(tableMetadata.getProperties(), CSV_QUOTE)
.ifPresent(quote -> {
checkFormatForProperty(hiveStorageFormat, CSV, CSV_QUOTE);
tableProperties.put(CSV_QUOTE_KEY, quote.toString());
});
getCsvProperty(tableMetadata.getProperties(), CSV_SEPARATOR)
getSingleCharacterProperty(tableMetadata.getProperties(), CSV_SEPARATOR)
.ifPresent(separator -> {
checkFormatForProperty(hiveStorageFormat, CSV, CSV_SEPARATOR);
tableProperties.put(CSV_SEPARATOR_KEY, separator.toString());
});

// TEXT specific properties
getSingleCharacterProperty(tableMetadata.getProperties(), TEXTFILE_FIELD_DELIM)
.ifPresent(fieldDelim -> {
checkFormatForProperty(hiveStorageFormat, TEXTFILE, TEXTFILE_FIELD_DELIM_KEY);
tableProperties.put(TEXTFILE_FIELD_DELIM_KEY, fieldDelim.toString());
});
getSingleCharacterProperty(tableMetadata.getProperties(), TEXTFILE_ESCAPE_DELIM)
.ifPresent(escapeDelim -> {
checkFormatForProperty(hiveStorageFormat, TEXTFILE, TEXTFILE_ESCAPE_DELIM_KEY);
tableProperties.put(TEXTFILE_ESCAPE_DELIM_KEY, escapeDelim.toString());
});
getSingleCharacterProperty(tableMetadata.getProperties(), TEXTFILE_COLLECTION_DELIM)
.ifPresent(collectionDelim -> {
checkFormatForProperty(hiveStorageFormat, TEXTFILE, TEXTFILE_COLLECTION_DELIM_KEY);
tableProperties.put(TEXTFILE_COLLECTION_DELIM_KEY, collectionDelim.toString());
});

getSingleCharacterProperty(tableMetadata.getProperties(), TEXTFILE_MAPKEY_DELIM)
.ifPresent(mapKeyDelim -> {
checkFormatForProperty(hiveStorageFormat, TEXTFILE, TEXTFILE_MAPKEY_DELIM_KEY);
tableProperties.put(TEXTFILE_MAPKEY_DELIM_KEY, mapKeyDelim.toString());
});

// Table comment property
tableMetadata.getComment().ifPresent(value -> tableProperties.put(TABLE_COMMENT, value));

Expand Down Expand Up @@ -1461,10 +1509,14 @@ private static Table buildTableObject(
}
}

Map<String, String> serdeParameters = extractSerdeParameters(additionalTableParameters);

ImmutableMap.Builder<String, String> tableParameters = ImmutableMap.<String, String>builder()
.put(PRESTO_VERSION_NAME, prestoVersion)
.put(PRESTO_QUERY_ID_NAME, queryId)
.putAll(additionalTableParameters);
.putAll(additionalTableParameters.entrySet().stream()
.filter(entry -> !serdeParameters.containsKey(entry.getKey()))
.collect(toImmutableMap(Map.Entry::getKey, Map.Entry::getValue)));

if (tableType.equals(EXTERNAL_TABLE)) {
tableParameters.put("EXTERNAL", "TRUE");
Expand All @@ -1483,6 +1535,7 @@ private static Table buildTableObject(
.setStorageFormat(fromHiveStorageFormat(hiveStorageFormat))
.setBucketProperty(bucketProperty)
.setParameters(ImmutableMap.of(PREFERRED_ORDERING_COLUMNS, encodePreferredOrderingColumns(preferredOrderingColumns)))
.setSerdeParameters(serdeParameters)
.setLocation(targetPath.toString());

return tableBuilder.build();
Expand Down Expand Up @@ -3509,6 +3562,13 @@ private static HiveStorageFormat extractHiveStorageFormat(Table table)
throw new PrestoException(HIVE_UNSUPPORTED_FORMAT, format("Output format %s with SerDe %s is not supported", outputFormat, serde));
}

private static Map<String, String> extractSerdeParameters(Map<String, String> tableParameters)
{
return tableParameters.entrySet().stream()
.filter(entry -> TEXTFILE_SERDE_KEYS.contains(entry.getKey()))
.collect(toImmutableMap(Map.Entry::getKey, Map.Entry::getValue));
}

@VisibleForTesting
static String encodePreferredOrderingColumns(List<SortingColumn> preferredOrderingColumns)
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -57,6 +57,11 @@ public class HiveTableProperties
public static final String CSV_SEPARATOR = "csv_separator";
public static final String CSV_QUOTE = "csv_quote";
public static final String CSV_ESCAPE = "csv_escape";
public static final String TEXTFILE_FIELD_DELIM = "textfile_field_delim";
public static final String TEXTFILE_MAPKEY_DELIM = "textfile_mapkey_delim";
public static final String TEXTFILE_COLLECTION_DELIM = "textfile_collection_delim";
public static final String TEXTFILE_ESCAPE_DELIM = "textfile_escape_delim";

public static final String SKIP_HEADER_LINE_COUNT = "skip_header_line_count";
public static final String SKIP_FOOTER_LINE_COUNT = "skip_footer_line_count";

Expand Down Expand Up @@ -157,6 +162,10 @@ public HiveTableProperties(TypeManager typeManager, HiveClientConfig config)
stringProperty(CSV_SEPARATOR, "CSV separator character", null, false),
stringProperty(CSV_QUOTE, "CSV quote character", null, false),
stringProperty(CSV_ESCAPE, "CSV escape character", null, false),
stringProperty(TEXTFILE_FIELD_DELIM, "Textfile field delimiter character", null, false),
stringProperty(TEXTFILE_ESCAPE_DELIM, "Textfile escape delimiter character", null, false),
stringProperty(TEXTFILE_COLLECTION_DELIM, "Textfile collection delimiter character", null, false),
stringProperty(TEXTFILE_MAPKEY_DELIM, "Textfile map key delimiter character", null, false),
integerProperty(SKIP_HEADER_LINE_COUNT, "Number of header lines", null, false),
integerProperty(SKIP_FOOTER_LINE_COUNT, "Number of footer lines", null, false),
new PropertyMetadata<>(
Expand Down Expand Up @@ -248,17 +257,17 @@ public static Double getOrcBloomFilterFpp(Map<String, Object> tableProperties)
return (Double) tableProperties.get(ORC_BLOOM_FILTER_FPP);
}

public static Optional<Character> getCsvProperty(Map<String, Object> tableProperties, String key)
public static Optional<Character> getSingleCharacterProperty(Map<String, Object> tableProperties, String key)
{
Object value = tableProperties.get(key);
if (value == null) {
return Optional.empty();
}
String csvValue = (String) value;
if (csvValue.length() != 1) {
throw new PrestoException(INVALID_TABLE_PROPERTY, format("%s must be a single character string, but was: '%s'", key, csvValue));
String stringValue = (String) value;
if (stringValue.length() != 1) {
throw new PrestoException(INVALID_TABLE_PROPERTY, format("%s must be a single character string, but was: '%s'", key, stringValue));
}
return Optional.of(csvValue.charAt(0));
return Optional.of(stringValue.charAt(0));
}

@SuppressWarnings("unchecked")
Expand Down
Loading
Loading