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
98769871
Commit
98769871
authored
May 13, 2016
by
Hemanth Yamijala
Browse files
Options
Browse Files
Download
Email Patches
Plain Diff
ATLAS-629 Kafka messages in ATLAS_HOOK might be lost in HA mode at the instant…
ATLAS-629 Kafka messages in ATLAS_HOOK might be lost in HA mode at the instant of failover. (yhemanth)
parent
07b8b4d3
Show whitespace changes
Inline
Side-by-side
Showing
14 changed files
with
354 additions
and
103 deletions
+354
-103
atlas-application.properties
distro/src/conf/atlas-application.properties
+1
-0
KafkaConsumer.java
...n/src/main/java/org/apache/atlas/kafka/KafkaConsumer.java
+22
-5
KafkaNotification.java
...c/main/java/org/apache/atlas/kafka/KafkaNotification.java
+24
-11
AbstractNotification.java
...a/org/apache/atlas/notification/AbstractNotification.java
+7
-2
AbstractNotificationConsumer.java
...ache/atlas/notification/AbstractNotificationConsumer.java
+2
-0
NotificationConsumer.java
...a/org/apache/atlas/notification/NotificationConsumer.java
+9
-0
KafkaConsumerTest.java
...c/test/java/org/apache/atlas/kafka/KafkaConsumerTest.java
+49
-6
KafkaNotificationTest.java
...st/java/org/apache/atlas/kafka/KafkaNotificationTest.java
+44
-73
AbstractNotificationConsumerTest.java
.../atlas/notification/AbstractNotificationConsumerTest.java
+5
-0
release-log.txt
release-log.txt
+1
-0
atlas-application.properties
typesystem/src/main/resources/atlas-application.properties
+1
-0
NotificationHookConsumer.java
...g/apache/atlas/notification/NotificationHookConsumer.java
+12
-6
NotificationHookConsumerKafkaTest.java
...atlas/notification/NotificationHookConsumerKafkaTest.java
+141
-0
NotificationHookConsumerTest.java
...ache/atlas/notification/NotificationHookConsumerTest.java
+36
-0
No files found.
distro/src/conf/atlas-application.properties
View file @
98769871
...
@@ -59,6 +59,7 @@ atlas.kafka.zookeeper.sync.time.ms=20
...
@@ -59,6 +59,7 @@ atlas.kafka.zookeeper.sync.time.ms=20
atlas.kafka.auto.commit.interval.ms
=
1000
atlas.kafka.auto.commit.interval.ms
=
1000
atlas.kafka.auto.offset.reset
=
smallest
atlas.kafka.auto.offset.reset
=
smallest
atlas.kafka.hook.group.id
=
atlas
atlas.kafka.hook.group.id
=
atlas
atlas.kafka.auto.commit.enable
=
false
######### Hive Lineage Configs #########
######### Hive Lineage Configs #########
...
...
notification/src/main/java/org/apache/atlas/kafka/KafkaConsumer.java
View file @
98769871
...
@@ -19,6 +19,7 @@ package org.apache.atlas.kafka;
...
@@ -19,6 +19,7 @@ package org.apache.atlas.kafka;
import
kafka.consumer.ConsumerIterator
;
import
kafka.consumer.ConsumerIterator
;
import
kafka.consumer.KafkaStream
;
import
kafka.consumer.KafkaStream
;
import
kafka.javaapi.consumer.ConsumerConnector
;
import
kafka.message.MessageAndMetadata
;
import
kafka.message.MessageAndMetadata
;
import
org.apache.atlas.notification.AbstractNotificationConsumer
;
import
org.apache.atlas.notification.AbstractNotificationConsumer
;
import
org.apache.atlas.notification.MessageDeserializer
;
import
org.apache.atlas.notification.MessageDeserializer
;
...
@@ -35,24 +36,29 @@ public class KafkaConsumer<T> extends AbstractNotificationConsumer<T> {
...
@@ -35,24 +36,29 @@ public class KafkaConsumer<T> extends AbstractNotificationConsumer<T> {
private
final
int
consumerId
;
private
final
int
consumerId
;
private
final
ConsumerIterator
iterator
;
private
final
ConsumerIterator
iterator
;
private
final
ConsumerConnector
consumerConnector
;
private
final
boolean
autoCommitEnabled
;
private
long
lastSeenOffset
;
// ----- Constructors ----------------------------------------------------
// ----- Constructors ----------------------------------------------------
/**
/**
* Create a Kafka consumer.
* Create a Kafka consumer.
*
* @param type the notification type returned by this consumer
* @param deserializer the message deserializer used for this consumer
* @param deserializer the message deserializer used for this consumer
* @param stream the underlying Kafka stream
* @param stream the underlying Kafka stream
* @param consumerId an id value for this consumer
* @param consumerId an id value for this consumer
* @param consumerConnector the {@link ConsumerConnector} which created the underlying Kafka stream
* @param autoCommitEnabled true if consumer does not need to commit offsets explicitly, false otherwise.
*/
*/
public
KafkaConsumer
(
Class
<
T
>
type
,
public
KafkaConsumer
(
MessageDeserializer
<
T
>
deserializer
,
KafkaStream
<
String
,
String
>
stream
,
int
consumerId
,
MessageDeserializer
<
T
>
deserializer
,
KafkaStream
<
String
,
String
>
stream
,
int
consumerI
d
)
{
ConsumerConnector
consumerConnector
,
boolean
autoCommitEnable
d
)
{
super
(
deserializer
);
super
(
deserializer
);
this
.
consumerConnector
=
consumerConnector
;
this
.
lastSeenOffset
=
0
;
this
.
iterator
=
stream
.
iterator
();
this
.
iterator
=
stream
.
iterator
();
this
.
consumerId
=
consumerId
;
this
.
consumerId
=
consumerId
;
this
.
autoCommitEnabled
=
autoCommitEnabled
;
}
}
...
@@ -71,6 +77,7 @@ public class KafkaConsumer<T> extends AbstractNotificationConsumer<T> {
...
@@ -71,6 +77,7 @@ public class KafkaConsumer<T> extends AbstractNotificationConsumer<T> {
MessageAndMetadata
message
=
iterator
.
next
();
MessageAndMetadata
message
=
iterator
.
next
();
LOG
.
debug
(
"Read message: conumerId: {}, topic - {}, partition - {}, offset - {}, message - {}"
,
LOG
.
debug
(
"Read message: conumerId: {}, topic - {}, partition - {}, offset - {}, message - {}"
,
consumerId
,
message
.
topic
(),
message
.
partition
(),
message
.
offset
(),
message
.
message
());
consumerId
,
message
.
topic
(),
message
.
partition
(),
message
.
offset
(),
message
.
message
());
lastSeenOffset
=
message
.
offset
();
return
(
String
)
message
.
message
();
return
(
String
)
message
.
message
();
}
}
...
@@ -79,4 +86,14 @@ public class KafkaConsumer<T> extends AbstractNotificationConsumer<T> {
...
@@ -79,4 +86,14 @@ public class KafkaConsumer<T> extends AbstractNotificationConsumer<T> {
MessageAndMetadata
message
=
(
MessageAndMetadata
)
iterator
.
peek
();
MessageAndMetadata
message
=
(
MessageAndMetadata
)
iterator
.
peek
();
return
(
String
)
message
.
message
();
return
(
String
)
message
.
message
();
}
}
@Override
public
void
commit
()
{
if
(
autoCommitEnabled
)
{
LOG
.
debug
(
"Auto commit is disabled, not committing."
);
}
else
{
consumerConnector
.
commitOffsets
();
LOG
.
debug
(
"Committed offset: {}"
,
lastSeenOffset
);
}
}
}
}
notification/src/main/java/org/apache/atlas/kafka/KafkaNotification.java
View file @
98769871
...
@@ -17,6 +17,7 @@
...
@@ -17,6 +17,7 @@
*/
*/
package
org
.
apache
.
atlas
.
kafka
;
package
org
.
apache
.
atlas
.
kafka
;
import
com.google.common.annotations.VisibleForTesting
;
import
com.google.inject.Singleton
;
import
com.google.inject.Singleton
;
import
kafka.consumer.Consumer
;
import
kafka.consumer.Consumer
;
import
kafka.consumer.KafkaStream
;
import
kafka.consumer.KafkaStream
;
...
@@ -112,9 +113,6 @@ public class KafkaNotification extends AbstractNotification implements Service {
...
@@ -112,9 +113,6 @@ public class KafkaNotification extends AbstractNotification implements Service {
"org.apache.kafka.common.serialization.StringSerializer"
);
"org.apache.kafka.common.serialization.StringSerializer"
);
properties
.
put
(
ProducerConfig
.
VALUE_SERIALIZER_CLASS_CONFIG
,
properties
.
put
(
ProducerConfig
.
VALUE_SERIALIZER_CLASS_CONFIG
,
"org.apache.kafka.common.serialization.StringSerializer"
);
"org.apache.kafka.common.serialization.StringSerializer"
);
properties
.
put
(
ConsumerConfig
.
ENABLE_AUTO_COMMIT_CONFIG
,
"true"
);
properties
.
put
(
ConsumerConfig
.
KEY_DESERIALIZER_CLASS_CONFIG
,
properties
.
put
(
ConsumerConfig
.
KEY_DESERIALIZER_CLASS_CONFIG
,
"org.apache.kafka.common.serialization.StringDeserializer"
);
"org.apache.kafka.common.serialization.StringDeserializer"
);
properties
.
put
(
ConsumerConfig
.
VALUE_DESERIALIZER_CLASS_CONFIG
,
properties
.
put
(
ConsumerConfig
.
VALUE_DESERIALIZER_CLASS_CONFIG
,
...
@@ -123,6 +121,10 @@ public class KafkaNotification extends AbstractNotification implements Service {
...
@@ -123,6 +121,10 @@ public class KafkaNotification extends AbstractNotification implements Service {
properties
.
put
(
ConsumerConfig
.
AUTO_OFFSET_RESET_CONFIG
,
"smallest"
);
properties
.
put
(
ConsumerConfig
.
AUTO_OFFSET_RESET_CONFIG
,
"smallest"
);
}
}
@VisibleForTesting
protected
KafkaNotification
(
Properties
properties
)
{
this
.
properties
=
properties
;
}
// ----- Service ---------------------------------------------------------
// ----- Service ---------------------------------------------------------
...
@@ -159,26 +161,34 @@ public class KafkaNotification extends AbstractNotification implements Service {
...
@@ -159,26 +161,34 @@ public class KafkaNotification extends AbstractNotification implements Service {
@Override
@Override
public
<
T
>
List
<
NotificationConsumer
<
T
>>
createConsumers
(
NotificationType
notificationType
,
public
<
T
>
List
<
NotificationConsumer
<
T
>>
createConsumers
(
NotificationType
notificationType
,
int
numConsumers
)
{
int
numConsumers
)
{
return
createConsumers
(
notificationType
,
numConsumers
,
Boolean
.
valueOf
(
properties
.
getProperty
(
"auto.commit.enable"
,
"true"
)));
}
@VisibleForTesting
public
<
T
>
List
<
NotificationConsumer
<
T
>>
createConsumers
(
NotificationType
notificationType
,
int
numConsumers
,
boolean
autoCommitEnabled
)
{
String
topic
=
TOPIC_MAP
.
get
(
notificationType
);
String
topic
=
TOPIC_MAP
.
get
(
notificationType
);
Properties
consumerProperties
=
getConsumerProperties
(
notificationType
);
Properties
consumerProperties
=
getConsumerProperties
(
notificationType
);
List
<
NotificationConsumer
<
T
>>
consumers
=
new
ArrayList
<>(
numConsumers
);
for
(
int
i
=
0
;
i
<
numConsumers
;
i
++)
{
ConsumerConnector
consumerConnector
=
createConsumerConnector
(
consumerProperties
);
ConsumerConnector
consumerConnector
=
createConsumerConnector
(
consumerProperties
);
Map
<
String
,
Integer
>
topicCountMap
=
new
HashMap
<>();
Map
<
String
,
Integer
>
topicCountMap
=
new
HashMap
<>();
topicCountMap
.
put
(
topic
,
numConsumers
);
topicCountMap
.
put
(
topic
,
1
);
StringDecoder
decoder
=
new
StringDecoder
(
null
);
StringDecoder
decoder
=
new
StringDecoder
(
null
);
Map
<
String
,
List
<
KafkaStream
<
String
,
String
>>>
streamsMap
=
Map
<
String
,
List
<
KafkaStream
<
String
,
String
>>>
streamsMap
=
consumerConnector
.
createMessageStreams
(
topicCountMap
,
decoder
,
decoder
);
consumerConnector
.
createMessageStreams
(
topicCountMap
,
decoder
,
decoder
);
List
<
KafkaStream
<
String
,
String
>>
kafkaConsumers
=
streamsMap
.
get
(
topic
);
List
<
KafkaStream
<
String
,
String
>>
kafkaConsumers
=
streamsMap
.
get
(
topic
);
List
<
NotificationConsumer
<
T
>>
consumers
=
new
ArrayList
<>(
numConsumers
);
int
consumerId
=
0
;
for
(
KafkaStream
stream
:
kafkaConsumers
)
{
for
(
KafkaStream
stream
:
kafkaConsumers
)
{
KafkaConsumer
<
T
>
kafkaConsumer
=
KafkaConsumer
<
T
>
kafkaConsumer
=
createKafkaConsumer
(
notificationType
.
getClassType
(),
notificationType
.
getDeserializer
(),
createKafkaConsumer
(
notificationType
.
getClassType
(),
notificationType
.
getDeserializer
(),
stream
,
consumerId
++
);
stream
,
i
,
consumerConnector
,
autoCommitEnabled
);
consumers
.
add
(
kafkaConsumer
);
consumers
.
add
(
kafkaConsumer
);
}
}
consumerConnectors
.
add
(
consumerConnector
);
consumerConnectors
.
add
(
consumerConnector
);
}
return
consumers
;
return
consumers
;
}
}
...
@@ -245,12 +255,14 @@ public class KafkaNotification extends AbstractNotification implements Service {
...
@@ -245,12 +255,14 @@ public class KafkaNotification extends AbstractNotification implements Service {
* @param stream the Kafka stream
* @param stream the Kafka stream
* @param consumerId the id for the new consumer
* @param consumerId the id for the new consumer
*
*
* @param consumerConnector
* @return a new Kafka consumer
* @return a new Kafka consumer
*/
*/
protected
<
T
>
org
.
apache
.
atlas
.
kafka
.
KafkaConsumer
<
T
>
createKafkaConsumer
(
Class
<
T
>
type
,
protected
<
T
>
org
.
apache
.
atlas
.
kafka
.
KafkaConsumer
<
T
>
MessageDeserializer
<
T
>
deserializer
,
KafkaStream
stream
,
createKafkaConsumer
(
Class
<
T
>
type
,
MessageDeserializer
<
T
>
deserializer
,
KafkaStream
stream
,
int
consumerId
)
{
int
consumerId
,
ConsumerConnector
consumerConnector
,
boolean
autoCommitEnabled
)
{
return
new
org
.
apache
.
atlas
.
kafka
.
KafkaConsumer
<
T
>(
type
,
deserializer
,
stream
,
consumerId
);
return
new
org
.
apache
.
atlas
.
kafka
.
KafkaConsumer
<
T
>(
deserializer
,
stream
,
consumerId
,
consumerConnector
,
autoCommitEnabled
);
}
}
// Get properties for consumer request
// Get properties for consumer request
...
@@ -266,6 +278,7 @@ public class KafkaNotification extends AbstractNotification implements Service {
...
@@ -266,6 +278,7 @@ public class KafkaNotification extends AbstractNotification implements Service {
consumerProperties
.
putAll
(
properties
);
consumerProperties
.
putAll
(
properties
);
consumerProperties
.
put
(
ConsumerConfig
.
GROUP_ID_CONFIG
,
groupId
);
consumerProperties
.
put
(
ConsumerConfig
.
GROUP_ID_CONFIG
,
groupId
);
LOG
.
info
(
"Consumer property: auto.commit.enable: "
+
consumerProperties
.
getProperty
(
"auto.commit.enable"
));
return
consumerProperties
;
return
consumerProperties
;
}
}
...
...
notification/src/main/java/org/apache/atlas/notification/AbstractNotification.java
View file @
98769871
...
@@ -17,6 +17,7 @@
...
@@ -17,6 +17,7 @@
*/
*/
package
org
.
apache
.
atlas
.
notification
;
package
org
.
apache
.
atlas
.
notification
;
import
com.google.common.annotations.VisibleForTesting
;
import
com.google.gson.Gson
;
import
com.google.gson.Gson
;
import
com.google.gson.GsonBuilder
;
import
com.google.gson.GsonBuilder
;
import
com.google.gson.JsonElement
;
import
com.google.gson.JsonElement
;
...
@@ -46,7 +47,7 @@ public abstract class AbstractNotification implements NotificationInterface {
...
@@ -46,7 +47,7 @@ public abstract class AbstractNotification implements NotificationInterface {
*/
*/
public
static
final
MessageVersion
CURRENT_MESSAGE_VERSION
=
new
MessageVersion
(
"1.0.0"
);
public
static
final
MessageVersion
CURRENT_MESSAGE_VERSION
=
new
MessageVersion
(
"1.0.0"
);
p
rivate
static
final
String
PROPERTY_EMBEDDED
=
PROPERTY_PREFIX
+
".embedded"
;
p
ublic
static
final
String
PROPERTY_EMBEDDED
=
PROPERTY_PREFIX
+
".embedded"
;
private
final
boolean
embedded
;
private
final
boolean
embedded
;
private
final
boolean
isHAEnabled
;
private
final
boolean
isHAEnabled
;
...
@@ -59,7 +60,6 @@ public abstract class AbstractNotification implements NotificationInterface {
...
@@ -59,7 +60,6 @@ public abstract class AbstractNotification implements NotificationInterface {
registerTypeAdapter
(
JSONArray
.
class
,
new
JSONArraySerializer
()).
registerTypeAdapter
(
JSONArray
.
class
,
new
JSONArraySerializer
()).
create
();
create
();
// ----- Constructors ----------------------------------------------------
// ----- Constructors ----------------------------------------------------
public
AbstractNotification
(
Configuration
applicationProperties
)
throws
AtlasException
{
public
AbstractNotification
(
Configuration
applicationProperties
)
throws
AtlasException
{
...
@@ -67,6 +67,11 @@ public abstract class AbstractNotification implements NotificationInterface {
...
@@ -67,6 +67,11 @@ public abstract class AbstractNotification implements NotificationInterface {
this
.
isHAEnabled
=
HAConfiguration
.
isHAEnabled
(
applicationProperties
);
this
.
isHAEnabled
=
HAConfiguration
.
isHAEnabled
(
applicationProperties
);
}
}
@VisibleForTesting
protected
AbstractNotification
()
{
embedded
=
false
;
isHAEnabled
=
false
;
}
// ----- NotificationInterface -------------------------------------------
// ----- NotificationInterface -------------------------------------------
...
...
notification/src/main/java/org/apache/atlas/notification/AbstractNotificationConsumer.java
View file @
98769871
...
@@ -68,4 +68,6 @@ public abstract class AbstractNotificationConsumer<T> implements NotificationCon
...
@@ -68,4 +68,6 @@ public abstract class AbstractNotificationConsumer<T> implements NotificationCon
public
T
peek
()
{
public
T
peek
()
{
return
deserializer
.
deserialize
(
peekMessage
());
return
deserializer
.
deserialize
(
peekMessage
());
}
}
public
abstract
void
commit
();
}
}
notification/src/main/java/org/apache/atlas/notification/NotificationConsumer.java
View file @
98769871
...
@@ -43,4 +43,13 @@ public interface NotificationConsumer<T> {
...
@@ -43,4 +43,13 @@ public interface NotificationConsumer<T> {
* @return the next notification
* @return the next notification
*/
*/
T
peek
();
T
peek
();
/**
* Commit the offset of messages that have been successfully processed.
*
* This API should be called when messages read with {@link #next()} have been successfully processed and
* the consumer is ready to handle the next message, which could happen even after a normal or an abnormal
* restart.
*/
void
commit
();
}
}
notification/src/test/java/org/apache/atlas/kafka/KafkaConsumerTest.java
View file @
98769871
...
@@ -20,6 +20,7 @@ package org.apache.atlas.kafka;
...
@@ -20,6 +20,7 @@ package org.apache.atlas.kafka;
import
kafka.consumer.ConsumerIterator
;
import
kafka.consumer.ConsumerIterator
;
import
kafka.consumer.KafkaStream
;
import
kafka.consumer.KafkaStream
;
import
kafka.javaapi.consumer.ConsumerConnector
;
import
kafka.message.MessageAndMetadata
;
import
kafka.message.MessageAndMetadata
;
import
org.apache.atlas.notification.AbstractNotification
;
import
org.apache.atlas.notification.AbstractNotification
;
import
org.apache.atlas.notification.MessageVersion
;
import
org.apache.atlas.notification.MessageVersion
;
...
@@ -33,6 +34,9 @@ import org.apache.atlas.typesystem.IStruct;
...
@@ -33,6 +34,9 @@ import org.apache.atlas.typesystem.IStruct;
import
org.apache.atlas.typesystem.Referenceable
;
import
org.apache.atlas.typesystem.Referenceable
;
import
org.apache.atlas.typesystem.Struct
;
import
org.apache.atlas.typesystem.Struct
;
import
org.codehaus.jettison.json.JSONException
;
import
org.codehaus.jettison.json.JSONException
;
import
org.mockito.Mock
;
import
org.mockito.MockitoAnnotations
;
import
org.testng.annotations.BeforeMethod
;
import
org.testng.annotations.Test
;
import
org.testng.annotations.Test
;
import
java.util.Collections
;
import
java.util.Collections
;
...
@@ -41,6 +45,8 @@ import java.util.List;
...
@@ -41,6 +45,8 @@ import java.util.List;
import
java.util.NoSuchElementException
;
import
java.util.NoSuchElementException
;
import
static
org
.
mockito
.
Mockito
.
mock
;
import
static
org
.
mockito
.
Mockito
.
mock
;
import
static
org
.
mockito
.
Mockito
.
never
;
import
static
org
.
mockito
.
Mockito
.
verify
;
import
static
org
.
mockito
.
Mockito
.
when
;
import
static
org
.
mockito
.
Mockito
.
when
;
import
static
org
.
testng
.
Assert
.*;
import
static
org
.
testng
.
Assert
.*;
...
@@ -51,6 +57,14 @@ public class KafkaConsumerTest {
...
@@ -51,6 +57,14 @@ public class KafkaConsumerTest {
private
static
final
String
TRAIT_NAME
=
"MyTrait"
;
private
static
final
String
TRAIT_NAME
=
"MyTrait"
;
@Mock
private
ConsumerConnector
consumerConnector
;
@BeforeMethod
public
void
setup
()
{
MockitoAnnotations
.
initMocks
(
this
);
}
@Test
@Test
public
void
testNext
()
throws
Exception
{
public
void
testNext
()
throws
Exception
{
KafkaStream
<
String
,
String
>
stream
=
mock
(
KafkaStream
.
class
);
KafkaStream
<
String
,
String
>
stream
=
mock
(
KafkaStream
.
class
);
...
@@ -70,8 +84,9 @@ public class KafkaConsumerTest {
...
@@ -70,8 +84,9 @@ public class KafkaConsumerTest {
when
(
messageAndMetadata
.
message
()).
thenReturn
(
json
);
when
(
messageAndMetadata
.
message
()).
thenReturn
(
json
);
NotificationConsumer
<
HookNotification
.
HookNotificationMessage
>
consumer
=
NotificationConsumer
<
HookNotification
.
HookNotificationMessage
>
consumer
=
new
KafkaConsumer
<>(
NotificationInterface
.
NotificationType
.
HOOK
.
getClassType
(),
new
KafkaConsumer
<>(
NotificationInterface
.
NotificationType
.
HOOK
.
getDeserializer
(),
stream
,
99
);
NotificationInterface
.
NotificationType
.
HOOK
.
getDeserializer
(),
stream
,
99
,
consumerConnector
,
false
);
assertTrue
(
consumer
.
hasNext
());
assertTrue
(
consumer
.
hasNext
());
...
@@ -101,8 +116,9 @@ public class KafkaConsumerTest {
...
@@ -101,8 +116,9 @@ public class KafkaConsumerTest {
when
(
messageAndMetadata
.
message
()).
thenReturn
(
json
);
when
(
messageAndMetadata
.
message
()).
thenReturn
(
json
);
NotificationConsumer
<
HookNotification
.
HookNotificationMessage
>
consumer
=
NotificationConsumer
<
HookNotification
.
HookNotificationMessage
>
consumer
=
new
KafkaConsumer
<>(
NotificationInterface
.
NotificationType
.
HOOK
.
getClassType
(),
new
KafkaConsumer
<>(
NotificationInterface
.
NotificationType
.
HOOK
.
getDeserializer
(),
stream
,
99
);
NotificationInterface
.
NotificationType
.
HOOK
.
getDeserializer
(),
stream
,
99
,
consumerConnector
,
false
);
assertTrue
(
consumer
.
hasNext
());
assertTrue
(
consumer
.
hasNext
());
...
@@ -135,8 +151,9 @@ public class KafkaConsumerTest {
...
@@ -135,8 +151,9 @@ public class KafkaConsumerTest {
when
(
messageAndMetadata
.
message
()).
thenReturn
(
json
);
when
(
messageAndMetadata
.
message
()).
thenReturn
(
json
);
NotificationConsumer
<
HookNotification
.
HookNotificationMessage
>
consumer
=
NotificationConsumer
<
HookNotification
.
HookNotificationMessage
>
consumer
=
new
KafkaConsumer
<>(
NotificationInterface
.
NotificationType
.
HOOK
.
getClassType
(),
new
KafkaConsumer
<>(
NotificationInterface
.
NotificationType
.
HOOK
.
getDeserializer
(),
stream
,
99
);
NotificationInterface
.
NotificationType
.
HOOK
.
getDeserializer
(),
stream
,
99
,
consumerConnector
,
false
);
assertTrue
(
consumer
.
hasNext
());
assertTrue
(
consumer
.
hasNext
());
...
@@ -147,6 +164,32 @@ public class KafkaConsumerTest {
...
@@ -147,6 +164,32 @@ public class KafkaConsumerTest {
assertTrue
(
consumer
.
hasNext
());
assertTrue
(
consumer
.
hasNext
());
}
}
@Test
public
void
testCommitIsCalledIfAutoCommitDisabled
()
{
KafkaStream
<
String
,
String
>
stream
=
mock
(
KafkaStream
.
class
);
NotificationConsumer
<
HookNotification
.
HookNotificationMessage
>
consumer
=
new
KafkaConsumer
<>(
NotificationInterface
.
NotificationType
.
HOOK
.
getDeserializer
(),
stream
,
99
,
consumerConnector
,
false
);
consumer
.
commit
();
verify
(
consumerConnector
).
commitOffsets
();
}
@Test
public
void
testCommitIsNotCalledIfAutoCommitEnabled
()
{
KafkaStream
<
String
,
String
>
stream
=
mock
(
KafkaStream
.
class
);
NotificationConsumer
<
HookNotification
.
HookNotificationMessage
>
consumer
=
new
KafkaConsumer
<>(
NotificationInterface
.
NotificationType
.
HOOK
.
getDeserializer
(),
stream
,
99
,
consumerConnector
,
true
);
consumer
.
commit
();
verify
(
consumerConnector
,
never
()).
commitOffsets
();
}
private
Referenceable
getEntity
(
String
traitName
)
{
private
Referenceable
getEntity
(
String
traitName
)
{
Referenceable
entity
=
EntityNotificationImplTest
.
getEntity
(
"id"
);
Referenceable
entity
=
EntityNotificationImplTest
.
getEntity
(
"id"
);
List
<
IStruct
>
traitInfo
=
new
LinkedList
<>();
List
<
IStruct
>
traitInfo
=
new
LinkedList
<>();
...
...
notification/src/test/java/org/apache/atlas/kafka/KafkaNotificationTest.java
View file @
98769871
...
@@ -17,26 +17,16 @@
...
@@ -17,26 +17,16 @@
*/
*/
package
org
.
apache
.
atlas
.
kafka
;
package
org
.
apache
.
atlas
.
kafka
;
import
com.google.inject.Inject
;
import
kafka.consumer.KafkaStream
;
import
kafka.consumer.KafkaStream
;
import
kafka.javaapi.consumer.ConsumerConnector
;
import
kafka.javaapi.consumer.ConsumerConnector
;
import
kafka.serializer.StringDecoder
;
import
kafka.serializer.StringDecoder
;
import
org.apache.atlas.AtlasException
;
import
org.apache.atlas.notification.MessageDeserializer
;
import
org.apache.atlas.notification.MessageDeserializer
;
import
org.apache.atlas.notification.NotificationConsumer
;
import
org.apache.atlas.notification.NotificationConsumer
;
import
org.apache.atlas.notification.NotificationInterface
;
import
org.apache.atlas.notification.NotificationInterface
;
import
org.apache.atlas.notification.NotificationModule
;
import
org.apache.commons.configuration.Configuration
;
import
org.apache.kafka.clients.consumer.ConsumerConfig
;
import
org.testng.annotations.AfterClass
;
import
org.testng.annotations.BeforeClass
;
import
org.testng.annotations.Guice
;
import
org.testng.annotations.Test
;
import
org.testng.annotations.Test
;
import
java.util.
Collections
;
import
java.util.
ArrayList
;
import
java.util.HashMap
;
import
java.util.HashMap
;
import
java.util.Iterator
;
import
java.util.LinkedList
;
import
java.util.List
;
import
java.util.List
;
import
java.util.Map
;
import
java.util.Map
;
import
java.util.Properties
;
import
java.util.Properties
;
...
@@ -44,99 +34,80 @@ import java.util.Properties;
...
@@ -44,99 +34,80 @@ import java.util.Properties;
import
static
org
.
mockito
.
Matchers
.
any
;
import
static
org
.
mockito
.
Matchers
.
any
;
import
static
org
.
mockito
.
Matchers
.
eq
;
import
static
org
.
mockito
.
Matchers
.
eq
;
import
static
org
.
mockito
.
Mockito
.
mock
;
import
static
org
.
mockito
.
Mockito
.
mock
;
import
static
org
.
mockito
.
Mockito
.
times
;
import
static
org
.
mockito
.
Mockito
.
verify
;
import
static
org
.
mockito
.
Mockito
.
when
;
import
static
org
.
mockito
.
Mockito
.
when
;
import
static
org
.
testng
.
Assert
.
assertEquals
;
import
static
org
.
testng
.
Assert
.
assertEquals
;
import
static
org
.
testng
.
Assert
.
assertTrue
;
import
static
org
.
testng
.
Assert
.
assertTrue
;
@Guice
(
modules
=
NotificationModule
.
class
)
public
class
KafkaNotificationTest
{
public
class
KafkaNotificationTest
{
@Inject
private
KafkaNotification
kafka
;
@BeforeClass
public
void
setUp
()
throws
Exception
{
kafka
.
start
();
}
@Test
@Test
@SuppressWarnings
(
"unchecked"
)
@SuppressWarnings
(
"unchecked"
)
public
void
testCreateConsumers
()
throws
Exception
{
public
void
testCreateConsumers
()
throws
Exception
{
Configuration
configuration
=
mock
(
Configuration
.
class
);
Properties
properties
=
mock
(
Properties
.
class
);
Iterator
iterator
=
mock
(
Iterator
.
class
);
when
(
properties
.
getProperty
(
"entities.group.id"
)).
thenReturn
(
"atlas"
);
ConsumerConnector
consumerConnector
=
mock
(
ConsumerConnector
.
class
);
final
ConsumerConnector
consumerConnector
=
mock
(
ConsumerConnector
.
class
);
KafkaStream
kafkaStream1
=
mock
(
KafkaStream
.
class
);
KafkaStream
kafkaStream2
=
mock
(
KafkaStream
.
class
);
String
groupId
=
"groupId9999"
;
when
(
configuration
.
subset
(
KafkaNotification
.
PROPERTY_PREFIX
)).
thenReturn
(
configuration
);
when
(
configuration
.
getKeys
()).
thenReturn
(
iterator
);
when
(
iterator
.
hasNext
()).
thenReturn
(
true
).
thenReturn
(
false
);
when
(
iterator
.
next
()).
thenReturn
(
"entities."
+
KafkaNotification
.
CONSUMER_GROUP_ID_PROPERTY
);
when
(
configuration
.
getList
(
"entities."
+
KafkaNotification
.
CONSUMER_GROUP_ID_PROPERTY
))
.
thenReturn
(
Collections
.<
Object
>
singletonList
(
groupId
));
Map
<
String
,
List
<
KafkaStream
<
String
,
String
>>>
streamsMap
=
new
HashMap
<>();
List
<
KafkaStream
<
String
,
String
>>
kafkaStreamList
=
new
LinkedList
<>();
kafkaStreamList
.
add
(
kafkaStream1
);
kafkaStreamList
.
add
(
kafkaStream2
);
streamsMap
.
put
(
KafkaNotification
.
ATLAS_ENTITIES_TOPIC
,
kafkaStreamList
);
Map
<
String
,
Integer
>
topicCountMap
=
new
HashMap
<>();
Map
<
String
,
Integer
>
topicCountMap
=
new
HashMap
<>();
topicCountMap
.
put
(
KafkaNotification
.
ATLAS_ENTITIES_TOPIC
,
2
);
topicCountMap
.
put
(
KafkaNotification
.
ATLAS_ENTITIES_TOPIC
,
1
);
Map
<
String
,
List
<
KafkaStream
<
String
,
String
>>>
kafkaStreamsMap
=
new
HashMap
<>();
List
<
KafkaStream
<
String
,
String
>>
kafkaStreams
=
new
ArrayList
<>();
KafkaStream
kafkaStream
=
mock
(
KafkaStream
.
class
);
kafkaStreams
.
add
(
kafkaStream
);
kafkaStreamsMap
.
put
(
KafkaNotification
.
ATLAS_ENTITIES_TOPIC
,
kafkaStreams
);
when
(
consumerConnector
.
createMessageStreams
(
when
(
consumerConnector
.
createMessageStreams
(
eq
(
topicCountMap
),
any
(
StringDecoder
.
class
),
any
(
StringDecoder
.
class
))).
thenReturn
(
s
treamsMap
);
eq
(
topicCountMap
),
any
(
StringDecoder
.
class
),
any
(
StringDecoder
.
class
))).
thenReturn
(
kafkaS
treamsMap
);
TestKafkaNotification
kafkaNotification
=
new
TestKafkaNotification
(
configuration
,
consumerConnector
);
final
KafkaConsumer
consumer1
=
mock
(
KafkaConsumer
.
class
);
final
KafkaConsumer
consumer2
=
mock
(
KafkaConsumer
.
class
);
KafkaNotification
kafkaNotification
=
new
TestKafkaNotification
(
properties
,
consumerConnector
,
consumer1
,
consumer2
);
List
<
NotificationConsumer
<
String
>>
consumers
=
List
<
NotificationConsumer
<
String
>>
consumers
=
kafkaNotification
.
createConsumers
(
NotificationInterface
.
NotificationType
.
ENTITIES
,
2
);
kafkaNotification
.
createConsumers
(
NotificationInterface
.
NotificationType
.
ENTITIES
,
2
);
assertEquals
(
2
,
consumers
.
size
());
verify
(
consumerConnector
,
times
(
2
)).
createMessageStreams
(
eq
(
topicCountMap
),
any
(
StringDecoder
.
class
),
any
(
StringDecoder
.
class
));
// assert that all of the given kafka streams were used to create kafka consumers
assertEquals
(
consumers
.
size
(),
2
);
List
<
KafkaStream
>
streams
=
kafkaNotification
.
kafkaStreams
;
assertTrue
(
consumers
.
contains
(
consumer1
));
assertTrue
(
streams
.
contains
(
kafkaStream1
));
assertTrue
(
consumers
.
contains
(
consumer2
));
assertTrue
(
streams
.
contains
(
kafkaStream2
));
// assert that the given consumer group id was added to the properties used to create the consumer connector
Properties
properties
=
kafkaNotification
.
myProperties
;
assertEquals
(
groupId
,
properties
.
getProperty
(
ConsumerConfig
.
GROUP_ID_CONFIG
));
}
}
@AfterClass
class
TestKafkaNotification
extends
KafkaNotification
{
public
void
teardown
()
throws
Exception
{
kafka
.
stop
();
}
// Extended kafka notification class for testing.
private
static
class
TestKafkaNotification
extends
KafkaNotification
{
private
final
ConsumerConnector
consumerConnector
;
private
final
ConsumerConnector
consumerConnector
;
private
final
KafkaConsumer
consumer1
;
private
final
KafkaConsumer
consumer2
;
private
Properties
myProperties
;
TestKafkaNotification
(
Properties
properties
,
ConsumerConnector
consumerConnector
,
private
List
<
KafkaStream
>
kafkaStreams
=
new
LinkedList
<>();
KafkaConsumer
consumer1
,
KafkaConsumer
consumer2
)
{
super
(
properties
);
public
TestKafkaNotification
(
Configuration
applicationProperties
,
ConsumerConnector
consumerConnector
)
throws
AtlasException
{
super
(
applicationProperties
);
this
.
consumerConnector
=
consumerConnector
;
this
.
consumerConnector
=
consumerConnector
;
this
.
consumer1
=
consumer1
;
this
.
consumer2
=
consumer2
;
}
}
@Override
@Override
protected
ConsumerConnector
createConsumerConnector
(
Properties
consumerProperties
)
{
protected
ConsumerConnector
createConsumerConnector
(
Properties
consumerProperties
)
{
this
.
myProperties
=
consumerProperties
;
kafkaStreams
.
clear
();
return
consumerConnector
;
return
consumerConnector
;
}
}
@Override
@Override
protected
<
T
>
org
.
apache
.
atlas
.
kafka
.
KafkaConsumer
<
T
>
createKafkaConsumer
(
Class
<
T
>
type
,
protected
<
T
>
org
.
apache
.
atlas
.
kafka
.
KafkaConsumer
<
T
>
MessageDeserializer
<
T
>
deserializer
,
createKafkaConsumer
(
Class
<
T
>
type
,
MessageDeserializer
<
T
>
deserializer
,
KafkaStream
stream
,
KafkaStream
stream
,
int
consumerId
,
ConsumerConnector
connector
,
boolean
autoCommitEnabled
)
{
int
consumerId
)
{
if
(
consumerId
==
0
)
{
kafkaStreams
.
add
(
stream
);
return
consumer1
;
return
super
.
createKafkaConsumer
(
type
,
deserializer
,
stream
,
consumerId
);
}
else
if
(
consumerId
==
1
)
{
return
consumer2
;
}
}
return
null
;
}
}
}
}
}
notification/src/test/java/org/apache/atlas/notification/AbstractNotificationConsumerTest.java
View file @
98769871
...
@@ -253,6 +253,11 @@ public class AbstractNotificationConsumerTest {
...
@@ -253,6 +253,11 @@ public class AbstractNotificationConsumerTest {
public
boolean
hasNext
()
{
public
boolean
hasNext
()
{
return
index
<
messageList
.
size
();
return
index
<
messageList
.
size
();
}
}
@Override
public
void
commit
()
{
// do nothing.
}
}
}
private
static
final
class
TestDeserializer
<
T
>
extends
VersionedMessageDeserializer
<
T
>
{
private
static
final
class
TestDeserializer
<
T
>
extends
VersionedMessageDeserializer
<
T
>
{
...
...
release-log.txt
View file @
98769871
...
@@ -20,6 +20,7 @@ ATLAS-409 Atlas will not import avro tables with schema read from a file (dosset
...
@@ -20,6 +20,7 @@ ATLAS-409 Atlas will not import avro tables with schema read from a file (dosset
ATLAS-379 Create sqoop and falcon metadata addons (venkatnrangan,bvellanki,sowmyaramesh via shwethags)
ATLAS-379 Create sqoop and falcon metadata addons (venkatnrangan,bvellanki,sowmyaramesh via shwethags)
ALL CHANGES:
ALL CHANGES:
ATLAS-629 Kafka messages in ATLAS_HOOK might be lost in HA mode at the instant of failover. (yhemanth)
ATLAS-758 hdfs location of hive table is pointing to old location even after rename ( sumasai )
ATLAS-758 hdfs location of hive table is pointing to old location even after rename ( sumasai )
ATLAS-667 Entity delete should check for required reverse references ( dkantor via sumasai )
ATLAS-667 Entity delete should check for required reverse references ( dkantor via sumasai )
ATLAS-738 Add query ability on system properties like guid, state, createdtime etc (shwethags)
ATLAS-738 Add query ability on system properties like guid, state, createdtime etc (shwethags)
...
...
typesystem/src/main/resources/atlas-application.properties
View file @
98769871
...
@@ -70,6 +70,7 @@ atlas.kafka.consumer.timeout.ms=100
...
@@ -70,6 +70,7 @@ atlas.kafka.consumer.timeout.ms=100
atlas.kafka.auto.commit.interval.ms
=
100
atlas.kafka.auto.commit.interval.ms
=
100
atlas.kafka.hook.group.id
=
atlas
atlas.kafka.hook.group.id
=
atlas
atlas.kafka.entities.group.id
=
atlas_entities
atlas.kafka.entities.group.id
=
atlas_entities
atlas.kafka.auto.commit.enable
=
false
######### Entity Audit Configs #########
######### Entity Audit Configs #########
atlas.audit.hbase.tablename
=
ATLAS_ENTITY_AUDIT_EVENTS
atlas.audit.hbase.tablename
=
ATLAS_ENTITY_AUDIT_EVENTS
...
...
webapp/src/main/java/org/apache/atlas/notification/NotificationHookConsumer.java
View file @
98769871
...
@@ -17,6 +17,7 @@
...
@@ -17,6 +17,7 @@
*/
*/
package
org
.
apache
.
atlas
.
notification
;
package
org
.
apache
.
atlas
.
notification
;
import
com.google.common.annotations.VisibleForTesting
;
import
com.google.common.util.concurrent.ThreadFactoryBuilder
;
import
com.google.common.util.concurrent.ThreadFactoryBuilder
;
import
com.google.inject.Inject
;
import
com.google.inject.Inject
;
import
com.google.inject.Singleton
;
import
com.google.inject.Singleton
;
...
@@ -183,7 +184,16 @@ public class NotificationHookConsumer implements Service, ActiveStateChangeHandl
...
@@ -183,7 +184,16 @@ public class NotificationHookConsumer implements Service, ActiveStateChangeHandl
while
(
shouldRun
.
get
())
{
while
(
shouldRun
.
get
())
{
try
{
try
{
if
(
hasNext
())
{
if
(
hasNext
())
{
HookNotification
.
HookNotificationMessage
message
=
consumer
.
next
();
handleMessage
(
consumer
.
next
());
}
}
catch
(
Throwable
t
)
{
LOG
.
warn
(
"Failure in NotificationHookConsumer"
,
t
);
}
}
}
@VisibleForTesting
void
handleMessage
(
HookNotification
.
HookNotificationMessage
message
)
{
atlasClient
.
setUser
(
message
.
getUser
());
atlasClient
.
setUser
(
message
.
getUser
());
try
{
try
{
switch
(
message
.
getType
())
{
switch
(
message
.
getType
())
{
...
@@ -222,11 +232,7 @@ public class NotificationHookConsumer implements Service, ActiveStateChangeHandl
...
@@ -222,11 +232,7 @@ public class NotificationHookConsumer implements Service, ActiveStateChangeHandl
//todo handle failures
//todo handle failures
LOG
.
warn
(
"Error handling message {}"
,
message
,
e
);
LOG
.
warn
(
"Error handling message {}"
,
message
,
e
);
}
}
}
consumer
.
commit
();
}
catch
(
Throwable
t
)
{
LOG
.
warn
(
"Failure in NotificationHookConsumer"
,
t
);
}
}
}
}
boolean
serverAvailable
(
Timer
timer
)
{
boolean
serverAvailable
(
Timer
timer
)
{
...
...
webapp/src/test/java/org/apache/atlas/notification/NotificationHookConsumerKafkaTest.java
0 → 100644
View file @
98769871
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package
org
.
apache
.
atlas
.
notification
;
import
com.google.inject.Inject
;
import
org.apache.atlas.AtlasClient
;
import
org.apache.atlas.AtlasException
;
import
org.apache.atlas.LocalAtlasClient
;
import
org.apache.atlas.kafka.KafkaNotification
;
import
org.apache.atlas.notification.hook.HookNotification
;
import
org.apache.atlas.typesystem.Referenceable
;
import
org.apache.commons.lang.RandomStringUtils
;
import
org.testng.annotations.AfterTest
;
import
org.testng.annotations.BeforeTest
;
import
org.testng.annotations.Guice
;
import
org.testng.annotations.Test
;
import
static
org
.
mockito
.
Mockito
.
mock
;
import
static
org
.
mockito
.
Mockito
.
verify
;
@Guice
(
modules
=
NotificationModule
.
class
)
public
class
NotificationHookConsumerKafkaTest
{
@Inject
private
NotificationInterface
notificationInterface
;
private
KafkaNotification
kafkaNotification
;
@BeforeTest
public
void
setup
()
throws
AtlasException
{
kafkaNotification
=
startKafkaServer
();
}
@AfterTest
public
void
shutdown
()
{
kafkaNotification
.
stop
();
}
@Test
public
void
testConsumerConsumesNewMessageWithAutoCommitDisabled
()
throws
AtlasException
,
InterruptedException
{
produceMessage
(
new
HookNotification
.
EntityCreateRequest
(
"test_user1"
,
createEntity
()));
NotificationConsumer
<
HookNotification
.
HookNotificationMessage
>
consumer
=
createNewConsumer
(
kafkaNotification
,
false
);
LocalAtlasClient
localAtlasClient
=
mock
(
LocalAtlasClient
.
class
);
NotificationHookConsumer
notificationHookConsumer
=
new
NotificationHookConsumer
(
kafkaNotification
,
localAtlasClient
);
NotificationHookConsumer
.
HookConsumer
hookConsumer
=
notificationHookConsumer
.
new
HookConsumer
(
consumer
);
consumeOneMessage
(
consumer
,
hookConsumer
);
verify
(
localAtlasClient
).
setUser
(
"test_user1"
);
// produce another message, and make sure it moves ahead. If commit succeeded, this would work.
produceMessage
(
new
HookNotification
.
EntityCreateRequest
(
"test_user2"
,
createEntity
()));
consumeOneMessage
(
consumer
,
hookConsumer
);
verify
(
localAtlasClient
).
setUser
(
"test_user2"
);
kafkaNotification
.
close
();
}
@Test
public
void
testConsumerRemainsAtSameMessageWithAutoCommitEnabled
()
throws
NotificationException
,
InterruptedException
{
produceMessage
(
new
HookNotification
.
EntityCreateRequest
(
"test_user3"
,
createEntity
()));
NotificationConsumer
<
HookNotification
.
HookNotificationMessage
>
consumer
=
createNewConsumer
(
kafkaNotification
,
true
);
LocalAtlasClient
localAtlasClient
=
mock
(
LocalAtlasClient
.
class
);
NotificationHookConsumer
notificationHookConsumer
=
new
NotificationHookConsumer
(
kafkaNotification
,
localAtlasClient
);
NotificationHookConsumer
.
HookConsumer
hookConsumer
=
notificationHookConsumer
.
new
HookConsumer
(
consumer
);
consumeOneMessage
(
consumer
,
hookConsumer
);
verify
(
localAtlasClient
).
setUser
(
"test_user3"
);
// produce another message, but this will not be consumed, as commit code is not executed in hook consumer.
produceMessage
(
new
HookNotification
.
EntityCreateRequest
(
"test_user4"
,
createEntity
()));
consumeOneMessage
(
consumer
,
hookConsumer
);
verify
(
localAtlasClient
).
setUser
(
"test_user3"
);
kafkaNotification
.
close
();
}
NotificationConsumer
<
HookNotification
.
HookNotificationMessage
>
createNewConsumer
(
KafkaNotification
kafkaNotification
,
boolean
autoCommitEnabled
)
{
return
kafkaNotification
.<
HookNotification
.
HookNotificationMessage
>
createConsumers
(
NotificationInterface
.
NotificationType
.
HOOK
,
1
,
autoCommitEnabled
).
get
(
0
);
}
void
consumeOneMessage
(
NotificationConsumer
<
HookNotification
.
HookNotificationMessage
>
consumer
,
NotificationHookConsumer
.
HookConsumer
hookConsumer
)
throws
InterruptedException
{
while
(!
consumer
.
hasNext
())
{
Thread
.
sleep
(
1000
);
}
hookConsumer
.
handleMessage
(
consumer
.
next
());
}
Referenceable
createEntity
()
{
final
Referenceable
entity
=
new
Referenceable
(
AtlasClient
.
DATA_SET_SUPER_TYPE
);
entity
.
set
(
"name"
,
"db"
+
randomString
());
entity
.
set
(
"description"
,
randomString
());
return
entity
;
}
KafkaNotification
startKafkaServer
()
throws
AtlasException
{
KafkaNotification
kafkaNotification
=
(
KafkaNotification
)
notificationInterface
;
kafkaNotification
.
start
();
return
kafkaNotification
;
}
protected
String
randomString
()
{
return
RandomStringUtils
.
randomAlphanumeric
(
10
);
}
private
void
produceMessage
(
HookNotification
.
HookNotificationMessage
message
)
throws
NotificationException
{
notificationInterface
.
send
(
NotificationInterface
.
NotificationType
.
HOOK
,
message
);
}
}
webapp/src/test/java/org/apache/atlas/notification/NotificationHookConsumerTest.java
View file @
98769871
...
@@ -21,6 +21,7 @@ import org.apache.atlas.AtlasClient;
...
@@ -21,6 +21,7 @@ import org.apache.atlas.AtlasClient;
import
org.apache.atlas.AtlasServiceException
;
import
org.apache.atlas.AtlasServiceException
;
import
org.apache.atlas.LocalAtlasClient
;
import
org.apache.atlas.LocalAtlasClient
;
import
org.apache.atlas.ha.HAConfiguration
;
import
org.apache.atlas.ha.HAConfiguration
;
import
org.apache.atlas.notification.hook.HookNotification
;
import
org.apache.commons.configuration.Configuration
;
import
org.apache.commons.configuration.Configuration
;
import
org.mockito.Mock
;
import
org.mockito.Mock
;
import
org.mockito.MockitoAnnotations
;
import
org.mockito.MockitoAnnotations
;
...
@@ -34,6 +35,7 @@ import java.util.concurrent.ExecutorService;
...
@@ -34,6 +35,7 @@ import java.util.concurrent.ExecutorService;
import
static
org
.
mockito
.
Mockito
.
any
;
import
static
org
.
mockito
.
Mockito
.
any
;
import
static
org
.
mockito
.
Mockito
.
doThrow
;
import
static
org
.
mockito
.
Mockito
.
doThrow
;
import
static
org
.
mockito
.
Mockito
.
mock
;
import
static
org
.
mockito
.
Mockito
.
mock
;
import
static
org
.
mockito
.
Mockito
.
never
;
import
static
org
.
mockito
.
Mockito
.
times
;
import
static
org
.
mockito
.
Mockito
.
times
;
import
static
org
.
mockito
.
Mockito
.
verify
;
import
static
org
.
mockito
.
Mockito
.
verify
;
import
static
org
.
mockito
.
Mockito
.
verifyZeroInteractions
;
import
static
org
.
mockito
.
Mockito
.
verifyZeroInteractions
;
...
@@ -87,6 +89,40 @@ public class NotificationHookConsumerTest {
...
@@ -87,6 +89,40 @@ public class NotificationHookConsumerTest {
}
}
@Test
@Test
public
void
testCommitIsCalledWhenMessageIsProcessed
()
throws
AtlasServiceException
{
NotificationHookConsumer
notificationHookConsumer
=
new
NotificationHookConsumer
(
notificationInterface
,
atlasClient
);
NotificationConsumer
consumer
=
mock
(
NotificationConsumer
.
class
);
NotificationHookConsumer
.
HookConsumer
hookConsumer
=
notificationHookConsumer
.
new
HookConsumer
(
consumer
);
HookNotification
.
EntityCreateRequest
message
=
mock
(
HookNotification
.
EntityCreateRequest
.
class
);
when
(
message
.
getUser
()).
thenReturn
(
"user"
);
when
(
message
.
getType
()).
thenReturn
(
HookNotification
.
HookNotificationType
.
ENTITY_CREATE
);
hookConsumer
.
handleMessage
(
message
);
verify
(
consumer
).
commit
();
}
@Test
public
void
testCommitIsCalledEvenWhenMessageProcessingFails
()
throws
AtlasServiceException
{
NotificationHookConsumer
notificationHookConsumer
=
new
NotificationHookConsumer
(
notificationInterface
,
atlasClient
);
NotificationConsumer
consumer
=
mock
(
NotificationConsumer
.
class
);
NotificationHookConsumer
.
HookConsumer
hookConsumer
=
notificationHookConsumer
.
new
HookConsumer
(
consumer
);
HookNotification
.
EntityCreateRequest
message
=
mock
(
HookNotification
.
EntityCreateRequest
.
class
);
when
(
message
.
getUser
()).
thenReturn
(
"user"
);
when
(
message
.
getType
()).
thenReturn
(
HookNotification
.
HookNotificationType
.
ENTITY_CREATE
);
when
(
atlasClient
.
createEntity
(
any
(
List
.
class
))).
thenThrow
(
new
RuntimeException
(
"Simulating exception in processing message"
));
hookConsumer
.
handleMessage
(
message
);
verify
(
consumer
).
commit
();
}
@Test
public
void
testConsumerProceedsWithFalseIfInterrupted
()
throws
AtlasServiceException
,
InterruptedException
{
public
void
testConsumerProceedsWithFalseIfInterrupted
()
throws
AtlasServiceException
,
InterruptedException
{
NotificationHookConsumer
notificationHookConsumer
=
new
NotificationHookConsumer
(
notificationInterface
,
atlasClient
);
NotificationHookConsumer
notificationHookConsumer
=
new
NotificationHookConsumer
(
notificationInterface
,
atlasClient
);
NotificationHookConsumer
.
HookConsumer
hookConsumer
=
NotificationHookConsumer
.
HookConsumer
hookConsumer
=
...
...
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