public interface HiveShim extends Serializable
Modifier and Type | Method and Description |
---|---|
void |
alterPartition(org.apache.hadoop.hive.metastore.IMetaStoreClient client,
String databaseName,
String tableName,
org.apache.hadoop.hive.metastore.api.Partition partition) |
void |
alterTable(org.apache.hadoop.hive.metastore.IMetaStoreClient client,
String databaseName,
String tableName,
org.apache.hadoop.hive.metastore.api.Table table)
Alters a Hive table.
|
BulkWriter.Factory<RowData> |
createOrcBulkWriterFactory(Configuration conf,
String schema,
LogicalType[] fieldTypes)
Create orc
BulkWriter.Factory for different hive versions. |
void |
createTableWithConstraints(org.apache.hadoop.hive.metastore.IMetaStoreClient client,
org.apache.hadoop.hive.metastore.api.Table table,
Configuration conf,
UniqueConstraint pk,
List<Byte> pkTraits,
List<String> notNullCols,
List<Byte> nnTraits)
Creates a table with PK and NOT NULL constraints.
|
org.apache.hadoop.hive.ql.udf.generic.SimpleGenericUDAFParameterInfo |
createUDAFParameterInfo(org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector[] params,
boolean isWindowing,
boolean distinct,
boolean allColumns)
Creates SimpleGenericUDAFParameterInfo.
|
Optional<org.apache.hadoop.hive.ql.exec.FunctionInfo> |
getBuiltInFunctionInfo(String name)
Get a Hive built-in function by name.
|
Class<?> |
getDateDataTypeClass()
Hive Date data type class was changed in Hive 3.1.0.
|
List<org.apache.hadoop.hive.metastore.api.FieldSchema> |
getFieldsFromDeserializer(Configuration conf,
org.apache.hadoop.hive.metastore.api.Table table,
boolean skipConfError)
Get Hive table schema from deserializer.
|
org.apache.hadoop.hive.metastore.IMetaStoreClient |
getHiveMetastoreClient(HiveConf hiveConf)
Create a Hive Metastore client based on the given HiveConf object.
|
Class<?> |
getHiveMetaStoreUtilsClass()
Get the class of Hive's HiveMetaStoreUtils as it was split from MetaStoreUtils class in Hive
3.1.0.
|
Class |
getHiveOutputFormatClass(Class outputFormatClz)
For a given OutputFormat class, get the corresponding
HiveOutputFormat class. |
org.apache.hadoop.hive.ql.exec.FileSinkOperator.RecordWriter |
getHiveRecordWriter(org.apache.hadoop.mapred.JobConf jobConf,
Class outputFormatClz,
Class<? extends org.apache.hadoop.io.Writable> outValClz,
boolean isCompressed,
Properties tableProps,
org.apache.hadoop.fs.Path outPath)
Get Hive's FileSinkOperator.RecordWriter.
|
default org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo |
getIntervalDayTimeTypeInfo() |
default org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo |
getIntervalYearMonthTypeInfo() |
Class<?> |
getMetaStoreUtilsClass()
Get the class of Hive's MetaStoreUtils because its package name was changed in Hive 3.1.0.
|
Set<String> |
getNotNullColumns(org.apache.hadoop.hive.metastore.IMetaStoreClient client,
Configuration conf,
String dbName,
String tableName)
Get the set of columns that have NOT NULL constraints.
|
Optional<UniqueConstraint> |
getPrimaryKey(org.apache.hadoop.hive.metastore.IMetaStoreClient client,
String dbName,
String tableName,
byte requiredTrait)
Get the primary key of a Hive table and convert it to a UniqueConstraint.
|
Class<?> |
getTimestampDataTypeClass()
Hive Timestamp data type class was changed in Hive 3.1.0.
|
List<String> |
getViews(org.apache.hadoop.hive.metastore.IMetaStoreClient client,
String databaseName)
Get a list of views in the given database from the given Hive Metastore client.
|
org.apache.hadoop.io.Writable |
hivePrimitiveToWritable(Object value)
Converts a Hive primitive java object to corresponding Writable object.
|
boolean |
isDateStats(org.apache.hadoop.hive.metastore.api.ColumnStatisticsData colStatsData)
Whether a Hive ColumnStatisticsData is for DATE columns.
|
default boolean |
isIntervalDayTimeType(org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.PrimitiveCategory primitiveCategory) |
default boolean |
isIntervalYearMonthType(org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.PrimitiveCategory primitiveCategory) |
default boolean |
isMaterializedView(org.apache.hadoop.hive.ql.metadata.Table table)
Checks whether a hive table is a materialized view.
|
Set<String> |
listBuiltInFunctions()
List names of all built-in functions.
|
void |
registerTemporaryFunction(String funcName,
Class funcClass) |
java.time.LocalDate |
toFlinkDate(Object hiveDate)
Converts a hive date instance to LocalDate which is expected by DataFormatConverter.
|
CatalogColumnStatisticsDataDate |
toFlinkDateColStats(org.apache.hadoop.hive.metastore.api.ColumnStatisticsData hiveDateColStats)
Generate Flink CatalogColumnStatisticsDataDate from Hive ColumnStatisticsData for DATE
columns.
|
java.time.LocalDateTime |
toFlinkTimestamp(Object hiveTimestamp)
Converts a hive timestamp instance to LocalDateTime which is expected by DataFormatConverter.
|
Object |
toHiveDate(Object flinkDate)
Converts a Flink date instance to what's expected by Hive.
|
org.apache.hadoop.hive.metastore.api.ColumnStatisticsData |
toHiveDateColStats(CatalogColumnStatisticsDataDate flinkDateColStats)
Generate Hive ColumnStatisticsData from Flink CatalogColumnStatisticsDataDate for DATE
columns.
|
Object |
toHiveTimestamp(Object flinkTimestamp)
Converts a Flink timestamp instance to what's expected by Hive.
|
org.apache.hadoop.hive.metastore.IMetaStoreClient getHiveMetastoreClient(HiveConf hiveConf)
hiveConf
- HiveConf instanceList<String> getViews(org.apache.hadoop.hive.metastore.IMetaStoreClient client, String databaseName) throws org.apache.hadoop.hive.metastore.api.UnknownDBException, org.apache.thrift.TException
client
- Hive Metastore clientdatabaseName
- the name of the databaseorg.apache.hadoop.hive.metastore.api.UnknownDBException
- if the database doesn't existorg.apache.thrift.TException
- for any other generic exceptions caused by Thriftvoid alterTable(org.apache.hadoop.hive.metastore.IMetaStoreClient client, String databaseName, String tableName, org.apache.hadoop.hive.metastore.api.Table table) throws org.apache.hadoop.hive.metastore.api.InvalidOperationException, org.apache.hadoop.hive.metastore.api.MetaException, org.apache.thrift.TException
client
- the Hive metastore clientdatabaseName
- the name of the database to which the table belongstableName
- the name of the table to be alteredtable
- the new Hive tableorg.apache.hadoop.hive.metastore.api.InvalidOperationException
org.apache.hadoop.hive.metastore.api.MetaException
org.apache.thrift.TException
void alterPartition(org.apache.hadoop.hive.metastore.IMetaStoreClient client, String databaseName, String tableName, org.apache.hadoop.hive.metastore.api.Partition partition) throws org.apache.hadoop.hive.metastore.api.InvalidOperationException, org.apache.hadoop.hive.metastore.api.MetaException, org.apache.thrift.TException
org.apache.hadoop.hive.metastore.api.InvalidOperationException
org.apache.hadoop.hive.metastore.api.MetaException
org.apache.thrift.TException
org.apache.hadoop.hive.ql.udf.generic.SimpleGenericUDAFParameterInfo createUDAFParameterInfo(org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector[] params, boolean isWindowing, boolean distinct, boolean allColumns)
Class<?> getMetaStoreUtilsClass()
Class<?> getHiveMetaStoreUtilsClass()
Class<?> getDateDataTypeClass()
Class<?> getTimestampDataTypeClass()
org.apache.hadoop.hive.metastore.api.ColumnStatisticsData toHiveDateColStats(CatalogColumnStatisticsDataDate flinkDateColStats)
boolean isDateStats(org.apache.hadoop.hive.metastore.api.ColumnStatisticsData colStatsData)
CatalogColumnStatisticsDataDate toFlinkDateColStats(org.apache.hadoop.hive.metastore.api.ColumnStatisticsData hiveDateColStats)
org.apache.hadoop.hive.ql.exec.FileSinkOperator.RecordWriter getHiveRecordWriter(org.apache.hadoop.mapred.JobConf jobConf, Class outputFormatClz, Class<? extends org.apache.hadoop.io.Writable> outValClz, boolean isCompressed, Properties tableProps, org.apache.hadoop.fs.Path outPath)
Class getHiveOutputFormatClass(Class outputFormatClz)
HiveOutputFormat
class.List<org.apache.hadoop.hive.metastore.api.FieldSchema> getFieldsFromDeserializer(Configuration conf, org.apache.hadoop.hive.metastore.api.Table table, boolean skipConfError)
Optional<org.apache.hadoop.hive.ql.exec.FunctionInfo> getBuiltInFunctionInfo(String name)
Set<String> getNotNullColumns(org.apache.hadoop.hive.metastore.IMetaStoreClient client, Configuration conf, String dbName, String tableName)
Optional<UniqueConstraint> getPrimaryKey(org.apache.hadoop.hive.metastore.IMetaStoreClient client, String dbName, String tableName, byte requiredTrait)
@Nullable Object toHiveTimestamp(@Nullable Object flinkTimestamp)
java.time.LocalDateTime toFlinkTimestamp(Object hiveTimestamp)
@Nullable Object toHiveDate(@Nullable Object flinkDate)
java.time.LocalDate toFlinkDate(Object hiveDate)
@Nullable org.apache.hadoop.io.Writable hivePrimitiveToWritable(@Nullable Object value)
void createTableWithConstraints(org.apache.hadoop.hive.metastore.IMetaStoreClient client, org.apache.hadoop.hive.metastore.api.Table table, Configuration conf, UniqueConstraint pk, List<Byte> pkTraits, List<String> notNullCols, List<Byte> nnTraits)
BulkWriter.Factory<RowData> createOrcBulkWriterFactory(Configuration conf, String schema, LogicalType[] fieldTypes)
BulkWriter.Factory
for different hive versions.default boolean isMaterializedView(org.apache.hadoop.hive.ql.metadata.Table table)
default org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo getIntervalYearMonthTypeInfo()
default org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo getIntervalDayTimeTypeInfo()
default boolean isIntervalYearMonthType(org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.PrimitiveCategory primitiveCategory)
default boolean isIntervalDayTimeType(org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.PrimitiveCategory primitiveCategory)
Copyright © 2014–2023 The Apache Software Foundation. All rights reserved.