Skip to content
体验新版
项目
组织
正在加载...
登录
切换导航
打开侧边栏
doujutun3207
flink
提交
2eb2a0ef
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,发现更多精彩内容 >>
提交
2eb2a0ef
编写于
2月 04, 2016
作者:
S
Stephan Ewen
浏览文件
操作
浏览文件
下载
电子邮件补丁
差异文件
[FLINK-3338] [kafka] Use proper classloader when cloning the deserialization schema.
This closes #1590
上级
fe0c3b53
变更
4
隐藏空白更改
内联
并排
Showing
4 changed file
with
48 addition
and
12 deletion
+48
-12
flink-core/src/main/java/org/apache/flink/util/InstantiationUtil.java
...rc/main/java/org/apache/flink/util/InstantiationUtil.java
+27
-2
flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer08.java
...link/streaming/connectors/kafka/FlinkKafkaConsumer08.java
+2
-1
flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/LegacyFetcher.java
...k/streaming/connectors/kafka/internals/LegacyFetcher.java
+18
-8
flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/MockRuntimeContext.java
...eaming/connectors/kafka/testutils/MockRuntimeContext.java
+1
-1
未找到文件。
flink-core/src/main/java/org/apache/flink/util/InstantiationUtil.java
浏览文件 @
2eb2a0ef
...
@@ -310,8 +310,33 @@ public final class InstantiationUtil {
...
@@ -310,8 +310,33 @@ public final class InstantiationUtil {
* @throws ClassNotFoundException
* @throws ClassNotFoundException
*/
*/
public
static
<
T
extends
Serializable
>
T
clone
(
T
obj
)
throws
IOException
,
ClassNotFoundException
{
public
static
<
T
extends
Serializable
>
T
clone
(
T
obj
)
throws
IOException
,
ClassNotFoundException
{
final
byte
[]
serializedObject
=
serializeObject
(
obj
);
if
(
obj
==
null
)
{
return
deserializeObject
(
serializedObject
,
obj
.
getClass
().
getClassLoader
());
return
null
;
}
else
{
return
clone
(
obj
,
obj
.
getClass
().
getClassLoader
());
}
}
/**
* Clones the given serializable object using Java serialization, using the given classloader to
* resolve the cloned classes.
*
* @param obj Object to clone
* @param classLoader The classloader to resolve the classes during deserialization.
* @param <T> Type of the object to clone
*
* @return Cloned object
*
* @throws IOException
* @throws ClassNotFoundException
*/
public
static
<
T
extends
Serializable
>
T
clone
(
T
obj
,
ClassLoader
classLoader
)
throws
IOException
,
ClassNotFoundException
{
if
(
obj
==
null
)
{
return
null
;
}
else
{
final
byte
[]
serializedObject
=
serializeObject
(
obj
);
return
deserializeObject
(
serializedObject
,
classLoader
);
}
}
}
// --------------------------------------------------------------------------------------------
// --------------------------------------------------------------------------------------------
...
...
flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumer08.java
浏览文件 @
2eb2a0ef
...
@@ -251,7 +251,8 @@ public class FlinkKafkaConsumer08<T> extends FlinkKafkaConsumerBase<T> {
...
@@ -251,7 +251,8 @@ public class FlinkKafkaConsumer08<T> extends FlinkKafkaConsumerBase<T> {
}
}
// create fetcher
// create fetcher
fetcher
=
new
LegacyFetcher
(
this
.
subscribedPartitions
,
props
,
getRuntimeContext
().
getTaskName
());
fetcher
=
new
LegacyFetcher
(
this
.
subscribedPartitions
,
props
,
getRuntimeContext
().
getTaskName
(),
getRuntimeContext
().
getUserCodeClassLoader
());
// offset handling
// offset handling
offsetHandler
=
new
ZookeeperOffsetHandler
(
props
);
offsetHandler
=
new
ZookeeperOffsetHandler
(
props
);
...
...
flink-streaming-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/internals/LegacyFetcher.java
浏览文件 @
2eb2a0ef
...
@@ -31,11 +31,12 @@ import kafka.message.MessageAndOffset;
...
@@ -31,11 +31,12 @@ import kafka.message.MessageAndOffset;
import
org.apache.flink.streaming.api.functions.source.SourceFunction
;
import
org.apache.flink.streaming.api.functions.source.SourceFunction
;
import
org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer08
;
import
org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer08
;
import
org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema
;
import
org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema
;
import
org.apache.flink.util.InstantiationUtil
;
import
org.apache.flink.util.InstantiationUtil
;
import
org.apache.flink.util.StringUtils
;
import
org.apache.flink.util.StringUtils
;
import
org.apache.kafka.clients.consumer.ConsumerConfig
;
import
org.apache.kafka.clients.consumer.ConsumerConfig
;
import
org.apache.kafka.common.Node
;
import
org.apache.kafka.common.Node
;
import
org.slf4j.Logger
;
import
org.slf4j.Logger
;
import
org.slf4j.LoggerFactory
;
import
org.slf4j.LoggerFactory
;
...
@@ -49,7 +50,7 @@ import java.util.Map;
...
@@ -49,7 +50,7 @@ import java.util.Map;
import
java.util.Properties
;
import
java.util.Properties
;
import
java.util.concurrent.atomic.AtomicReference
;
import
java.util.concurrent.atomic.AtomicReference
;
import
static
com
.
google
.
common
.
base
.
Preconditions
.
checkNot
Null
;
import
static
java
.
util
.
Objects
.
requireNon
Null
;
/**
/**
* This fetcher uses Kafka's low-level API to pull data from a specific
* This fetcher uses Kafka's low-level API to pull data from a specific
...
@@ -70,6 +71,9 @@ public class LegacyFetcher implements Fetcher {
...
@@ -70,6 +71,9 @@ public class LegacyFetcher implements Fetcher {
/** The first error that occurred in a connection thread */
/** The first error that occurred in a connection thread */
private
final
AtomicReference
<
Throwable
>
error
;
private
final
AtomicReference
<
Throwable
>
error
;
/** The classloader for dynamically loaded classes */
private
final
ClassLoader
userCodeClassloader
;
/** The partitions that the fetcher should read, with their starting offsets */
/** The partitions that the fetcher should read, with their starting offsets */
private
Map
<
KafkaTopicPartitionLeader
,
Long
>
partitionsToRead
;
private
Map
<
KafkaTopicPartitionLeader
,
Long
>
partitionsToRead
;
...
@@ -86,8 +90,13 @@ public class LegacyFetcher implements Fetcher {
...
@@ -86,8 +90,13 @@ public class LegacyFetcher implements Fetcher {
/** Flag to shot the fetcher down */
/** Flag to shot the fetcher down */
private
volatile
boolean
running
=
true
;
private
volatile
boolean
running
=
true
;
public
LegacyFetcher
(
List
<
KafkaTopicPartitionLeader
>
partitions
,
Properties
props
,
String
taskName
)
{
public
LegacyFetcher
(
this
.
config
=
checkNotNull
(
props
,
"The config properties cannot be null"
);
List
<
KafkaTopicPartitionLeader
>
partitions
,
Properties
props
,
String
taskName
,
ClassLoader
userCodeClassloader
)
{
this
.
config
=
requireNonNull
(
props
,
"The config properties cannot be null"
);
this
.
userCodeClassloader
=
requireNonNull
(
userCodeClassloader
);
//this.topic = checkNotNull(topic, "The topic cannot be null");
//this.topic = checkNotNull(topic, "The topic cannot be null");
this
.
partitionsToRead
=
new
HashMap
<>();
this
.
partitionsToRead
=
new
HashMap
<>();
for
(
KafkaTopicPartitionLeader
p:
partitions
)
{
for
(
KafkaTopicPartitionLeader
p:
partitions
)
{
...
@@ -200,7 +209,8 @@ public class LegacyFetcher implements Fetcher {
...
@@ -200,7 +209,8 @@ public class LegacyFetcher implements Fetcher {
FetchPartition
[]
partitions
=
partitionsList
.
toArray
(
new
FetchPartition
[
partitionsList
.
size
()]);
FetchPartition
[]
partitions
=
partitionsList
.
toArray
(
new
FetchPartition
[
partitionsList
.
size
()]);
final
KeyedDeserializationSchema
<
T
>
clonedDeserializer
=
InstantiationUtil
.
clone
(
deserializer
);
final
KeyedDeserializationSchema
<
T
>
clonedDeserializer
=
InstantiationUtil
.
clone
(
deserializer
,
userCodeClassloader
);
SimpleConsumerThread
<
T
>
thread
=
new
SimpleConsumerThread
<>(
this
,
config
,
SimpleConsumerThread
<
T
>
thread
=
new
SimpleConsumerThread
<>(
this
,
config
,
broker
,
partitions
,
sourceContext
,
clonedDeserializer
,
lastOffsets
);
broker
,
partitions
,
sourceContext
,
clonedDeserializer
,
lastOffsets
);
...
@@ -344,9 +354,9 @@ public class LegacyFetcher implements Fetcher {
...
@@ -344,9 +354,9 @@ public class LegacyFetcher implements Fetcher {
this
.
config
=
config
;
this
.
config
=
config
;
this
.
broker
=
broker
;
this
.
broker
=
broker
;
this
.
partitions
=
partitions
;
this
.
partitions
=
partitions
;
this
.
sourceContext
=
checkNot
Null
(
sourceContext
);
this
.
sourceContext
=
requireNon
Null
(
sourceContext
);
this
.
deserializer
=
checkNot
Null
(
deserializer
);
this
.
deserializer
=
requireNon
Null
(
deserializer
);
this
.
offsetsState
=
checkNot
Null
(
offsetsState
);
this
.
offsetsState
=
requireNon
Null
(
offsetsState
);
}
}
@Override
@Override
...
...
flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/MockRuntimeContext.java
浏览文件 @
2eb2a0ef
...
@@ -98,7 +98,7 @@ public class MockRuntimeContext extends StreamingRuntimeContext {
...
@@ -98,7 +98,7 @@ public class MockRuntimeContext extends StreamingRuntimeContext {
@Override
@Override
public
ClassLoader
getUserCodeClassLoader
()
{
public
ClassLoader
getUserCodeClassLoader
()
{
throw
new
UnsupportedOperationException
();
return
getClass
().
getClassLoader
();
}
}
@Override
@Override
...
...
编辑
预览
Markdown
is supported
0%
请重试
或
添加新附件
.
添加附件
取消
You are about to add
0
people
to the discussion. Proceed with caution.
先完成此消息的编辑!
取消
想要评论请
注册
或
登录