235235import static com .facebook .presto .hive .HiveSessionProperties .shouldCreateEmptyBucketFilesForTemporaryTable ;
236236import static com .facebook .presto .hive .HiveStatisticsUtil .createPartitionStatistics ;
237237import static com .facebook .presto .hive .HiveStorageFormat .AVRO ;
238+ import static com .facebook .presto .hive .HiveStorageFormat .CSV ;
238239import static com .facebook .presto .hive .HiveStorageFormat .DWRF ;
239240import static com .facebook .presto .hive .HiveStorageFormat .ORC ;
240241import static com .facebook .presto .hive .HiveStorageFormat .PAGEFILE ;
241242import static com .facebook .presto .hive .HiveStorageFormat .PARQUET ;
243+ import static com .facebook .presto .hive .HiveStorageFormat .TEXTFILE ;
242244import static com .facebook .presto .hive .HiveStorageFormat .values ;
243245import static com .facebook .presto .hive .HiveTableProperties .AVRO_SCHEMA_URL ;
244246import static com .facebook .presto .hive .HiveTableProperties .BUCKETED_BY_PROPERTY ;
255257import static com .facebook .presto .hive .HiveTableProperties .ORC_BLOOM_FILTER_FPP ;
256258import static com .facebook .presto .hive .HiveTableProperties .PARTITIONED_BY_PROPERTY ;
257259import static com .facebook .presto .hive .HiveTableProperties .PREFERRED_ORDERING_COLUMNS ;
260+ import static com .facebook .presto .hive .HiveTableProperties .SKIP_FOOTER_LINE_COUNT ;
261+ import static com .facebook .presto .hive .HiveTableProperties .SKIP_HEADER_LINE_COUNT ;
258262import static com .facebook .presto .hive .HiveTableProperties .SORTED_BY_PROPERTY ;
259263import static com .facebook .presto .hive .HiveTableProperties .STORAGE_FORMAT_PROPERTY ;
260264import static com .facebook .presto .hive .HiveTableProperties .getAvroSchemaUrl ;
265269import static com .facebook .presto .hive .HiveTableProperties .getEncryptColumns ;
266270import static com .facebook .presto .hive .HiveTableProperties .getEncryptTable ;
267271import static com .facebook .presto .hive .HiveTableProperties .getExternalLocation ;
272+ import static com .facebook .presto .hive .HiveTableProperties .getFooterSkipCount ;
273+ import static com .facebook .presto .hive .HiveTableProperties .getHeaderSkipCount ;
268274import static com .facebook .presto .hive .HiveTableProperties .getHiveStorageFormat ;
269275import static com .facebook .presto .hive .HiveTableProperties .getOrcBloomFilterColumns ;
270276import static com .facebook .presto .hive .HiveTableProperties .getOrcBloomFilterFpp ;
@@ -412,6 +418,9 @@ public class HiveMetadata
412418 private static final String CSV_QUOTE_KEY = OpenCSVSerde .QUOTECHAR ;
413419 private static final String CSV_ESCAPE_KEY = OpenCSVSerde .ESCAPECHAR ;
414420
421+ public static final String SKIP_HEADER_COUNT_KEY = "skip.header.line.count" ;
422+ public static final String SKIP_FOOTER_COUNT_KEY = "skip.footer.line.count" ;
423+
415424 private static final JsonCodec <MaterializedViewDefinition > MATERIALIZED_VIEW_JSON_CODEC = jsonCodec (MaterializedViewDefinition .class );
416425
417426 private final boolean allowCorruptWritesForTesting ;
@@ -753,6 +762,12 @@ private ConnectorTableMetadata getTableMetadata(Optional<Table> table, SchemaTab
753762 properties .put (AVRO_SCHEMA_URL , avroSchemaUrl );
754763 }
755764
765+ // Textfile and CSV specific properties
766+ getSerdeProperty (table .get (), SKIP_HEADER_COUNT_KEY )
767+ .ifPresent (skipHeaderCount -> properties .put (SKIP_HEADER_LINE_COUNT , Integer .valueOf (skipHeaderCount )));
768+ getSerdeProperty (table .get (), SKIP_FOOTER_COUNT_KEY )
769+ .ifPresent (skipFooterCount -> properties .put (SKIP_FOOTER_LINE_COUNT , Integer .valueOf (skipFooterCount )));
770+
756771 // CSV specific property
757772 getCsvSerdeProperty (table .get (), CSV_SEPARATOR_KEY )
758773 .ifPresent (csvSeparator -> properties .put (CSV_SEPARATOR , csvSeparator ));
@@ -1294,20 +1309,42 @@ private Map<String, String> getEmptyTableProperties(
12941309 tableProperties .put (AVRO_SCHEMA_URL_KEY , validateAndNormalizeAvroSchemaUrl (avroSchemaUrl , hdfsContext ));
12951310 }
12961311
1312+ // Textfile and CSV specific properties
1313+ Set <HiveStorageFormat > csvAndTextFile = ImmutableSet .of (TEXTFILE , CSV );
1314+ getHeaderSkipCount (tableMetadata .getProperties ()).ifPresent (headerSkipCount -> {
1315+ if (headerSkipCount > 0 ) {
1316+ checkFormatForProperty (hiveStorageFormat , csvAndTextFile , SKIP_HEADER_LINE_COUNT );
1317+ tableProperties .put (SKIP_HEADER_COUNT_KEY , String .valueOf (headerSkipCount ));
1318+ }
1319+ if (headerSkipCount < 0 ) {
1320+ throw new PrestoException (HIVE_INVALID_METADATA , format ("Invalid value for %s property: %s" , SKIP_HEADER_LINE_COUNT , headerSkipCount ));
1321+ }
1322+ });
1323+
1324+ getFooterSkipCount (tableMetadata .getProperties ()).ifPresent (footerSkipCount -> {
1325+ if (footerSkipCount > 0 ) {
1326+ checkFormatForProperty (hiveStorageFormat , csvAndTextFile , SKIP_FOOTER_LINE_COUNT );
1327+ tableProperties .put (SKIP_FOOTER_COUNT_KEY , String .valueOf (footerSkipCount ));
1328+ }
1329+ if (footerSkipCount < 0 ) {
1330+ throw new PrestoException (HIVE_INVALID_METADATA , format ("Invalid value for %s property: %s" , SKIP_FOOTER_LINE_COUNT , footerSkipCount ));
1331+ }
1332+ });
1333+
12971334 // CSV specific properties
12981335 getCsvProperty (tableMetadata .getProperties (), CSV_ESCAPE )
12991336 .ifPresent (escape -> {
1300- checkFormatForProperty (hiveStorageFormat , HiveStorageFormat . CSV , CSV_ESCAPE );
1337+ checkFormatForProperty (hiveStorageFormat , CSV , CSV_ESCAPE );
13011338 tableProperties .put (CSV_ESCAPE_KEY , escape .toString ());
13021339 });
13031340 getCsvProperty (tableMetadata .getProperties (), CSV_QUOTE )
13041341 .ifPresent (quote -> {
1305- checkFormatForProperty (hiveStorageFormat , HiveStorageFormat . CSV , CSV_QUOTE );
1342+ checkFormatForProperty (hiveStorageFormat , CSV , CSV_QUOTE );
13061343 tableProperties .put (CSV_QUOTE_KEY , quote .toString ());
13071344 });
13081345 getCsvProperty (tableMetadata .getProperties (), CSV_SEPARATOR )
13091346 .ifPresent (separator -> {
1310- checkFormatForProperty (hiveStorageFormat , HiveStorageFormat . CSV , CSV_SEPARATOR );
1347+ checkFormatForProperty (hiveStorageFormat , CSV , CSV_SEPARATOR );
13111348 tableProperties .put (CSV_SEPARATOR_KEY , separator .toString ());
13121349 });
13131350
@@ -1327,6 +1364,13 @@ private static void checkFormatForProperty(HiveStorageFormat actualStorageFormat
13271364 }
13281365 }
13291366
1367+ private static void checkFormatForProperty (HiveStorageFormat actualStorageFormat , Set <HiveStorageFormat > expectedStorageFormats , String propertyName )
1368+ {
1369+ if (!expectedStorageFormats .contains (actualStorageFormat )) {
1370+ throw new PrestoException (INVALID_TABLE_PROPERTY , format ("Cannot specify %s table property for storage format: %s" , propertyName , actualStorageFormat ));
1371+ }
1372+ }
1373+
13301374 private String validateAndNormalizeAvroSchemaUrl (String url , HdfsContext context )
13311375 {
13321376 try {
@@ -1647,7 +1691,17 @@ public HiveOutputTableHandle beginCreateTable(ConnectorSession session, Connecto
16471691 if (getAvroSchemaUrl (tableMetadata .getProperties ()) != null ) {
16481692 throw new PrestoException (NOT_SUPPORTED , "CREATE TABLE AS not supported when Avro schema url is set" );
16491693 }
1694+ getHeaderSkipCount (tableMetadata .getProperties ()).ifPresent (headerSkipCount -> {
1695+ if (headerSkipCount > 1 ) {
1696+ throw new PrestoException (NOT_SUPPORTED , format ("Creating Hive table with data with value of %s property greater than 1 is not supported" , SKIP_HEADER_COUNT_KEY ));
1697+ }
1698+ });
16501699
1700+ getFooterSkipCount (tableMetadata .getProperties ()).ifPresent (footerSkipCount -> {
1701+ if (footerSkipCount > 0 ) {
1702+ throw new PrestoException (NOT_SUPPORTED , format ("Creating Hive table with data with value of %s property greater than 0 is not supported" , SKIP_FOOTER_COUNT_KEY ));
1703+ }
1704+ });
16511705 HiveStorageFormat tableStorageFormat = getHiveStorageFormat (tableMetadata .getProperties ());
16521706 List <String > partitionedBy = getPartitionedBy (tableMetadata .getProperties ());
16531707 Optional <HiveBucketProperty > bucketProperty = getBucketProperty (tableMetadata .getProperties ());
@@ -2016,6 +2070,15 @@ private HiveInsertTableHandle beginInsertInternal(ConnectorSession session, Conn
20162070 locationHandle = locationService .forExistingTable (metastore , session , table , tempPathRequired );
20172071 }
20182072
2073+ Optional .ofNullable (table .getParameters ().get (SKIP_HEADER_COUNT_KEY )).map (Integer ::parseInt ).ifPresent (headerSkipCount -> {
2074+ if (headerSkipCount > 1 ) {
2075+ throw new PrestoException (NOT_SUPPORTED , format ("INSERT into %s Hive table with value of %s property greater than 1 is not supported" , tableName , SKIP_HEADER_COUNT_KEY ));
2076+ }
2077+ });
2078+ if (table .getParameters ().containsKey (SKIP_FOOTER_COUNT_KEY )) {
2079+ throw new PrestoException (NOT_SUPPORTED , format ("INSERT into %s Hive table with %s property not supported" , tableName , SKIP_FOOTER_COUNT_KEY ));
2080+ }
2081+
20192082 Optional <? extends TableEncryptionProperties > tableEncryptionProperties = getTableEncryptionPropertiesFromHiveProperties (table .getParameters (), tableStorageFormat );
20202083
20212084 HiveStorageFormat partitionStorageFormat = isRespectTableFormat (session ) ? tableStorageFormat : getHiveStorageFormat (session );
@@ -3677,7 +3740,7 @@ else if (column.isHidden()) {
36773740
36783741 private static void validateCsvColumns (ConnectorTableMetadata tableMetadata )
36793742 {
3680- if (getHiveStorageFormat (tableMetadata .getProperties ()) != HiveStorageFormat . CSV ) {
3743+ if (getHiveStorageFormat (tableMetadata .getProperties ()) != CSV ) {
36813744 return ;
36823745 }
36833746
0 commit comments