Re: Review Request 52598: FLUME-2999 - Kafka channel and sink should enable statically assigned partition per event via header

2016-10-07 Thread Tristan Stevens


> On Oct. 7, 2016, 2:25 p.m., Mike Percy wrote:
> > flume-ng-sinks/flume-ng-kafka-sink/src/main/java/org/apache/flume/sink/kafka/KafkaSink.java,
> >  line 214
> > 
> >
> > nit: missing space after comma. Did you run checkstyle?

Yes, not sure why it wasn't picked up.


> On Oct. 7, 2016, 2:25 p.m., Mike Percy wrote:
> > flume-ng-sinks/flume-ng-kafka-sink/src/main/java/org/apache/flume/sink/kafka/KafkaSink.java,
> >  line 221
> > 
> >
> > Do you really need to catch Exception? If so, then why bother catching 
> > all the rest as well?

// N.B. The producer.send() method throws all sorts of RuntimeExceptions
  // Catching Exception here to wrap them neatly in an 
EventDeliveryException
  // which is what our consumers will expect


> On Oct. 7, 2016, 2:25 p.m., Mike Percy wrote:
> > flume-ng-sinks/flume-ng-kafka-sink/src/test/java/org/apache/flume/sink/kafka/util/KafkaConsumer.java,
> >  line 55
> > 
> >
> > hmm, does this change our user contract or not really?

It's only in test, but actually it was a mistake. Reverted.


> On Oct. 7, 2016, 2:25 p.m., Mike Percy wrote:
> > flume-ng-doc/sphinx/FlumeUserGuide.rst, line 1268
> > 
> >
> > I am confused about what this means in the context of the source.
> > 
> > 2. 1. Did you really mean to put this in the docs for the source?
> > 2. s/channel/source/ on the first line here ?

I knew I did that somewhere but couldn't find it. It's entirely wrong - removed.


> On Oct. 7, 2016, 2:25 p.m., Mike Percy wrote:
> > flume-ng-sinks/flume-ng-kafka-sink/src/test/java/org/apache/flume/sink/kafka/TestKafkaSink.java,
> >  line 391
> > 
> >
> > Is this just copy / paste from the other one? Can we find a way to make 
> > them share code?

My Maven fu is not quite there - I've refactored the method out, but need to 
get them into a shared module that is accessible only for test.


- Tristan


---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/52598/#review151795
---


On Oct. 6, 2016, 12:13 p.m., Tristan Stevens wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/52598/
> ---
> 
> (Updated Oct. 6, 2016, 12:13 p.m.)
> 
> 
> Review request for Flume and Grant Henke.
> 
> 
> Repository: flume-git
> 
> 
> Description
> ---
> 
> This feature is useful for anyone who needs greater control of which 
> partitions are being written to - normally in a situation where multiple 
> Flume agents are being deployed in order to horizontally scale, or 
> alternatively if there is a scenario where there is a skew in data that might 
> lead to one or more partitions hotspotting.
> We also have the ability to specify custom partitions on to the Kafka 
> Producer itself using the kafka.* configuration properties.
> 
> The Kafka Producer provides the ability to set the partition ID using the 
> following constructor 
> (https://kafka.apache.org/090/javadoc/org/apache/kafka/clients/producer/ProducerRecord.html#ProducerRecord%28java.lang.String,%20java.lang.Integer,%20K,%20V%29
>  ), this is just a matter of providing the option to use this constructor.
> 
> This is specified in one of two ways: either via the staticPartition 
> configuration property, which means that every message goes to the specified 
> partition, or via the partitionHeader configuration property, which directs 
> the implementation to retrieve the partitionId from one of the event headers.
> 
> 
> Diffs
> -
> 
>   
> flume-ng-channels/flume-kafka-channel/src/main/java/org/apache/flume/channel/kafka/KafkaChannel.java
>  66b553a 
>   
> flume-ng-channels/flume-kafka-channel/src/main/java/org/apache/flume/channel/kafka/KafkaChannelConfiguration.java
>  3ab807b 
>   
> flume-ng-channels/flume-kafka-channel/src/test/java/org/apache/flume/channel/kafka/TestKafkaChannel.java
>  57c0b28 
>   flume-ng-doc/sphinx/FlumeUserGuide.rst ab71d38 
>   
> flume-ng-sinks/flume-ng-kafka-sink/src/main/java/org/apache/flume/sink/kafka/KafkaSink.java
>  89bdd84 
>   
> flume-ng-sinks/flume-ng-kafka-sink/src/main/java/org/apache/flume/sink/kafka/KafkaSinkConstants.java
>  1bf380c 
>   
> flume-ng-sinks/flume-ng-kafka-sink/src/test/java/org/apache/flume/sink/kafka/TestKafkaSink.java
>  76eca37 
>   
> flume-ng-sinks/flume-ng-kafka-sink/src/test/java/org/apache/flume/sink/kafka/util/KafkaConsumer.java
>  

Re: Review Request 52627: FLUME-2971. Document secure Kafka Sink/Source/Channel setup

2016-10-07 Thread Tristan Stevens

---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/52627/#review151849
---




flume-ng-doc/sphinx/FlumeUserGuide.rst (line 1319)


Confusingly, SSL refers to TLS not SSL. So even though the parameter is 
named SSL, the actual protocol is TLS



flume-ng-doc/sphinx/FlumeUserGuide.rst (line 1320)


Presumably we can't use this as we only ship the 0.9 client API in Flume 
1.7?



flume-ng-doc/sphinx/FlumeUserGuide.rst (line 1328)


TLS based encryption with no authentication.



flume-ng-doc/sphinx/FlumeUserGuide.rst (line 1339)


s/SSL/TLS



flume-ng-doc/sphinx/FlumeUserGuide.rst (line 1373)


s/certification/certificate



flume-ng-doc/sphinx/FlumeUserGuide.rst (line 1374)


s/certification/certificate



flume-ng-doc/sphinx/FlumeUserGuide.rst (line 1431)


Since the Kafka Source may also connect to Zookeeper for offset migration, 
the "Client" section was also added to this example. This won't be needed 
unless you require offset migration, or you require this section for other 
secure components.



flume-ng-doc/sphinx/FlumeUserGuide.rst (lines 1436 - 1450)


Do we need to specify useTicketCache=false ? That's what I've used when 
configuring this so far. Although the default is false.

Also, I've not set storeKey=true before.



flume-ng-doc/sphinx/FlumeUserGuide.rst (lines 1455 - 1456)


Are we not documenting this?



flume-ng-doc/sphinx/FlumeUserGuide.rst (line 2884)


