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
0267eecd
Commit
0267eecd
authored
Aug 01, 2017
by
nixonrodrigues
Committed by
Madhan Neethiraj
Aug 02, 2017
Browse files
Options
Browse Files
Download
Email Patches
Plain Diff
ATLAS-1944: updated handling of shutdown in KafkaConsumer
Change-Id: I07cbe1955cd08005660f5189f30f0690809ce1b1 Signed-off-by:
Madhan Neethiraj
<
madhan@apache.org
>
parent
b59460ff
Hide whitespace changes
Inline
Side-by-side
Showing
4 changed files
with
50 additions
and
10 deletions
+50
-10
AtlasKafkaConsumer.java
.../main/java/org/apache/atlas/kafka/AtlasKafkaConsumer.java
+7
-0
NotificationConsumer.java
...a/org/apache/atlas/notification/NotificationConsumer.java
+3
-0
AbstractNotificationConsumerTest.java
.../atlas/notification/AbstractNotificationConsumerTest.java
+5
-0
NotificationHookConsumer.java
...g/apache/atlas/notification/NotificationHookConsumer.java
+35
-10
No files found.
notification/src/main/java/org/apache/atlas/kafka/AtlasKafkaConsumer.java
View file @
0267eecd
...
...
@@ -96,4 +96,11 @@ public class AtlasKafkaConsumer<T> extends AbstractNotificationConsumer<T> {
kafkaConsumer
.
close
();
}
}
@Override
public
void
wakeup
()
{
if
(
kafkaConsumer
!=
null
)
{
kafkaConsumer
.
wakeup
();
}
}
}
notification/src/main/java/org/apache/atlas/notification/NotificationConsumer.java
View file @
0267eecd
...
...
@@ -39,6 +39,8 @@ public interface NotificationConsumer<T> {
void
close
();
void
wakeup
();
/**
* Fetch data for the topics from Kafka
* @return List containing kafka message and partionId and offset.
...
...
@@ -53,4 +55,5 @@ public interface NotificationConsumer<T> {
List
<
AtlasKafkaMessage
<
T
>>
receive
(
long
timeoutMilliSeconds
);
}
notification/src/test/java/org/apache/atlas/notification/AbstractNotificationConsumerTest.java
View file @
0267eecd
...
...
@@ -203,6 +203,11 @@ public class AbstractNotificationConsumerTest {
}
@Override
public
void
wakeup
()
{
}
@Override
public
List
<
AtlasKafkaMessage
<
T
>>
receive
()
{
return
receive
(
1000L
);
}
...
...
webapp/src/main/java/org/apache/atlas/notification/NotificationHookConsumer.java
View file @
0267eecd
...
...
@@ -168,12 +168,16 @@ public class NotificationHookConsumer implements Service, ActiveStateChangeHandl
}
private
void
stopConsumerThreads
()
{
LOG
.
info
(
"==> stopConsumerThreads()"
);
if
(
consumers
!=
null
)
{
for
(
HookConsumer
consumer
:
consumers
)
{
consumer
.
s
top
();
consumer
.
s
hutdown
();
}
consumers
.
clear
();
}
LOG
.
info
(
"<== stopConsumerThreads()"
);
}
/**
...
...
@@ -218,21 +222,35 @@ public class NotificationHookConsumer implements Service, ActiveStateChangeHandl
@Override
public
void
doWork
()
{
LOG
.
info
(
"==> HookConsumer doWork()"
);
shouldRun
.
set
(
true
);
if
(!
serverAvailable
(
new
NotificationHookConsumer
.
Timer
()))
{
return
;
}
while
(
shouldRun
.
get
())
{
try
{
List
<
AtlasKafkaMessage
<
HookNotificationMessage
>>
messages
=
consumer
.
receive
();
for
(
AtlasKafkaMessage
<
HookNotificationMessage
>
msg
:
messages
)
{
handleMessage
(
msg
);
try
{
while
(
shouldRun
.
get
())
{
try
{
List
<
AtlasKafkaMessage
<
HookNotificationMessage
>>
messages
=
consumer
.
receive
();
for
(
AtlasKafkaMessage
<
HookNotificationMessage
>
msg
:
messages
)
{
handleMessage
(
msg
);
}
}
catch
(
Exception
e
)
{
if
(
shouldRun
.
get
())
{
LOG
.
warn
(
"Exception in NotificationHookConsumer"
,
e
);
}
}
}
catch
(
Throwable
t
)
{
LOG
.
warn
(
"Failure in NotificationHookConsumer"
,
t
);
}
}
finally
{
if
(
consumer
!=
null
)
{
LOG
.
info
(
"closing NotificationConsumer"
);
consumer
.
close
();
}
LOG
.
info
(
"<== HookConsumer doWork()"
);
}
}
...
...
@@ -369,7 +387,7 @@ public class NotificationHookConsumer implements Service, ActiveStateChangeHandl
private
void
commit
(
AtlasKafkaMessage
<
HookNotificationMessage
>
kafkaMessage
)
{
recordFailedMessages
();
TopicPartition
partition
=
new
TopicPartition
(
"ATLAS_HOOK"
,
kafkaMessage
.
getPartition
());
consumer
.
commit
(
partition
,
kafkaMessage
.
getOffset
());
consumer
.
commit
(
partition
,
kafkaMessage
.
getOffset
()
+
1
);
}
boolean
serverAvailable
(
Timer
timer
)
{
...
...
@@ -397,11 +415,18 @@ public class NotificationHookConsumer implements Service, ActiveStateChangeHandl
@Override
public
void
shutdown
()
{
LOG
.
info
(
"==> HookConsumer shutdown()"
);
super
.
initiateShutdown
();
shouldRun
.
set
(
false
);
consumer
.
close
();
if
(
consumer
!=
null
)
{
consumer
.
wakeup
();
}
super
.
awaitShutdown
();
LOG
.
info
(
"<== HookConsumer shutdown()"
);
}
}
private
void
audit
(
String
messageUser
,
String
method
,
String
path
)
{
...
...
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