Skip to content
Projects
Groups
Snippets
Help
This project
Loading...
Sign in / Register
Toggle navigation
A
atlas
Project
Overview
Details
Activity
Cycle Analytics
Repository
Repository
Files
Commits
Branches
Tags
Contributors
Graph
Compare
Charts
Issues
0
Issues
0
List
Board
Labels
Milestones
Merge Requests
0
Merge Requests
0
CI / CD
CI / CD
Pipelines
Jobs
Schedules
Charts
Wiki
Wiki
Snippets
Snippets
Members
Members
Collapse sidebar
Close sidebar
Activity
Graph
Charts
Create a new issue
Jobs
Commits
Issue Boards
Open sidebar
dataplatform
atlas
Commits
fff94633
Commit
fff94633
authored
Jun 10, 2018
by
Sarath Subramanian
Browse files
Options
Browse Files
Download
Email Patches
Plain Diff
ATLAS-2751: Atlas is not consuming messages from ATLAS_HOOK topic after…
ATLAS-2751: Atlas is not consuming messages from ATLAS_HOOK topic after recovering from zookeeper connection timeout.
parent
ce5ffeb7
Hide whitespace changes
Inline
Side-by-side
Showing
1 changed file
with
22 additions
and
3 deletions
+22
-3
KafkaNotification.java
...c/main/java/org/apache/atlas/kafka/KafkaNotification.java
+22
-3
No files found.
notification/src/main/java/org/apache/atlas/kafka/KafkaNotification.java
View file @
fff94633
...
@@ -56,6 +56,8 @@ public class KafkaNotification extends AbstractNotification implements Service {
...
@@ -56,6 +56,8 @@ public class KafkaNotification extends AbstractNotification implements Service {
public
static
final
String
ATLAS_ENTITIES_TOPIC
=
"ATLAS_ENTITIES"
;
public
static
final
String
ATLAS_ENTITIES_TOPIC
=
"ATLAS_ENTITIES"
;
protected
static
final
String
CONSUMER_GROUP_ID_PROPERTY
=
"group.id"
;
protected
static
final
String
CONSUMER_GROUP_ID_PROPERTY
=
"group.id"
;
private
static
final
String
DEFAULT_CONSUMER_CLOSED_ERROR_MESSAGE
=
"This consumer has already been closed."
;
private
static
final
Map
<
NotificationType
,
String
>
TOPIC_MAP
=
new
HashMap
<
NotificationType
,
String
>()
{
private
static
final
Map
<
NotificationType
,
String
>
TOPIC_MAP
=
new
HashMap
<
NotificationType
,
String
>()
{
{
{
put
(
NotificationType
.
HOOK
,
ATLAS_HOOK_TOPIC
);
put
(
NotificationType
.
HOOK
,
ATLAS_HOOK_TOPIC
);
...
@@ -67,6 +69,7 @@ public class KafkaNotification extends AbstractNotification implements Service {
...
@@ -67,6 +69,7 @@ public class KafkaNotification extends AbstractNotification implements Service {
private
final
Long
pollTimeOutMs
;
private
final
Long
pollTimeOutMs
;
private
KafkaConsumer
consumer
;
private
KafkaConsumer
consumer
;
private
KafkaProducer
producer
;
private
KafkaProducer
producer
;
private
String
consumerClosedErrorMsg
;
// ----- Constructors ----------------------------------------------------
// ----- Constructors ----------------------------------------------------
...
@@ -85,8 +88,9 @@ public class KafkaNotification extends AbstractNotification implements Service {
...
@@ -85,8 +88,9 @@ public class KafkaNotification extends AbstractNotification implements Service {
Configuration
kafkaConf
=
ApplicationProperties
.
getSubsetConfiguration
(
applicationProperties
,
PROPERTY_PREFIX
);
Configuration
kafkaConf
=
ApplicationProperties
.
getSubsetConfiguration
(
applicationProperties
,
PROPERTY_PREFIX
);
properties
=
ConfigurationConverter
.
getProperties
(
kafkaConf
);
properties
=
ConfigurationConverter
.
getProperties
(
kafkaConf
);
pollTimeOutMs
=
kafkaConf
.
getLong
(
"poll.timeout.ms"
,
1000
);
pollTimeOutMs
=
kafkaConf
.
getLong
(
"poll.timeout.ms"
,
1000
);
consumerClosedErrorMsg
=
kafkaConf
.
getString
(
"error.message.consumer_closed"
,
DEFAULT_CONSUMER_CLOSED_ERROR_MESSAGE
);
//Override default configs
//Override default configs
properties
.
put
(
ProducerConfig
.
KEY_SERIALIZER_CLASS_CONFIG
,
"org.apache.kafka.common.serialization.StringSerializer"
);
properties
.
put
(
ProducerConfig
.
KEY_SERIALIZER_CLASS_CONFIG
,
"org.apache.kafka.common.serialization.StringSerializer"
);
...
@@ -223,7 +227,7 @@ public class KafkaNotification extends AbstractNotification implements Service {
...
@@ -223,7 +227,7 @@ public class KafkaNotification extends AbstractNotification implements Service {
public
KafkaConsumer
getKafkaConsumer
(
Properties
consumerProperties
,
NotificationType
type
,
boolean
autoCommitEnabled
)
{
public
KafkaConsumer
getKafkaConsumer
(
Properties
consumerProperties
,
NotificationType
type
,
boolean
autoCommitEnabled
)
{
if
(
this
.
consumer
==
null
)
{
if
(
consumer
==
null
||
!
isKafkaConsumerOpen
(
consumer
)
)
{
try
{
try
{
String
topic
=
TOPIC_MAP
.
get
(
type
);
String
topic
=
TOPIC_MAP
.
get
(
type
);
...
@@ -287,4 +291,19 @@ public class KafkaNotification extends AbstractNotification implements Service {
...
@@ -287,4 +291,19 @@ public class KafkaNotification extends AbstractNotification implements Service {
return
message
;
return
message
;
}
}
}
}
// kafka-client doesn't have method to check if consumer is open, hence checking list topics and catching exception
private
boolean
isKafkaConsumerOpen
(
KafkaConsumer
consumer
)
{
boolean
ret
=
true
;
try
{
consumer
.
listTopics
();
}
catch
(
IllegalStateException
ex
)
{
if
(
ex
.
getMessage
().
equalsIgnoreCase
(
consumerClosedErrorMsg
))
{
ret
=
false
;
}
}
return
ret
;
}
}
}
Write
Preview
Markdown
is supported
0%
Try again
or
attach a new file
Attach a file
Cancel
You are about to add
0
people
to the discussion. Proceed with caution.
Finish editing this message first!
Cancel
Please
register
or
sign in
to comment