It's an oxymoron for the Sink to need to do offset migration. Therefore 
let's rephrase to:

Unlike the Kafka Source / Kafka Channel a "client" section is not required, 
unless it is needed by other connecting components.


Great job Simon - thanks for the time you've put into this. 

I've got a feeling however that we're unnecessarily duplicating some of the 
wordage here. Could we take the whole "Security and Kafka *" section and place 
under the "Security" section - and just make it clear which bits apply to 
consumers (Source and Channel) and which bits apply to Producers (Sink and 
Channel)? I think this would make it tidier and also aid maintainability.

Some of the comments apply to both Source and Sink, but I've only raised them 
once. I also take the point that they are probably also defects in the Channel 
bit, that you didn't write. Sorry about that!

- Tristan Stevens


On Oct. 7, 2016, 1:27 p.m., Attila Simon wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/52627/
> ---
> 
> (Updated Oct. 7, 2016, 1:27 p.m.)
> 
> 
> Review request for Flume.
> 
> 
> Bugs: FLUME-2971
> https://issues.apache.org/jira/browse/FLUME-2971
> 
> 
> Repository: flume-git
> 
> 
> Description
> ---
> 
> The patch aims to extend the existing documentation of secure Kafka channel 
> with describing SSL+Plaintext setup as well as providing the whole package 
> (SSL+Kerberos+Plain) for KafkaSource and KafkaSink.
> 
> 
> Diffs
> -
> 
>   flume-ng-doc/sphinx/FlumeUserGuide.rst ab71d38 
> 
> Diff: https://reviews.apache.org/r/52627/diff/
> 
> 
> Testing
> ---
> 
> "mvn site" generated the user guide without an error message in the html. 
> Embedded links are checked not to be broken.
> 
> Known to require attention: Content of the jaas file has to be checked 
> focusing on the requirement of the Client section in every setup.
> 
> 
> Thanks,
> 
> Attila Simon
> 
>



Re: Review Request 52598: FLUME-2999 - Kafka channel and sink should enable statically assigned partition per event via header

2016-10-07 Thread Attila Simon

---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/52598/#review151794
---



Patch applies clearly on trunk currently (probably will conflicts after 
FLUME-2971). Code compiles with "mvn clean install -DskipTests", site builds 
without error message on the generated html. When running tests I got errors in 
the "mvn test -pl flume-ng-sinks/flume-ng-kafka-sink -fae" results:

Failed tests:   
  - testDefaultTopic(org.apache.flume.sink.kafka.TestKafkaSink): 
expected:<[default-topic-test]> but was:<[9]>
  - testEmptyChannel(org.apache.flume.sink.kafka.TestKafkaSink)
"mvn test -pl flume-ng-sinks/flume-ng-kafka-sink -fae" passes on trunk, so this 
patch broke the tests. 

"mvn test -pl flume-ng-channels/flume-kafka-channel -fae" passed.


flume-ng-channels/flume-kafka-channel/src/main/java/org/apache/flume/channel/kafka/KafkaChannel.java
 (line 439)


NumberFormatException is raised if header contains non number istead of 
defaulting to staticPartitionId and sending the event with that. I agree with 
this behaviour ie dropping whole event because of occasionally malformed 
header. Flume should either rely on the content of a processed event or not 
(including body and headers). What happens with that Event afterwards? Retry or 
backoff?



flume-ng-channels/flume-kafka-channel/src/test/java/org/apache/flume/channel/kafka/TestKafkaChannel.java
 (lines 254 - 288)


Would you mind refactoring these? I would expect an extra test class for 
testing headers, common parts in @Before @After or utility methods and test fn 
body would define the header setup. Please don't use boolean flags+utility fn 
for code reuse. If the extra test class has been already created then the 
slightly better Enum+switch cases can also be refactored there as well to test 
fn bodies.



flume-ng-channels/flume-kafka-channel/src/test/java/org/apache/flume/channel/kafka/TestKafkaChannel.java
 (lines 430 - 449)


Could you please explain this a bit more? I understand you would like to 
get a specific message/topic distribution. What I don't get is why intermixing 
cycle variable with a check for staticPtn==null helping in this task.



flume-ng-channels/flume-kafka-channel/src/test/java/org/apache/flume/channel/kafka/TestKafkaChannel.java
 (lines 463 - 479)


this looks fine. it collects the results from each TopicPartition it could 
have been a utility fn



flume-ng-channels/flume-kafka-channel/src/test/java/org/apache/flume/channel/kafka/TestKafkaChannel.java
 (lines 485 - 497)


Could you please a bit more explicit what these cases cover? eg introducing 
a new booleans with somwhat more descriptive names.



flume-ng-doc/sphinx/FlumeUserGuide.rst (lines 2584 - 2587)


Please do not throw a TODo :)



flume-ng-sinks/flume-ng-kafka-sink/src/main/java/org/apache/flume/sink/kafka/KafkaSink.java
 (lines 217 - 222)


If no difference in the action and the last is base for the specialized 
ones then the specialized ones are not needed. ie only 

"} catch (Exception ex) {...}" 

is required



flume-ng-sinks/flume-ng-kafka-sink/src/test/java/org/apache/flume/sink/kafka/util/KafkaConsumer.java
 (line 55)


Is this utility used outside of this class? I haven't found any references. 
If not then I think visibility modifier shouldn't change.


- Attila Simon


On Oct. 6, 2016, 12:13 p.m., Tristan Stevens wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/52598/
> ---
> 
> (Updated Oct. 6, 2016, 12:13 p.m.)
> 
> 
> Review request for Flume and Grant Henke.
> 
> 
> Repository: flume-git
> 
> 
> Description
> ---
> 
> This feature is useful for anyone who needs greater control of which 
> partitions are being written to - normally in a situation where multiple 
> Flume agents are being deployed in order to horizontally scale, or 
> alternatively if there is a scenario where there is a skew in data that might 
> lead to one or more partitions hotspotting.
> We also have the ability to specify custom partitions on to the Kafka 
> Producer itself using the kafka.* configuration properties.
> 
> The Kafka Producer provides the ability to set the partition ID using the 
> following constructor 
> 

Re: Review Request 52598: FLUME-2999 - Kafka channel and sink should enable statically assigned partition per event via header

2016-10-07 Thread Mike Percy

---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/52598/#review151795
---




flume-ng-channels/flume-kafka-channel/src/main/java/org/apache/flume/channel/kafka/KafkaChannelConfiguration.java
 (line 52)


please see my comments on the docs w.r.t. naming



flume-ng-channels/flume-kafka-channel/src/test/java/org/apache/flume/channel/kafka/TestKafkaChannel.java
 (line 369)


