Commit Graph

6093 Commits

Author SHA1 Message Date
Colin P. Mccabe a674ded0b3 MINOR: fix throttling and status in ConnectionStressWorker
Each separate thread should have its own throttle, so that it can sleep
for an appropriate amount of time when needed.

ConnectionStressWorker should avoid recalculating the status after
shutting down the runnables.  Otherwise, if one runnable is slow to
stop, it will skew the average down in a way that doesn't reflect
reality.  This change moves the status calculation into a separate
periodic runnable that gets shut down cleanly before the other ones.

Author: Colin P. Mccabe <cmccabe@confluent.io>

Reviewers: Gwen Shapira, Stanislav Kozlovski

Closes #6533 from cmccabe/fix_connection_stress_worker
2019-04-04 14:16:56 -07:00
Mickael Maison c301025484 KAFKA-8090: Use automatic RPC generation in ControlledShutdown
Author: Mickael Maison <mickael.maison@gmail.com>

Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>

Closes #6423 from mimaison/controlled-shutdown
2019-04-04 22:05:04 +05:30
John Roesler 9bd0d6aa93 KAFKA-6399: Remove Streams max.poll.interval override (#6509)
Since we now call poll during restore, we can decrease the timeout
to a reasonable value, which should help Streams make progress if
threads get stuck.

Reviewers: Guozhang Wang <wangguoz@gmail.com>,  Bill Bejeck <bbejeck@gmail.com>
2019-04-04 09:38:25 -04:00
Doroszlai, Attila d3316bc6a7 KAFKA-8126: Flaky Test org.apache.kafka.connect.runtime.WorkerTest.testAddRemoveTask (#6475)
Changed the WorkerTest to use a mock Executor.

Author: Attila Doroszlai <adoroszlai@apache.org>
Reviewer: Randall Hauch <rhauch@gmail.com>
2019-04-03 15:00:05 -05:00
Guozhang Wang c23bf7a360 HOTFIX: Update unit test for KIP-443 2019-04-03 08:47:51 -07:00
Guozhang Wang 213466b3d4
KAFKA-7190: KIP-443; Remove streams overrides on repartition topics (#6511)
* remove streams overrides on segment.ms and segment.index.bytes

* kip comments
2019-04-02 17:12:32 -07:00
Stanislav Kozlovski 4825bc47a0 KAFKA-8183: Add retries to WorkerUtils#verifyTopics (#6532)
Reviewers: Colin P. McCabe <cmccabe@apache.org>
2019-04-02 17:09:40 -07:00
Victoria Bialas e5a131d9bf KAFKA-8181: Removed Avro topic from TOC on kafka (#6529)
Removed TOC entry in Streams Developer Guide for Avro, since we have no content for this

PR on kafka-site: apache/kafka-site#195

Reviewers: Guozhang Wang <wangguoz@gmail.com>
2019-04-01 16:32:14 -07:00
Viktor Somogyi-Vass e560bae22a KAFKA-8030: Fix flaky tests in TopicCommandWithAdminClientTest
This change adds waits for metadata updates after killing the broker in order to make the tests more stable.

Author: Viktor Somogyi-Vass <viktorsomogyi@gmail.com>

Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>

Closes #6505 from viktorsomogyi/flaky-min-isr-test
2019-04-01 21:47:20 +05:30
Jarrod Urban acb5188810 fix compile error for example (#6526)
Reviewers: Prashant Sabnekar, Bill Bejeck <bbejeck@gmail.com>
2019-04-01 11:39:05 -04:00
Michael Gruben Trejo 8592baf784 MINOR: Comment spelling nit
`CYGINW` probably should be `CYGWIN`

*More detailed description of your change,
if necessary. The PR title and PR message become
the squashed commit message, so use a separate
comment to ping reviewers.*

*Summary of testing strategy (including rationale)
for the feature or bug fix. Unit and/or integration
tests are expected for any behaviour change and
system tests should be considered for larger changes.*

Author: Michael Gruben Trejo <mgrubentrejo@linkedin.com>

Reviewers: Gwen Shapira

Closes #6523 from mgrubent/patch-1
2019-03-29 17:38:26 -07:00
Colin P. Mccabe 219c22113e MINOR: Optimize ConnectionStressWorker
Optimize ConnectionStressWorker by avoiding creating a new
ChannelBuilder each time we want to open a new connection.

Author: Colin P. Mccabe <cmccabe@confluent.io>

Reviewers: Gwen Shapira

Closes #6518 from cmccabe/optimize-connection-stress-worker
2019-03-29 15:02:10 -07:00
Mickael Maison 981815c8d1 KAFKA-8034: Use automatic RPC generation in DeleteTopics
Reviewers: Colin P. McCabe <cmccabe@apache.org>
2019-03-29 14:32:36 -07:00
Bill Bejeck 369d89f208
MINOR: Move KTable source topic for changelog to optimization framework (#6500)
Since we've added Kafka Streams optimizations in 2.1 we need to move the optimization for source KTable nodes (use source topic as changelog) to the optimization framework.

Reviewers: Guozhang Wang <wangguoz@gmail.com>
2019-03-29 17:16:56 -04:00
Lee Dongjin ff78c684ff KAFKA-7502: Cleanup KTable materialization logic in a single place (doMapValues) (#6520)
* Move materialization logic from TableProcessorNode to KTableImpl

1. TableProcessorNode: remove materializedInternal, use storeBuilder instead.
2. Instantiate StoreBuilder in KTableImpl#[doFilter, doMapValues, doTransformValues], instead of TableProcessorNode#init.

* Cleanup KTableImpl#doMapValues

* 1. Add TableProcessorNode(String, ProcessorParameters, StoreBuilder). 2. Reformat+trivial changes on TableProcessorNode.java.
2019-03-29 17:10:04 -04:00
Lee Dongjin d10023e8d3 Cleanup KTableImpl#doTransformValues (#6519)
This PR is a follow-up of #6174 and #6453, which cleans up KTableImpl#doTransformValues method.

Reviewers: Bill Bejeck <bbejeck@gmail.com>
2019-03-29 17:08:20 -04:00
Tcsalist 17f73b42ad MINOR: Streams input topic corrected (#6513)
Though out the tutorial, the name of the input topic that was created is `streams-plaintext-input`. However, this was mistaken at some point in the tutorial and changed to `streams-wordcount-input`.

This patch is to adjust that. Thanks.

Reviewers: Guozhang Wang <wangguoz@gmail.com>
2019-03-28 21:02:41 -07:00
Colin Patrick McCabe b25974c387
MINOR: WorkerUtils#abort: fix bug in abort logic (#6516)
doneFuture is supposed to be completed with an empty string (meaning success) or a non-empty string which is the error message.  Currently, due to exception.getMessage sometimes returning null or an empty string, this is not working correctly.  This patch fixes that.

Reviewers: David Arthur <mumrah@gmail.com>
2019-03-28 14:47:37 -07:00
Lee Dongjin d63d702252 KAFKA-7502: Cleanup KTable materialization logic in a single place (filter) (#6453)
This PR is a follow-up of #6174, which handles doFilter / doMapValues / doTransformValues methods.

Reviewers: Bill Bejeck <bill@confluent.io>, Guozhang Wang <guozhang@confluent.io>
2019-03-28 13:57:04 -07:00
Sönke Liebau b42d904f81 MINOR: Fix some spelling corrections in comments (#6507)
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>
2019-03-28 20:40:58 +05:30
Matthias J. Sax dc0601a1c6
KAFKA-3522: Add RocksDBTimestampedSegmentedBytesStore (#6186)
Reviewers: Bill Bejeck <bill@confluent.io>, John Roesler <john@confluent.io>, Guozhang Wang <guozhang@confluent.io>
2019-03-26 14:23:01 -07:00
Matthias J. Sax f47b8493bc
MINOR: Add 2.2.0 upgrade instructions (#6501)
Reviewers: Jason Gustafson <jason@confluent.io>
2019-03-26 09:41:29 -07:00
Ivan Yurchenko 13e265ab3d KAFKA-7986: Distinguish logging from different ZooKeeperClient instances (#6493)
A broken can have more than one instance of ZooKeeperClient. For example, SimpleAclAuthorizer creates a separate ZooKeeperClient instance when configured.

This commit makes it possible to optionally specify the name for the ZooKeeperClient instance. The name is specified only for a broker's ZooKeeperClient instances, but not for commands' and tests'.

Reviewers: Jun Rao <junrao@gmail.com>
2019-03-25 18:50:12 -07:00
Stanislav Kozlovski 0d55f0f3ec KAFKA-8102: Add an interval-based Trogdor transaction generator (#6444)
This patch adds a TimeIntervalTransactionsGenerator class which enables the Trogdor ProduceBench worker to commit transactions based on a configurable millisecond time interval.

Also, we now handle 409 create task responses in the coordinator command-line client by printing a more informative message

Reviewers: Colin P. McCabe <cmccabe@apache.org>
2019-03-25 09:58:11 -07:00
pierDipi 1baba1b347 MINOR: Fix misspelling in protocol documentation
Reviewers: Colin P. McCabe <cmccabe@apache.org>
2019-03-25 09:54:17 -07:00
Colin Patrick McCabe e0d028bf6c
KAFKA-8150: Fix bugs in handling null arrays in generated RPC code (#6489)
ToString functions must not get a NullPointException.  read() functions
must properly translate a negative array length to a null field.

Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>
2019-03-25 09:43:44 -07:00
Konstantine Karantasis e4cad35312 KAFKA-8014: Extend Connect integration tests to add and remove workers dynamically (#6342)
Extend Connect's integration test framework to add or remove workers to EmbeddedConnectCluster, and choosing whether to fail the test on ungraceful service shutdown. Also added more JavaDoc and other minor improvements. 

Author: Konstantine Karantasis <konstantine@confluent.io>

Reviewers: Arjun Satish <arjun@confluent.io>, Randall Hauch <rhauch@gmail.com>

Closes #6342 from kkonstantine/KAFKA-8014
2019-03-25 09:29:33 -05:00
Bill Bejeck c74acb24eb
MINOR: Remove line for testing repartition topic name (#6488)
With KIP-307 joined.name() is deprecated plus we don't need to test for repartition topic names.
Reviewers: Matthias J. Sax <mjsax@apache.org>
2019-03-24 12:47:58 -04:00
Matthias J. Sax 82615256ff MINOR: add MacOS requirement to Streams docs
*More detailed description of your change,
if necessary. The PR title and PR message become
the squashed commit message, so use a separate
comment to ping reviewers.*

*Summary of testing strategy (including rationale)
for the feature or bug fix. Unit and/or integration
tests are expected for any behaviour change and
system tests should be considered for larger changes.*

Author: Matthias J. Sax <matthias@confluent.io>

Reviewers: Bill Bejeck

Closes #6490 from mjsax/minor-streams-docs-rocksdb
2019-03-23 15:48:24 -07:00
Colin Patrick McCabe f304667df0
MINOR: fix message protocol help text for ElectPreferredLeadersResult (#6479)
Reviewers: Jun Rao <junrao@gmail.com>
2019-03-23 12:14:35 -07:00
Brian Bushree 860e957999 MINOR: list-topics should not require topic param
`kafka.list_topics(...)` should not require a topic parameter

Author: Brian Bushree <bbushree@confluent.io>

Reviewers: Ewen Cheslack-Postava <ewen@confluent.io>

Closes #6367 from brianbushree/list-topics-no-topic
2019-03-22 11:50:00 -07:00
Bill Bejeck 1acae2a67c
MINOR: Clean up ThreadCacheTest (#6485)
Minor clean up ofThreadCacheTest
Reviewers: Guozhang Wang <wangguoz@gmail.com>, Matthias J. Sax <mjsax@apache.org>
2019-03-22 09:27:58 -04:00
Radai Rosenblatt 1deb072f56 MINOR: Avoid unnecessary collection copy in MetadataCache (#6397)
`map` was being used to convert `Iterable[Integer]` to `Iterable[Int`]. That
operation represented 11% of total CPU time measured under load for us.
We also expect a positive impact on GC.

Reviewers: Joel Koshy <jjkoshy@gmail.com>, Ismael Juma <ismael@juma.me.uk>
2019-03-22 01:32:45 -07:00
Matthias J. Sax 1f692bdf53
KAFKA-8142: Fix NPE for nulls in Headers (#6484)
Reviewers: Bill Bejeck <bill@confluent.io>, Guozhang Wang <guozhang@confluent.io>
2019-03-21 23:53:56 -07:00
khairy 3124b07066 KAFKA-7243: Add unit integration tests to validate metrics in Kafka Streams (#6080)
The goal of this task is to implement an integration test for the kafka stream metrics.

We have to check 2 things:
1. After streams application are started, all metrics from different levels (thread, task, processor, store, cache) are correctly created and displaying recorded values.
2. When streams application are shutdown, all metrics are correctly de-registered and removed.

Reviewers: John Roesler <john@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
2019-03-21 17:21:55 -07:00
Matthias J. Sax 4cae4523fc MINOR: Add verification step for Streams archetype to Jenkins build (#6431)
Updates ./jenkins.sh to build stream archetype and install it in local maven cache. Afterward, archetype is used to create a new maven project and maven project is compiled for verification.

Reviewers: Guozhang Wang <wangguoz@gmail.com>, John Roesler <john@confluent.io>,  Bill Bejeck <bbejeck@gmail.com>
2019-03-21 15:51:18 -04:00
Stanislav Kozlovski 6217178139 KAFKA-7819: Improve RoundTripWorker (#6187)
RoundTripWorker to should use a long field for maxMessages rather than an int.  The consumer group used should unique as well.

Reviewers: Colin P. McCabe <cmccabe@apache.org>
2019-03-21 10:03:09 -07:00
Anna Povzner 58d057296a KAFKA-7989: RequestQuotaTest should wait for quota config change before running tests (#6482)
Reviewers: Rajini Sivaram <rajinisivaram@googlemail.com>
2019-03-21 10:15:43 +00:00
huxihx f6f8da7071 KAFKA-8098: Fix Flaky Test testConsumerGroups
- The flaky failure is caused by the fact that the main thread sometimes issues DescribeConsumerGroup request before the consumer assignment takes effect. Added a latch to make sure such situation is not going to happen.

Author: huxihx <huxi_2b@hotmail.com>
Author: huxi <huxi_2b@hotmail.com>
Author: Manikumar Reddy <manikumar.reddy@gmail.com>

Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>

Closes #6441 from huxihx/KAFKA-8098
2019-03-20 10:47:34 +05:30
Florian Hussonnois fa57eb065d KAFKA-6958: Add new NamedOperation interface to enforce consistency in naming operations (#6409)
Sub-task required to allow to define custom processor names with KStreams DSL(KIP-307) :
  - add new public interface NamedOperation
  - deprecate methods Joined.as() and Joined.name()
  - update Suppredded interface to extend NamedOperation

Reviewers: Matthias J. Sax <mjsax@apache.org>,  John Roesler <john@confluent.io>, Bill Bejeck <bbejeck@gmail.com>
2019-03-19 22:27:03 -04:00
Matthias J. Sax b5ce093a24
MINOR: capture result timestamps in Kafka Streams DSL tests (#6447)
Reviewers: Bill Bejeck <bill@confluent.io>, Guozhang Wang <guozhang@confluent.io>
2019-03-19 17:27:32 -07:00
Victoria Bialas 70ee72491f MINOR: updated names for deprecated streams constants (#6466)
* updated names for deprecated streams constants
* add DEFAULT_TIMESTAMP_EXTRACTOR_CLASS_CONFIG in place of deprecated

Reviewers: Jim Galasyn <jim.galasyn@confluent.io>, Matthias J. Sax <matthias@confluent.io>
2019-03-19 16:33:41 -07:00
Victoria Bialas 5092b26393 MINOR: update docs JSON serde links (#6465)
Reviewers: Joel Mamill <joel@confluent.io>, Matthias J. Sax <mjsax@apache.org>
2019-03-19 18:01:59 -04:00
A. Sophie Blee-Goldman 538bd7eddf KAFKA-8094: Iterating over cache with get(key) is inefficient (#6433)
Use concurrent data structure for the underlying cache in NamedCache, and iterate over it with subMap instead of many calls to get()

Reviewers: Guozhang Wang <wangguoz@gmail.com>, Bill Bejeck <bbejeck@gmail.com>
2019-03-19 11:51:10 -04:00
Guozhang Wang 6d649f503a KAFKA-8062: Do not remore StateListener when shutting down stream thread (#6468)
In a previous commit #6091, we've fixed a couple of edge cases and hence do not need to remove state listener anymore (before that we removed the state listener intentionally to avoid some race conditions, which has been gone for now).

Reviewers: Matthias J. Sax <mjsax@apache.org>,   Bill Bejeck <bbejeck@gmail.com>
2019-03-19 10:12:49 -04:00
Bob Barrett 70ddd8af71 MINOR: Improve logging around index files (#6385)
This patch adds additional DEBUG statements in AbstractIndex.scala, OffsetIndex.scala, and TimeIndex.scala. It also changes the logging on append from DEBUG to TRACE to make DEBUG logging less disruptive, and it ensures that exceptions raised from index classes include file/offset information.

Reviewers: Jason Gustafson <jason@confluent.io>
2019-03-18 23:00:01 -07:00
Boyang Chen 8406f3624d KAFKA-7858: Automatically generate JoinGroup request/response
Reviewers: Colin P. McCabe <cmccabe@apache.org>
2019-03-18 13:26:09 -07:00
Rajini Sivaram 810bc69b7a
KAFKA-8121; Shutdown ZK client expiry handler earlier during close (#6462)
Shutdown session expiry thread prior to closing ZooKeeper client to ensure that new clients are not created by the expiry thread and left active when returning from ZooKeeperClient.close().

Reviewers: Ismael Juma <ismael@juma.me.uk>
2019-03-18 18:07:24 +00:00
Anna Povzner 5192956581 MINOR: Improve verification in flaky testPartitionReassignmentDuringDeleteTopic (#6460)
Reviewers: Rajini Sivaram <rajinisivaram@googlemail.com>
2019-03-18 16:48:49 +00:00
Rajini Sivaram ca6ac9393b
MINOR: Retain public constructors of classes from public API (#6455)
TopicDescription and ConsumerGroupDescription in org.apache.kafka.clients.admin. are part of the public API, so we should retain the existing public constructor. Changed the new constructor with authorized operations to be package-private to avoid maintaining more public constructors since we only expect admin client to use this.

Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>
2019-03-18 08:51:50 +00:00