Skip to content
体验新版
项目
组织
正在加载...
登录
切换导航
打开侧边栏
doujutun3207
flink
提交
1444fd68
F
flink
项目概览
doujutun3207
/
flink
与 Fork 源项目一致
从无法访问的项目Fork
通知
24
Star
0
Fork
0
代码
文件
提交
分支
Tags
贡献者
分支图
Diff
Issue
0
列表
看板
标记
里程碑
合并请求
0
Wiki
0
Wiki
分析
仓库
DevOps
项目成员
Pages
F
flink
项目概览
项目概览
详情
发布
仓库
仓库
文件
提交
分支
标签
贡献者
分支图
比较
Issue
0
Issue
0
列表
看板
标记
里程碑
合并请求
0
合并请求
0
Pages
分析
分析
仓库分析
DevOps
Wiki
0
Wiki
成员
成员
收起侧边栏
关闭侧边栏
动态
分支图
创建新Issue
提交
Issue看板
体验新版 GitCode,发现更多精彩内容 >>
未验证
提交
1444fd68
编写于
3月 31, 2020
作者:
R
Rui Li
提交者:
GitHub
3月 31, 2020
浏览文件
操作
浏览文件
下载
电子邮件补丁
差异文件
[FLINK-16767][hive] Failed to read Hive table with RegexSerDe
This closes #11504
上级
e789bb71
变更
5
隐藏空白更改
内联
并排
Showing
5 changed file
with
38 addition
and
45 deletion
+38
-45
flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveTablePartition.java
.../org/apache/flink/connectors/hive/HiveTablePartition.java
+12
-3
flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveTableSource.java
...ava/org/apache/flink/connectors/hive/HiveTableSource.java
+7
-2
flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/read/HiveMapredSplitReader.java
...che/flink/connectors/hive/read/HiveMapredSplitReader.java
+1
-5
flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/util/HiveTableUtil.java
...g/apache/flink/table/catalog/hive/util/HiveTableUtil.java
+0
-35
flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/TableEnvHiveConnectorTest.java
...ache/flink/connectors/hive/TableEnvHiveConnectorTest.java
+18
-0
未找到文件。
flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveTablePartition.java
浏览文件 @
1444fd68
...
...
@@ -23,6 +23,7 @@ import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
import
java.io.Serializable
;
import
java.util.LinkedHashMap
;
import
java.util.Map
;
import
java.util.Properties
;
import
static
org
.
apache
.
flink
.
util
.
Preconditions
.
checkNotNull
;
...
...
@@ -40,13 +41,17 @@ public class HiveTablePartition implements Serializable {
/** The map of partition key names and their values. */
private
final
Map
<
String
,
Object
>
partitionSpec
;
public
HiveTablePartition
(
StorageDescriptor
storageDescriptor
)
{
this
(
storageDescriptor
,
new
LinkedHashMap
<>());
// Table properties that should be used to initialize SerDe
private
final
Properties
tableProps
;
public
HiveTablePartition
(
StorageDescriptor
storageDescriptor
,
Properties
tableProps
)
{
this
(
storageDescriptor
,
new
LinkedHashMap
<>(),
tableProps
);
}
public
HiveTablePartition
(
StorageDescriptor
storageDescriptor
,
Map
<
String
,
Object
>
partitionSpec
)
{
public
HiveTablePartition
(
StorageDescriptor
storageDescriptor
,
Map
<
String
,
Object
>
partitionSpec
,
Properties
tableProps
)
{
this
.
storageDescriptor
=
checkNotNull
(
storageDescriptor
,
"storageDescriptor can not be null"
);
this
.
partitionSpec
=
checkNotNull
(
partitionSpec
,
"partitionSpec can not be null"
);
this
.
tableProps
=
checkNotNull
(
tableProps
,
"tableProps can not be null"
);
}
public
StorageDescriptor
getStorageDescriptor
()
{
...
...
@@ -56,4 +61,8 @@ public class HiveTablePartition implements Serializable {
public
Map
<
String
,
Object
>
getPartitionSpec
()
{
return
partitionSpec
;
}
public
Properties
getTableProps
()
{
return
tableProps
;
}
}
flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveTableSource.java
浏览文件 @
1444fd68
...
...
@@ -35,6 +35,7 @@ import org.apache.flink.table.catalog.hive.client.HiveMetastoreClientWrapper;
import
org.apache.flink.table.catalog.hive.client.HiveShim
;
import
org.apache.flink.table.catalog.hive.client.HiveShimLoader
;
import
org.apache.flink.table.catalog.hive.descriptors.HiveCatalogValidator
;
import
org.apache.flink.table.catalog.hive.util.HiveReflectionUtils
;
import
org.apache.flink.table.dataformat.BaseRow
;
import
org.apache.flink.table.functions.hive.conversion.HiveInspectors
;
import
org.apache.flink.table.runtime.types.TypeInfoDataTypeConverter
;
...
...
@@ -51,6 +52,7 @@ import org.apache.flink.util.Preconditions;
import
org.apache.hadoop.hive.conf.HiveConf
;
import
org.apache.hadoop.hive.metastore.api.Partition
;
import
org.apache.hadoop.hive.metastore.api.StorageDescriptor
;
import
org.apache.hadoop.hive.metastore.api.Table
;
import
org.apache.hadoop.mapred.JobConf
;
import
org.apache.thrift.TException
;
import
org.slf4j.Logger
;
...
...
@@ -66,6 +68,7 @@ import java.util.Arrays;
import
java.util.HashMap
;
import
java.util.List
;
import
java.util.Map
;
import
java.util.Properties
;
import
java.util.stream.Collectors
;
/**
...
...
@@ -281,6 +284,8 @@ public class HiveTableSource implements
String
dbName
=
tablePath
.
getDatabaseName
();
String
tableName
=
tablePath
.
getObjectName
();
List
<
String
>
partitionColNames
=
catalogTable
.
getPartitionKeys
();
Table
hiveTable
=
client
.
getTable
(
dbName
,
tableName
);
Properties
tableProps
=
HiveReflectionUtils
.
getTableMetadata
(
hiveShim
,
hiveTable
);
if
(
partitionColNames
!=
null
&&
partitionColNames
.
size
()
>
0
)
{
final
String
defaultPartitionName
=
jobConf
.
get
(
HiveConf
.
ConfVars
.
DEFAULTPARTITIONNAME
.
varname
,
HiveConf
.
ConfVars
.
DEFAULTPARTITIONNAME
.
defaultStrVal
);
...
...
@@ -309,11 +314,11 @@ public class HiveTableSource implements
}
partitionColValues
.
put
(
partitionColName
,
partitionObject
);
}
HiveTablePartition
hiveTablePartition
=
new
HiveTablePartition
(
sd
,
partitionColValues
);
HiveTablePartition
hiveTablePartition
=
new
HiveTablePartition
(
sd
,
partitionColValues
,
tableProps
);
allHivePartitions
.
add
(
hiveTablePartition
);
}
}
else
{
allHivePartitions
.
add
(
new
HiveTablePartition
(
client
.
getTable
(
dbName
,
tableName
).
getSd
()
));
allHivePartitions
.
add
(
new
HiveTablePartition
(
hiveTable
.
getSd
(),
tableProps
));
}
}
catch
(
TException
e
)
{
throw
new
FlinkHiveException
(
"Failed to collect all partitions from hive metaStore"
,
e
);
...
...
flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/read/HiveMapredSplitReader.java
浏览文件 @
1444fd68
...
...
@@ -22,7 +22,6 @@ import org.apache.flink.api.java.hadoop.mapred.wrapper.HadoopDummyReporter;
import
org.apache.flink.connectors.hive.FlinkHiveException
;
import
org.apache.flink.connectors.hive.HiveTablePartition
;
import
org.apache.flink.table.catalog.hive.client.HiveShim
;
import
org.apache.flink.table.catalog.hive.util.HiveTableUtil
;
import
org.apache.flink.table.dataformat.BaseRow
;
import
org.apache.flink.table.dataformat.DataFormatConverters
;
import
org.apache.flink.table.dataformat.GenericRow
;
...
...
@@ -48,7 +47,6 @@ import org.slf4j.LoggerFactory;
import
java.io.IOException
;
import
java.util.Arrays
;
import
java.util.List
;
import
java.util.Properties
;
import
static
org
.
apache
.
hadoop
.
mapreduce
.
lib
.
input
.
FileInputFormat
.
INPUT_DIR
;
...
...
@@ -122,9 +120,7 @@ public class HiveMapredSplitReader implements SplitReader {
try
{
deserializer
=
(
Deserializer
)
Class
.
forName
(
sd
.
getSerdeInfo
().
getSerializationLib
()).
newInstance
();
Configuration
conf
=
new
Configuration
();
//properties are used to initialize hive Deserializer properly.
Properties
properties
=
HiveTableUtil
.
createPropertiesFromStorageDescriptor
(
sd
);
SerDeUtils
.
initializeSerDe
(
deserializer
,
conf
,
properties
,
null
);
SerDeUtils
.
initializeSerDe
(
deserializer
,
conf
,
hiveTablePartition
.
getTableProps
(),
null
);
structObjectInspector
=
(
StructObjectInspector
)
deserializer
.
getObjectInspector
();
structFields
=
structObjectInspector
.
getAllStructFieldRefs
();
}
catch
(
Exception
e
)
{
...
...
flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/util/HiveTableUtil.java
浏览文件 @
1444fd68
...
...
@@ -33,13 +33,9 @@ import org.apache.flink.table.functions.hive.conversion.HiveInspectors;
import
org.apache.flink.table.types.DataType
;
import
org.apache.flink.table.types.logical.LogicalTypeRoot
;
import
org.apache.commons.lang3.StringUtils
;
import
org.apache.hadoop.hive.metastore.api.FieldSchema
;
import
org.apache.hadoop.hive.metastore.api.Partition
;
import
org.apache.hadoop.hive.metastore.api.SerDeInfo
;
import
org.apache.hadoop.hive.metastore.api.StorageDescriptor
;
import
org.apache.hadoop.hive.serde.serdeConstants
;
import
org.apache.hadoop.hive.serde2.SerDeUtils
;
import
org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils
;
import
java.util.ArrayList
;
...
...
@@ -47,11 +43,8 @@ import java.util.HashMap;
import
java.util.List
;
import
java.util.Map
;
import
java.util.Optional
;
import
java.util.Properties
;
import
java.util.Set
;
import
static
org
.
apache
.
flink
.
table
.
catalog
.
hive
.
HiveCatalogConfig
.
DEFAULT_LIST_COLUMN_TYPES_SEPARATOR
;
/**
* Utils to for Hive-backed table.
*/
...
...
@@ -113,34 +106,6 @@ public class HiveTableUtil {
// Helper methods
// --------------------------------------------------------------------------------------------
/**
* Create properties info to initialize a SerDe.
* @param storageDescriptor
* @return
*/
public
static
Properties
createPropertiesFromStorageDescriptor
(
StorageDescriptor
storageDescriptor
)
{
SerDeInfo
serDeInfo
=
storageDescriptor
.
getSerdeInfo
();
Map
<
String
,
String
>
parameters
=
serDeInfo
.
getParameters
();
Properties
properties
=
new
Properties
();
properties
.
setProperty
(
serdeConstants
.
SERIALIZATION_FORMAT
,
parameters
.
get
(
serdeConstants
.
SERIALIZATION_FORMAT
));
List
<
String
>
colTypes
=
new
ArrayList
<>();
List
<
String
>
colNames
=
new
ArrayList
<>();
List
<
FieldSchema
>
cols
=
storageDescriptor
.
getCols
();
for
(
FieldSchema
col:
cols
){
colTypes
.
add
(
col
.
getType
());
colNames
.
add
(
col
.
getName
());
}
properties
.
setProperty
(
serdeConstants
.
LIST_COLUMNS
,
StringUtils
.
join
(
colNames
,
String
.
valueOf
(
SerDeUtils
.
COMMA
)));
// Note: serdeConstants.COLUMN_NAME_DELIMITER is not defined in previous Hive. We use a literal to save on shim
properties
.
setProperty
(
"column.name.delimite"
,
String
.
valueOf
(
SerDeUtils
.
COMMA
));
properties
.
setProperty
(
serdeConstants
.
LIST_COLUMN_TYPES
,
StringUtils
.
join
(
colTypes
,
DEFAULT_LIST_COLUMN_TYPES_SEPARATOR
));
properties
.
setProperty
(
serdeConstants
.
SERIALIZATION_NULL_FORMAT
,
"NULL"
);
properties
.
putAll
(
parameters
);
return
properties
;
}
/**
* Creates a Hive partition instance.
*/
...
...
flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/TableEnvHiveConnectorTest.java
浏览文件 @
1444fd68
...
...
@@ -565,6 +565,24 @@ public class TableEnvHiveConnectorTest {
}
}
@Test
public
void
testRegexSerDe
()
throws
Exception
{
hiveShell
.
execute
(
"create database db1"
);
try
{
hiveShell
.
execute
(
"create table db1.src (x int,y string) "
+
"row format serde 'org.apache.hadoop.hive.serde2.RegexSerDe' "
+
"with serdeproperties ('input.regex'='([\\\\d]+)\\u0001([\\\\S]+)')"
);
HiveTestUtils
.
createTextTableInserter
(
hiveShell
,
"db1"
,
"src"
)
.
addRow
(
new
Object
[]{
1
,
"a"
})
.
addRow
(
new
Object
[]{
2
,
"ab"
})
.
commit
();
TableEnvironment
tableEnv
=
getTableEnvWithHiveCatalog
();
assertEquals
(
"[1,a, 2,ab]"
,
TableUtils
.
collectToList
(
tableEnv
.
sqlQuery
(
"select * from db1.src order by x"
)).
toString
());
}
finally
{
hiveShell
.
execute
(
"drop database db1 cascade"
);
}
}
private
TableEnvironment
getTableEnvWithHiveCatalog
()
{
TableEnvironment
tableEnv
=
HiveTestUtils
.
createTableEnvWithBlinkPlannerBatchMode
();
tableEnv
.
registerCatalog
(
hiveCatalog
.
getName
(),
hiveCatalog
);
...
...
编辑
预览
Markdown
is supported
0%
请重试
或
添加新附件
.
添加附件
取消
You are about to add
0
people
to the discussion. Proceed with caution.
先完成此消息的编辑!
取消
想要评论请
注册
或
登录