Please add doc comment



flume-ng-channels/flume-kafka-channel/src/test/java/org/apache/flume/channel/kafka/TestKafkaChannel.java
 (line 399)


Pls document what the different values mean, i.e. when staticPtn == null 
what does that mean; also what are the params to this function.



flume-ng-channels/flume-kafka-channel/src/test/java/org/apache/flume/channel/kafka/TestKafkaChannel.java
 (line 400)


style nit: This should just be a Javadoc comment. However, in comments, for 
readability, please add punctuation (periods) to your sentences, and also leave 
a space after the "//" in your comments, like this:

// This method tests both the default behavior 
(usePartitionHeader=false)
// and the behaviour when the partitionId setting is used.
// Under the default behaviour you would expect an even distribution of
// messages to partitions, however when partitionId is used we manually 
create
// a large skew to some partitions and then verify that this actually 
happened.



flume-ng-channels/flume-kafka-channel/src/test/java/org/apache/flume/channel/kafka/TestKafkaChannel.java
 (line 407)


If that is the case why not use a test-wide constant for it?



flume-ng-channels/flume-kafka-channel/src/test/java/org/apache/flume/channel/kafka/TestKafkaChannel.java
 (line 430)


Would you mind factoring this loop into its own helper function, since it 
seems very self-contained?



flume-ng-channels/flume-kafka-channel/src/test/java/org/apache/flume/channel/kafka/TestKafkaChannel.java
 (line 484)


mind moving this comment to the top of the for-loop?



flume-ng-doc/sphinx/FlumeUserGuide.rst (line 1268)


I am confused about what this means in the context of the source.

2. 1. Did you really mean to put this in the docs for the source?
2. s/channel/source/ on the first line here ?



flume-ng-doc/sphinx/FlumeUserGuide.rst (line 2582)


defaultPartitionId  might be a better name. Also, what happens when neither 
defaultPartitionId nor the partition ID header are present? Who decides the 
partition then? (i.e. what is the current behavior). Would be good to document 
that as well.



flume-ng-doc/sphinx/FlumeUserGuide.rst (line 2584)


Hmm, how about partitionIdHeader



flume-ng-doc/sphinx/FlumeUserGuide.rst (line 2586)


TODO here? An error will be thrown and the event will be rejected from the 
channel?



flume-ng-doc/sphinx/FlumeUserGuide.rst (line 2791)


same as above



flume-ng-doc/sphinx/FlumeUserGuide.rst (line 2793)


same as above



flume-ng-sinks/flume-ng-kafka-sink/src/main/java/org/apache/flume/sink/kafka/KafkaSink.java
 (line 214)


nit: missing space after comma. Did you run checkstyle?



flume-ng-sinks/flume-ng-kafka-sink/src/main/java/org/apache/flume/sink/kafka/KafkaSink.java
 (line 216)


also missing space after comma on this line



flume-ng-sinks/flume-ng-kafka-sink/src/main/java/org/apache/flume/sink/kafka/KafkaSink.java
 (line 221)


Do you really need to catch Exception? If so, then why bother catching all 
the rest as well?



flume-ng-sinks/flume-ng-kafka-sink/src/test/java/org/apache/flume/sink/kafka/TestKafkaSink.java
 (line 391)


Is this just copy / paste from the other one? Can we find a way to make 
them share code?



flume-ng-sinks/flume-ng-kafka-sink/src/test/java/org/apache/flume/sink/kafka/util/KafkaConsumer.java
 (line 55)


hmm, does this change our user contract or not really?


- Mike Percy


On Oct. 6, 2016, 5:13 a.m., 

Re: Review Request 52627: FLUME-2971. Document secure Kafka Sink/Source/Channel setup

2016-10-07 Thread Balázs Donát Bessenyei

---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/52627/#review151790
---




flume-ng-doc/sphinx/FlumeUserGuide.rst 


Why did these parameters get removed?



flume-ng-doc/sphinx/FlumeUserGuide.rst (line 3101)


~ and Kafka Channel?


- Balázs Donát Bessenyei


On Oct. 7, 2016, 1:27 p.m., Attila Simon wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/52627/
> ---
> 
> (Updated Oct. 7, 2016, 1:27 p.m.)
> 
> 
> Review request for Flume.
> 
> 
> Bugs: FLUME-2971
> https://issues.apache.org/jira/browse/FLUME-2971
> 
> 
> Repository: flume-git
> 
> 
> Description
> ---
> 
> The patch aims to extend the existing documentation of secure Kafka channel 
> with describing SSL+Plaintext setup as well as providing the whole package 
> (SSL+Kerberos+Plain) for KafkaSource and KafkaSink.
> 
> 
> Diffs
> -
> 
>   flume-ng-doc/sphinx/FlumeUserGuide.rst ab71d38 
> 
> Diff: https://reviews.apache.org/r/52627/diff/
> 
> 
> Testing
> ---
> 
> "mvn site" generated the user guide without an error message in the html. 
> Embedded links are checked not to be broken.
> 
> Known to require attention: Content of the jaas file has to be checked 
> focusing on the requirement of the Client section in every setup.
> 
> 
> Thanks,
> 
> Attila Simon
> 
>



Re: Review Request 52627: FLUME-2971. Document secure Kafka Sink/Source/Channel setup

2016-10-07 Thread Attila Simon

---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/52627/
---

(Updated Oct. 7, 2016, 1:27 p.m.)


Review request for Flume.


Bugs: FLUME-2971
https://issues.apache.org/jira/browse/FLUME-2971


Repository: flume-git


Description
---

The patch aims to extend the existing documentation of secure Kafka channel 
with describing SSL+Plaintext setup as well as providing the whole package 
(SSL+Kerberos+Plain) for KafkaSource and KafkaSink.


Diffs (updated)
-

  flume-ng-doc/sphinx/FlumeUserGuide.rst ab71d38 

Diff: https://reviews.apache.org/r/52627/diff/


Testing
---

"mvn site" generated the user guide without an error message in the html. 
Embedded links are checked not to be broken.

Known to require attention: Content of the jaas file has to be checked focusing 
on the requirement of the Client section in every setup.


Thanks,

Attila Simon



Build failed in Jenkins: Flume-trunk-hbase-1 #207

2016-10-07 Thread Apache Jenkins Server
See 

Changes:

[bessbd] FLUME-3003. Fix flaky testSourceCounter in TestSyslogUdpSource

--
[...truncated 6730 lines...]
  TestFileChannel.testInOut:117 »  Failed to locate tar-ball distribution. 
Pleas...
  TestFileChannel.tearDown:95 »  Failed to locate tar-ball distribution. Please 
...
  TestRpcClientCommunicationFailure.testFailure:61 »  Failed to locate tar-ball 
...
  TestRpcClient.setUp:36 »  Failed to locate tar-ball distribution. Please 
speci...
  TestRpcClient.tearDown:42 »  Failed to locate tar-ball distribution. Please 
sp...
  TestSpooldirSource.setup:59 »  Failed to locate tar-ball distribution. Please 
...
  TestSpooldirSource.teardown:111 »  Failed to locate tar-ball distribution. 
Ple...
  TestSyslogSource.testKeepFields:72 »  Failed to locate tar-ball distribution. 
...
  TestSyslogSource.tearDown:63 »  Failed to locate tar-ball distribution. 
Please...
  TestSyslogSource.testRemoveFields:82 »  Failed to locate tar-ball 
distribution...
  TestSyslogSource.tearDown:63 »  Failed to locate tar-ball distribution. 
Please...
  TestSyslogSource.testKeepTimestampAndHostname:92 »  Failed to locate tar-ball 
...
  TestSyslogSource.tearDown:63 »  Failed to locate tar-ball distribution. 
Please...
  TestSyslogSource.testKeepFields:72 »  Failed to locate tar-ball distribution. 
...
  TestSyslogSource.tearDown:63 »  Failed to locate tar-ball distribution. 
Please...
  TestSyslogSource.testRemoveFields:82 »  Failed to locate tar-ball 
distribution...
  TestSyslogSource.tearDown:63 »  Failed to locate tar-ball distribution. 
Please...
  TestSyslogSource.testKeepTimestampAndHostname:92 »  Failed to locate tar-ball 
...
  TestSyslogSource.tearDown:63 »  Failed to locate tar-ball distribution. 
Please...

Tests run: 19, Failures: 0, Errors: 19, Skipped: 0

[ERROR] There are test failures.

Please refer to 

 for the individual test results.
[JENKINS] Recording test results
[INFO] 
[INFO] --- maven-jar-plugin:3.0.0:jar (default-jar) @ flume-ng-tests ---
[INFO] Building jar: 

[INFO] 
[INFO] --- maven-site-plugin:3.3:attach-descriptor (attach-descriptor) @ 
flume-ng-tests ---
[INFO] 
[INFO] --- apache-rat-plugin:0.11:check (verify.rat) @ flume-ng-tests ---
[INFO] 51 implicit excludes (use -debug for more details).
[INFO] Exclude: **/.idea/
[INFO] Exclude: **/*.iml
[INFO] Exclude: **/nb-configuration.xml
[INFO] Exclude: .git/
[INFO] Exclude: patchprocess/
[INFO] Exclude: .gitignore
[INFO] Exclude: .repository/
[INFO] Exclude: **/*.diff
[INFO] Exclude: **/*.patch
[INFO] Exclude: **/*.avsc
[INFO] Exclude: **/*.avro
[INFO] Exclude: **/docs/**
[INFO] Exclude: **/test/resources/**
[INFO] Exclude: **/.settings/*
[INFO] Exclude: **/.classpath
[INFO] Exclude: **/.project
[INFO] Exclude: **/target/**
[INFO] Exclude: **/derby.log
[INFO] Exclude: **/metastore_db/
[INFO] 9 resources included (use -debug for more details)
[INFO] Rat check: Summary of files. Unapproved: 0 unknown: 0 generated: 0 
approved: 9 licence.
[INFO] 
[INFO] --- maven-checkstyle-plugin:2.17:check (verify) @ flume-ng-tests ---
[INFO] 
[INFO] --- maven-install-plugin:2.5.2:install (default-install) @ 
flume-ng-tests ---
[INFO] Installing 

 to 

[INFO] Installing 
 
to 

[INFO] 
[INFO] Reactor Summary:
[INFO] 
[INFO] Flume checkstyle project ... SUCCESS [  8.036 s]
[INFO] Apache Flume ... SUCCESS [ 13.374 s]
[INFO] Flume NG SDK ... SUCCESS [01:45 min]
[INFO] Flume NG Configuration . SUCCESS [  5.473 s]
[INFO] Flume Auth . SUCCESS [ 40.981 s]
[INFO] Flume NG Core .. SUCCESS [08:38 min]
[INFO] Flume NG Sinks . SUCCESS [  0.885 s]
[INFO] Flume NG HDFS Sink . SUCCESS [02:34 min]
[INFO] Flume NG IRC Sink .. SUCCESS [  4.318 s]
[INFO] Flume NG Channels .. SUCCESS [  0.925 s]
[INFO] Flume NG JDBC channel .. SUCCESS [03:32 min]
[INFO] Flume NG file-based 

[jira] [Commented] (FLUME-3003) testSourceCounter in TestSyslogUdpSource is flaky

2016-10-07 Thread Hudson (JIRA)

[ 
https://issues.apache.org/jira/browse/FLUME-3003?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=1043#comment-1043
 ] 

Hudson commented on FLUME-3003:
---

FAILURE: Integrated in Jenkins build Flume-trunk-hbase-1 #207 (See 
[https://builds.apache.org/job/Flume-trunk-hbase-1/207/])
FLUME-3003. Fix flaky testSourceCounter in TestSyslogUdpSource (bessbd: 
[http://git-wip-us.apache.org/repos/asf/flume/repo?p=flume.git=commit=27c725802b16915acaa3ea05cdac0236972af1f0])
* (edit) 
flume-ng-core/src/test/java/org/apache/flume/source/TestSyslogUdpSource.java


> testSourceCounter in TestSyslogUdpSource is flaky
> -
>
> Key: FLUME-3003
> URL: https://issues.apache.org/jira/browse/FLUME-3003
> Project: Flume
>  Issue Type: Test
>  Components: Sinks+Sources
>Affects Versions: v1.6.0
>Reporter: Denes Arvay
>Assignee: Denes Arvay
> Fix For: v1.7.0
>
>




--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


Re: [DISCUSS] Flume 1.7 release plan

2016-10-07 Thread Mike Percy
+1 from me, keep up the good work guys.

Mike

On Fri, Oct 7, 2016 at 12:59 PM, Balazs Donat Bessenyei  wrote:

> Considering Attila's request, I'll do the branching on Monday and the
> first RC soon after that.
>
> Is that okay for everyone?
>
>
> Thank you,
>
> Donat
>
> On Fri, Oct 7, 2016 at 12:53 PM, Attila Simon  wrote:
> > Hi,
> >
> > Since we have a few patches in flight from me,Tristan,etc let me suggest
> > giving the weekend for code review rounds and branch on Monday.
> >
> > Cheers,
> > Attila
> >
> >
> > *Attila Simon*
> > Software Engineer
> > Email:   s...@cloudera.com
> >
> > [image: Cloudera Inc.]
> >
> > On Thu, Oct 6, 2016 at 12:05 PM, Balazs Donat Bessenyei <
> bes...@cloudera.com
> >> wrote:
> >
> >> Thank you all for the responses.
> >>
> >> @Lior: I have reviewed the change you have mentioned (FLUME-2994). If
> >> we manage to have the code done and +1/2-d soon, I'll commit the
> >> patch.
> >>
> >> @Denes: the change looks good to me. I'll leave some time for others
> >> to review, though.
> >>
> >> @Saikat: I've wanted to put integrations in the User Guide for Flume
> >> for so long. For example Flume with Spark (
> >> http://spark.apache.org/docs/latest/streaming-flume-integration.html
> >> ).
> >> I know it is available on the internet and I don't think we should
> >> increase redundancy this way, but I'd really like to have such
> >> examples in the docs.
> >> What do you all think of this?
> >>
> >> Misc: any help with the tickets I've mentioned in my last e-mail would
> >> be super welcome. (Even just triaging them and maybe deciding if they
> >> are not that important as I first thought.)
> >>
> >>
> >> Thank you,
> >>
> >> Donat
> >>
> >> On Wed, Oct 5, 2016 at 2:03 AM, Hari Shreedharan
> >>  wrote:
> >> > +1 for the release, and the branching and RC dates
> >> >
> >> > On Tue, Oct 4, 2016 at 10:54 AM Saikat Kanjilal 
> >> wrote:
> >> >
> >> >> Hi Donat,
> >> >>
> >> >> I can pickup any documentation related issues, are there anymore at
> this
> >> >> point outside of the FLUME-2971, I would be willing to pickup one of
> the
> >> >> ones below but cant guarantee that it'll be fixed in the timeframe
> for
> >> 1.7
> >> >> , let me know the best path for helping.
> >> >>
> >> >> Thanks
> >> >>
> >> >>
> >> >> 
> >> >> From: Balazs Donat Bessenyei 
> >> >> Sent: Tuesday, October 4, 2016 8:01 AM
> >> >> To: dev@flume.apache.org
> >> >> Subject: Re: [DISCUSS] Flume 1.7 release plan
> >> >>
> >> >> As there have been no objections, I am going to proceed with the plan
> >> >> I have outlined in my original mail.
> >> >>
> >> >> I will try to work a little more on some flaky tests to help with the
> >> >> release process. (Such as FLUME-3002: Some tests in TestBucketWriter
> >> >> are flaky.)
> >> >>
> >> >> It would be awesome if someone could fix FLUME-2971 (Document
> Kerberos
> >> >> set-up for Kafka Sink and Kafka Source).
> >> >>
> >> >> Also, I have found a few more tickets that could use a little care:
> >> >> FLUME-2689 (reloading conf file leads syslogTcpSource not receives
> any
> >> >> event) - probably only needs some more reviews,
> >> >> FLUME-2716 (File Channel cannot handle capacity Integer.MAX_VALUE) -
> >> >> missing a test,
> >> >> FLUME-2461 (memoryChannel bytesRemaining counting error) - missing
> test
> >> >> Patch-less (Needs some investigation and a fix. I am not sure about
> >> >> their complexity):
> >> >> FLUME-2912 (thrift Sources/Sinks can only authenticate with kerberos
> >> >> principal in format with hostname)
> >> >> FLUME-2871 (avro sink reset-connection-interval cause
> >> >> EventDeliveryException)
> >> >> FLUME-2811 (Taildir source doesn't call stop() on graceful shutdown)
> >> >>
> >> >> However, none of these seem to be a release blocker, so if they are
> >> >> not done by Oct 7-10, it would still be fine.
> >> >>
> >> >>
> >> >> Please, let me know your thoughts
> >> >>
> >> >> Donat
> >> >>
> >> >> On Mon, Oct 3, 2016 at 3:23 PM, Mike Percy 
> wrote:
> >> >> > On Fri, Sep 30, 2016 at 7:01 PM, Saikat Kanjilal <
> sxk1...@hotmail.com
> >> >
> >> >> > wrote:
> >> >> >
> >> >> >> I'd be willing to help, I've been heads down on other stuff and
> have
> >> had
> >> >> >> to postpone the graph sink implementation (
> >> https://issues.apache.org/
> >> >> >> jira/browse/FLUME-2035) but am looking to resume work and
> targeting
> >> an
> >> >> >> initial implementation by mid November sometime.  In the meantime
> >> let me
> >> >> >> know how I can get more deeply involved in the next release.
> >> >> >>
> >> >> >
> >> >> > I'd recommend taking a look at things like documentation and
> filling
> >> in
> >> >> > gaps so that we don't release with undocumented stuff. Other types
> of
> >> >> > polish seem like a helpful thing to do right before a release as
> well.
> >> >> >
> >> >> > Mike
> >> 

Re: [DISCUSS] Flume 1.7 release plan

2016-10-07 Thread Balazs Donat Bessenyei
Considering Attila's request, I'll do the branching on Monday and the
first RC soon after that.

Is that okay for everyone?


Thank you,

Donat

On Fri, Oct 7, 2016 at 12:53 PM, Attila Simon  wrote:
> Hi,
>
> Since we have a few patches in flight from me,Tristan,etc let me suggest
> giving the weekend for code review rounds and branch on Monday.
>
> Cheers,
> Attila
>
>
> *Attila Simon*
> Software Engineer
> Email:   s...@cloudera.com
>
> [image: Cloudera Inc.]
>
> On Thu, Oct 6, 2016 at 12:05 PM, Balazs Donat Bessenyei > wrote:
>
>> Thank you all for the responses.
>>
>> @Lior: I have reviewed the change you have mentioned (FLUME-2994). If
>> we manage to have the code done and +1/2-d soon, I'll commit the
>> patch.
>>
>> @Denes: the change looks good to me. I'll leave some time for others
>> to review, though.
>>
>> @Saikat: I've wanted to put integrations in the User Guide for Flume
>> for so long. For example Flume with Spark (
>> http://spark.apache.org/docs/latest/streaming-flume-integration.html
>> ).
>> I know it is available on the internet and I don't think we should
>> increase redundancy this way, but I'd really like to have such
>> examples in the docs.
>> What do you all think of this?
>>
>> Misc: any help with the tickets I've mentioned in my last e-mail would
>> be super welcome. (Even just triaging them and maybe deciding if they
>> are not that important as I first thought.)
>>
>>
>> Thank you,
>>
>> Donat
>>
>> On Wed, Oct 5, 2016 at 2:03 AM, Hari Shreedharan
>>  wrote:
>> > +1 for the release, and the branching and RC dates
>> >
>> > On Tue, Oct 4, 2016 at 10:54 AM Saikat Kanjilal 
>> wrote:
>> >
>> >> Hi Donat,
>> >>
>> >> I can pickup any documentation related issues, are there anymore at this
>> >> point outside of the FLUME-2971, I would be willing to pickup one of the
>> >> ones below but cant guarantee that it'll be fixed in the timeframe for
>> 1.7
>> >> , let me know the best path for helping.
>> >>
>> >> Thanks
>> >>
>> >>
>> >> 
>> >> From: Balazs Donat Bessenyei 
>> >> Sent: Tuesday, October 4, 2016 8:01 AM
>> >> To: dev@flume.apache.org
>> >> Subject: Re: [DISCUSS] Flume 1.7 release plan
>> >>
>> >> As there have been no objections, I am going to proceed with the plan
>> >> I have outlined in my original mail.
>> >>
>> >> I will try to work a little more on some flaky tests to help with the
>> >> release process. (Such as FLUME-3002: Some tests in TestBucketWriter
>> >> are flaky.)
>> >>
>> >> It would be awesome if someone could fix FLUME-2971 (Document Kerberos
>> >> set-up for Kafka Sink and Kafka Source).
>> >>
>> >> Also, I have found a few more tickets that could use a little care:
>> >> FLUME-2689 (reloading conf file leads syslogTcpSource not receives any
>> >> event) - probably only needs some more reviews,
>> >> FLUME-2716 (File Channel cannot handle capacity Integer.MAX_VALUE) -
>> >> missing a test,
>> >> FLUME-2461 (memoryChannel bytesRemaining counting error) - missing test
>> >> Patch-less (Needs some investigation and a fix. I am not sure about
>> >> their complexity):
>> >> FLUME-2912 (thrift Sources/Sinks can only authenticate with kerberos
>> >> principal in format with hostname)
>> >> FLUME-2871 (avro sink reset-connection-interval cause
>> >> EventDeliveryException)
>> >> FLUME-2811 (Taildir source doesn't call stop() on graceful shutdown)
>> >>
>> >> However, none of these seem to be a release blocker, so if they are
>> >> not done by Oct 7-10, it would still be fine.
>> >>
>> >>
>> >> Please, let me know your thoughts
>> >>
>> >> Donat
>> >>
>> >> On Mon, Oct 3, 2016 at 3:23 PM, Mike Percy  wrote:
>> >> > On Fri, Sep 30, 2016 at 7:01 PM, Saikat Kanjilal > >
>> >> > wrote:
>> >> >
>> >> >> I'd be willing to help, I've been heads down on other stuff and have
>> had
>> >> >> to postpone the graph sink implementation (
>> https://issues.apache.org/
>> >> >> jira/browse/FLUME-2035) but am looking to resume work and targeting
>> an
>> >> >> initial implementation by mid November sometime.  In the meantime
>> let me
>> >> >> know how I can get more deeply involved in the next release.
>> >> >>
>> >> >
>> >> > I'd recommend taking a look at things like documentation and filling
>> in
>> >> > gaps so that we don't release with undocumented stuff. Other types of
>> >> > polish seem like a helpful thing to do right before a release as well.
>> >> >
>> >> > Mike
>> >>
>>


Re: Review Request 52627: FLUME-2971. Document secure Kafka Sink/Source/Channel setup

2016-10-07 Thread Attila Simon

---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/52627/
---

(Updated Oct. 7, 2016, 11:18 a.m.)


Review request for Flume.


Bugs: FLUME-2971
https://issues.apache.org/jira/browse/FLUME-2971


Repository: flume-git


Description
---

The patch aims to extend the existing documentation of secure Kafka channel 
with describing SSL+Plaintext setup as well as providing the whole package 
(SSL+Kerberos+Plain) for KafkaSource and KafkaSink.


Diffs
-

  flume-ng-doc/sphinx/FlumeUserGuide.rst ab71d38 

Diff: https://reviews.apache.org/r/52627/diff/


Testing (updated)
---

"mvn site" generated the user guide without an error message in the html. 
Embedded links are checked not to be broken.

Known to require attention: Content of the jaas file has to be checked focusing 
on the requirement of the Client section in every setup.


Thanks,

Attila Simon



[jira] [Updated] (FLUME-2971) Document secure Kafka Sink/Source/Channel setup

2016-10-07 Thread Attila Simon (JIRA)

 [ 
https://issues.apache.org/jira/browse/FLUME-2971?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Attila Simon updated FLUME-2971:

Attachment: FLUME-2971-1.patch

> Document secure Kafka Sink/Source/Channel setup
> ---
>
> Key: FLUME-2971
> URL: https://issues.apache.org/jira/browse/FLUME-2971
> Project: Flume
>  Issue Type: Documentation
>  Components: Channel, Sinks+Sources
>Affects Versions: v1.7.0
>Reporter: Tristan Stevens
>Assignee: Attila Simon
> Attachments: FLUME-2971-0.patch, FLUME-2971-1.patch
>
>
> As per FLUME-2901 but for Kafka Source and Kafka Sink



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


Re: Review Request 52627: FLUME-2971. Document secure Kafka Sink/Source/Channel setup

2016-10-07 Thread Attila Simon

---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/52627/
---

(Updated Oct. 7, 2016, 11:13 a.m.)


Review request for Flume.


Bugs: FLUME-2971
https://issues.apache.org/jira/browse/FLUME-2971


Repository: flume-git


Description
---

The patch aims to extend the existing documentation of secure Kafka channel 
with describing SSL+Plaintext setup as well as providing the whole package 
(SSL+Kerberos+Plain) for KafkaSource and KafkaSink.


Diffs (updated)
-

  flume-ng-doc/sphinx/FlumeUserGuide.rst ab71d38 

Diff: https://reviews.apache.org/r/52627/diff/


Testing
---

"mvn site" generated the user guide without an error message in the html. 
Embedded links are checked not to be broken.


Thanks,

Attila Simon



Re: [DISCUSS] Flume 1.7 release plan

2016-10-07 Thread Attila Simon
Hi,

Since we have a few patches in flight from me,Tristan,etc let me suggest
giving the weekend for code review rounds and branch on Monday.

Cheers,
Attila


*Attila Simon*
Software Engineer
Email:   s...@cloudera.com

[image: Cloudera Inc.]

On Thu, Oct 6, 2016 at 12:05 PM, Balazs Donat Bessenyei  wrote:

> Thank you all for the responses.
>
> @Lior: I have reviewed the change you have mentioned (FLUME-2994). If
> we manage to have the code done and +1/2-d soon, I'll commit the
> patch.
>
> @Denes: the change looks good to me. I'll leave some time for others
> to review, though.
>
> @Saikat: I've wanted to put integrations in the User Guide for Flume
> for so long. For example Flume with Spark (
> http://spark.apache.org/docs/latest/streaming-flume-integration.html
> ).
> I know it is available on the internet and I don't think we should
> increase redundancy this way, but I'd really like to have such
> examples in the docs.
> What do you all think of this?
>
> Misc: any help with the tickets I've mentioned in my last e-mail would
> be super welcome. (Even just triaging them and maybe deciding if they
> are not that important as I first thought.)
>
>
> Thank you,
>
> Donat
>
> On Wed, Oct 5, 2016 at 2:03 AM, Hari Shreedharan
>  wrote:
> > +1 for the release, and the branching and RC dates
> >
> > On Tue, Oct 4, 2016 at 10:54 AM Saikat Kanjilal 
> wrote:
> >
> >> Hi Donat,
> >>
> >> I can pickup any documentation related issues, are there anymore at this
> >> point outside of the FLUME-2971, I would be willing to pickup one of the
> >> ones below but cant guarantee that it'll be fixed in the timeframe for
> 1.7
> >> , let me know the best path for helping.
> >>
> >> Thanks
> >>
> >>
> >> 
> >> From: Balazs Donat Bessenyei 
> >> Sent: Tuesday, October 4, 2016 8:01 AM
> >> To: dev@flume.apache.org
> >> Subject: Re: [DISCUSS] Flume 1.7 release plan
> >>
> >> As there have been no objections, I am going to proceed with the plan
> >> I have outlined in my original mail.
> >>
> >> I will try to work a little more on some flaky tests to help with the
> >> release process. (Such as FLUME-3002: Some tests in TestBucketWriter
> >> are flaky.)
> >>
> >> It would be awesome if someone could fix FLUME-2971 (Document Kerberos
> >> set-up for Kafka Sink and Kafka Source).
> >>
> >> Also, I have found a few more tickets that could use a little care:
> >> FLUME-2689 (reloading conf file leads syslogTcpSource not receives any
> >> event) - probably only needs some more reviews,
> >> FLUME-2716 (File Channel cannot handle capacity Integer.MAX_VALUE) -
> >> missing a test,
> >> FLUME-2461 (memoryChannel bytesRemaining counting error) - missing test
> >> Patch-less (Needs some investigation and a fix. I am not sure about
> >> their complexity):
> >> FLUME-2912 (thrift Sources/Sinks can only authenticate with kerberos
> >> principal in format with hostname)
> >> FLUME-2871 (avro sink reset-connection-interval cause
> >> EventDeliveryException)
> >> FLUME-2811 (Taildir source doesn't call stop() on graceful shutdown)
> >>
> >> However, none of these seem to be a release blocker, so if they are
> >> not done by Oct 7-10, it would still be fine.
> >>
> >>
> >> Please, let me know your thoughts
> >>
> >> Donat
> >>
> >> On Mon, Oct 3, 2016 at 3:23 PM, Mike Percy  wrote:
> >> > On Fri, Sep 30, 2016 at 7:01 PM, Saikat Kanjilal  >
> >> > wrote:
> >> >
> >> >> I'd be willing to help, I've been heads down on other stuff and have
> had
> >> >> to postpone the graph sink implementation (
> https://issues.apache.org/
> >> >> jira/browse/FLUME-2035) but am looking to resume work and targeting
> an
> >> >> initial implementation by mid November sometime.  In the meantime
> let me
> >> >> know how I can get more deeply involved in the next release.
> >> >>
> >> >
> >> > I'd recommend taking a look at things like documentation and filling
> in
> >> > gaps so that we don't release with undocumented stuff. Other types of
> >> > polish seem like a helpful thing to do right before a release as well.
> >> >
> >> > Mike
> >>
>


[jira] [Commented] (FLUME-2971) Document secure Kafka Sink/Source/Channel setup

2016-10-07 Thread Tristan Stevens (JIRA)

[ 
https://issues.apache.org/jira/browse/FLUME-2971?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15554757#comment-15554757
 ] 

Tristan Stevens commented on FLUME-2971:


[~sati] We should include a reference to 
https://issues.apache.org/jira/browse/KAFKA-2561 - some users have reported 
severe degradation of performance when using Kafka+TLS

> Document secure Kafka Sink/Source/Channel setup
> ---
>
> Key: FLUME-2971
> URL: https://issues.apache.org/jira/browse/FLUME-2971
> Project: Flume
>  Issue Type: Documentation
>  Components: Channel, Sinks+Sources
>Affects Versions: v1.7.0
>Reporter: Tristan Stevens
>Assignee: Attila Simon
> Attachments: FLUME-2971-0.patch
>
>
> As per FLUME-2901 but for Kafka Source and Kafka Sink



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (FLUME-2971) Document secure Kafka Sink/Source/Channel setup

2016-10-07 Thread Attila Simon (JIRA)

 [ 
https://issues.apache.org/jira/browse/FLUME-2971?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Attila Simon updated FLUME-2971:

Component/s: Channel

> Document secure Kafka Sink/Source/Channel setup
> ---
>
> Key: FLUME-2971
> URL: https://issues.apache.org/jira/browse/FLUME-2971
> Project: Flume
>  Issue Type: Documentation
>  Components: Channel, Sinks+Sources
>Affects Versions: v1.7.0
>Reporter: Tristan Stevens
>Assignee: Attila Simon
> Attachments: FLUME-2971-0.patch
>
>
> As per FLUME-2901 but for Kafka Source and Kafka Sink



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (FLUME-2971) Document secure Kafka Sink/Source/Channel setup

2016-10-07 Thread Attila Simon (JIRA)

 [ 
https://issues.apache.org/jira/browse/FLUME-2971?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Attila Simon updated FLUME-2971:

Issue Type: Documentation  (was: Bug)

> Document secure Kafka Sink/Source/Channel setup
> ---
>
> Key: FLUME-2971
> URL: https://issues.apache.org/jira/browse/FLUME-2971
> Project: Flume
>  Issue Type: Documentation
>  Components: Channel, Sinks+Sources
>Affects Versions: v1.7.0
>Reporter: Tristan Stevens
>Assignee: Attila Simon
> Attachments: FLUME-2971-0.patch
>
>
> As per FLUME-2901 but for Kafka Source and Kafka Sink



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (FLUME-2971) Document secure Kafka Sink/Source/Channel setup

2016-10-07 Thread Attila Simon (JIRA)

 [ 
https://issues.apache.org/jira/browse/FLUME-2971?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Attila Simon updated FLUME-2971:

Summary: Document secure Kafka Sink/Source/Channel setup  (was: Document 
Kerberos set-up for Kafka Sink and Kafka Source)

> Document secure Kafka Sink/Source/Channel setup
> ---
>
> Key: FLUME-2971
> URL: https://issues.apache.org/jira/browse/FLUME-2971
> Project: Flume
>  Issue Type: Bug
>  Components: Sinks+Sources
>Affects Versions: v1.7.0
>Reporter: Tristan Stevens
>Assignee: Attila Simon
> Attachments: FLUME-2971-0.patch
>
>
> As per FLUME-2901 but for Kafka Source and Kafka Sink



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


Review Request 52627: FLUME-2971. Document secure Kafka Sink/Source/Channel setup

2016-10-07 Thread Attila Simon

---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/52627/
---

Review request for Flume.


Bugs: FLUME-2971
https://issues.apache.org/jira/browse/FLUME-2971


Repository: flume-git


Description
---

The patch aims to extend the existing documentation of secure Kafka channel 
with describing SSL+Plaintext setup as well as providing the whole package 
(SSL+Kerberos+Plain) for KafkaSource and KafkaSink.


Diffs
-

  flume-ng-doc/sphinx/FlumeUserGuide.rst ab71d38 

Diff: https://reviews.apache.org/r/52627/diff/


Testing
---

"mvn site" generated the user guide without an error message in the html. 
Embedded links are checked not to be broken.


Thanks,

Attila Simon



[jira] [Commented] (FLUME-2971) Document Kerberos set-up for Kafka Sink and Kafka Source

2016-10-07 Thread Attila Simon (JIRA)

[ 
https://issues.apache.org/jira/browse/FLUME-2971?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15554728#comment-15554728
 ] 

Attila Simon commented on FLUME-2971:
-

The patch aims to extend the existing documentation of secure Kafka channel 
with describing SSL+Plaintext setup as well as providing the whole package 
(SSL+Kerberos+Plain) for KafkaSource and KafkaSink. 

> Document Kerberos set-up for Kafka Sink and Kafka Source
> 
>
> Key: FLUME-2971
> URL: https://issues.apache.org/jira/browse/FLUME-2971
> Project: Flume
>  Issue Type: Bug
>  Components: Sinks+Sources
>Affects Versions: v1.7.0
>Reporter: Tristan Stevens
>Assignee: Attila Simon
> Attachments: FLUME-2971-0.patch
>
>
> As per FLUME-2901 but for Kafka Source and Kafka Sink



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (FLUME-2971) Document Kerberos set-up for Kafka Sink and Kafka Source

2016-10-07 Thread Attila Simon (JIRA)

 [ 
https://issues.apache.org/jira/browse/FLUME-2971?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Attila Simon updated FLUME-2971:

Attachment: FLUME-2971-0.patch

> Document Kerberos set-up for Kafka Sink and Kafka Source
> 
>
> Key: FLUME-2971
> URL: https://issues.apache.org/jira/browse/FLUME-2971
> Project: Flume
>  Issue Type: Bug
>  Components: Sinks+Sources
>Affects Versions: v1.7.0
>Reporter: Tristan Stevens
>Assignee: Attila Simon
> Attachments: FLUME-2971-0.patch
>
>
> As per FLUME-2901 but for Kafka Source and Kafka Sink



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Resolved] (FLUME-3003) testSourceCounter in TestSyslogUdpSource is flaky

2016-10-07 Thread JIRA

 [ 
https://issues.apache.org/jira/browse/FLUME-3003?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Bessenyei Balázs Donát resolved FLUME-3003.
---
Resolution: Fixed

> testSourceCounter in TestSyslogUdpSource is flaky
> -
>
> Key: FLUME-3003
> URL: https://issues.apache.org/jira/browse/FLUME-3003
> Project: Flume
>  Issue Type: Test
>  Components: Sinks+Sources
>Affects Versions: v1.6.0
>Reporter: Denes Arvay
>Assignee: Denes Arvay
> Fix For: v1.7.0
>
>




--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (FLUME-3003) testSourceCounter in TestSyslogUdpSource is flaky

2016-10-07 Thread ASF subversion and git services (JIRA)

[ 
https://issues.apache.org/jira/browse/FLUME-3003?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15554714#comment-15554714
 ] 

ASF subversion and git services commented on FLUME-3003:


Commit 27c725802b16915acaa3ea05cdac0236972af1f0 in flume's branch 
refs/heads/trunk from [~denes]
[ https://git-wip-us.apache.org/repos/asf?p=flume.git;h=27c7258 ]

FLUME-3003. Fix flaky testSourceCounter in TestSyslogUdpSource

The event processing in SyslogUDPSource is handled on a separate thread by 
Netty so message delivery, thus the sourceCounter's increment can be delayed 
resulting in a flaky test.

This patch fixes that issue by adding more delay when necessary.

Reviewers: Lior Zeno, Bessenyei Balázs Donát

(Denes Arvay via Bessenyei Balázs Donát)


> testSourceCounter in TestSyslogUdpSource is flaky
> -
>
> Key: FLUME-3003
> URL: https://issues.apache.org/jira/browse/FLUME-3003
> Project: Flume
>  Issue Type: Test
>  Components: Sinks+Sources
>Affects Versions: v1.6.0
>Reporter: Denes Arvay
>Assignee: Denes Arvay
> Fix For: v1.7.0
>
>




--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (FLUME-3003) testSourceCounter in TestSyslogUdpSource is flaky

2016-10-07 Thread JIRA

[ 
https://issues.apache.org/jira/browse/FLUME-3003?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15554669#comment-15554669
 ] 

Bessenyei Balázs Donát commented on FLUME-3003:
---

I'm about to commit this

> testSourceCounter in TestSyslogUdpSource is flaky
> -
>
> Key: FLUME-3003
> URL: https://issues.apache.org/jira/browse/FLUME-3003
> Project: Flume
>  Issue Type: Test
>  Components: Sinks+Sources
>Affects Versions: v1.6.0
>Reporter: Denes Arvay
>Assignee: Denes Arvay
> Fix For: v1.7.0
>
>




--
This message was sent by Atlassian JIRA
(v6.3.4#6332)