Commit Graph

261 Commits

Author SHA1 Message Date
John Roesler cc43e77bbb MINOR: make Sensor#add idempotent (#4853)
This change makes adding a metric to a sensor idempotent.
That is, if the metric is already added to the sensor, the method
returns with success.

The current behavior is that any attempt to register a second metric
with the same name is an error.

Testing strategy: There is a new unit test covering this behavior

Reviewers: Guozhang Wang <guozhang@confluent.io>, Bill Bejeck <bill@confluent.io>
2018-04-11 20:50:10 -07:00
Benedict Jin 37efc79eb8 MINOR: Remove magic number and extract Pattern instance from method as class field (#4799)
* Remove magic number
* Extract Pattern instance from method as class field
* Add @Override declare

Reviewers: Randall Hauch <rhauch@gmail.com>
2018-04-08 11:54:22 -07:00
Randall Hauch d9369de8f2 KAFKA-6728: Corrected the worker’s instantiation of the HeaderConverter
## Summary of the problem
When the `header.converter` is not specified in the worker config or the connector config, a bug in the `Plugins` test causes it to never instantiate the `HeaderConverter` instance, even though there is a default value.

This is a problem as soon as the connector deals with headers, either in records created by a source connector or in messages on the Kafka topics consumed by a sink connector. As soon as that happens, a NPE occurs.

A workaround is to explicitly set the `header.converter` configuration property, but this was added in AK 1.1 and thus means that upgrading to AK 1.1 will not be backward compatible and will require this configuration change.

## The Changes

The `Plugins.newHeaderConverter` methods were always returning null if the `header.converter` configuration value was not specified in the supplied connector or worker configuration. Thus, even though the `header.converter` property has a default, it was never being used.

The fix was to only check whether a `header.converter` property was specified when the connector configuration was being used, and if no such property exists in the connector configuration to return null. Then, when the worker configuration is being used, the method simply gets the `header.converter` value (or the default if no value was explicitly set).

Also, the ConnectorConfig had the same default value for the `header.converter` property as the WorkerConfig, but this resulted in very confusing log messages that implied the default header converter should be used even when the worker config specified the `header.converter` value. By removing the default, the log messages now make sense, and the Worker still properly instantiates the correct header converter.

Finally, updated comments and added log messages to make it more clear which converters are being used and how they are being converted.

## Testing

Several new unit tests for `Plugins.newHeaderConverter` were added to check the various behavior. Additionally, the runtime JAR with these changes was built and inserted into an AK 1.1 installation, and a source connector was manually tested with 8 different combinations of settings for the `header.converter` configuration:

1. default value
1. worker configuration has `header.converter` explicitly set to the default
1. worker configuration has `header.converter` set to a custom `HeaderConverter` implementation in the same plugin
1. worker configuration has `header.converter` set to a custom `HeaderConverter` implementation in a _different_ plugin
1. connector configuration has `header.converter` explicitly set to the default
1. connector configuration has `header.converter` set to a custom `HeaderConverter` implementation in the same plugin
1. connector configuration has `header.converter` set to a custom `HeaderConverter` implementation in a _different_ plugin
1. worker configuration has `header.converter` explicitly set to the default, and the connector configuration has `header.converter` set to a custom `HeaderConverter` implementation in a _different_ plugin

The worker created the correct `HeaderConverter` implementation with the correct configuration in all of these tests.

Finally, the default configuration was used with the aforementioned custom source connector that generated records with headers, and an S3 connector that consumes the records with headers (but didn't do anything with them). This test also passed.

Author: Randall Hauch <rhauch@gmail.com>

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

Closes #4815 from rhauch/kafka-6728
2018-04-03 08:48:05 -07:00
Attila Sasvari 549a5cec1e MINOR: Fix potential resource leak in FileOffsetBackingStore (#4739)
Reviewers: Sandor Murakozi <smurakozi@gmail.com>, Jason Gustafson <jason@confluent.io>
2018-03-24 12:20:11 -07:00
Randall Hauch e7ef719a5b KAFKA-6661: Ensure sink connectors don’t resume consumer when task is paused
Changed WorkerSinkTaskContext to only resume the consumer topic partitions when the connector/task is not in the paused state.

The context tracks the set of topic partitions that are explicitly paused/resumed by the connector, and when the WorkerSinkTask resumes the tasks it currently resumes all topic partitions *except* those that are still explicitly paused in the context. Therefore, the change above should result in the desired behavior.

Several debug statements were added to record when the context is called by the connector.

This can be backported to older releases, since this bug goes back to 0.10 or 0.9.

Author: Randall Hauch <rhauch@gmail.com>

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

Closes #4716 from rhauch/kafka-6661
2018-03-15 15:52:53 -07:00
Randall Hauch 0e22fd6f8d KAFKA-6578: Changed the Connect distributed and standalone main method to log all exceptions (#4609)
Any exception thrown by calls within a `main()` method are not logged unless explicitly done so. This change simply adds a try-catch block around most of the content of the distributed and standalone `main()` methods.
2018-02-22 22:29:49 -08:00
Randall Hauch fc19c3e6f2 KAFKA-6577: Fix Connect system tests and add debug messages
**NOTE: This should be backported to the `1.1` branch, and is currently a blocker for 1.1.**

The `connect_test.py::ConnectStandaloneFileTest.test_file_source_and_sink` system test is failing with the SASL configuration without a sufficient explanation. During the test, the Connect worker fails to start, but the Connect log contains no useful information. There are actual several things compounding to cause the failure and make it difficult to understand the problem.

First, the `tests/kafkatest/tests/connect/templates/connect_standalone.properties` is only adding in the broker's security configuration with the `producer.` and `consumer.` prefixes, but is not adding them with no prefix. The worker uses the AdminClient to connect to the broker to get the Kafka cluster ID and to manage the three internal topics, and the AdminClient is configured via top-level properties. Because the SASL test requires the clients all connect using SASL, the lack of broker security configs means the AdminClient was attempting and failing to connect to the broker. This is corrected by adding the broker's security configuration to the Connect worker configuration file at the top-level. (This was already being done in the `connect_distributed.properties` file.)

Second, the default `request.timeout.ms` for the AdminClient (and the other clients) is 120 seconds, so the AdminClient was retrying for 120 seconds before it would give up and thrown an error. However, the test was only waiting for 60 seconds before determining that the service failed to start. This can be corrected by setting `request.timeout.ms=10000` in the Connect distributed and standalone worker configurations.

Third, the Connect workers were recently changed to lookup the Kafka cluster ID before it started the herder. This is unlike the older uses of the AdminClient to find and manage the internal topics, where failure to connect was not necessarily logged correctly but nevertheless still skipped over, relying upon broker auto-topic creation to create the internal topics. (This may be why the test did not fail prior to the recent change to always require a successful AdminClient connection.) Although the worker never got this far in its startup process, the fact that we missed such an error since the prior releases means that failure to connect with the AdminClient was not being properly reported.

The `ConnectStandaloneFileTest.test_file_source_and_sink` system tests were run locally prior to this fix, and they failed as with the nightlies. Once these fixes were made, the locally run system tests passed.

Author: Randall Hauch <rhauch@gmail.com>

Reviewers: Konstantine Karantasis <konstantine@confluent.io>, Ewen Cheslack-Postava <me@ewencp.org>

Closes #4610 from rhauch/kafka-6577-trunk
2018-02-22 09:39:59 +00:00
Konstantine Karantasis f3c4240e6c MINOR: Restore scanning for super types of Connect plugins (#4584)
Enabling scans for super types in reflections is required in order to discover Connect plugins.

Reviewers: Randall Hauch <rhauch@gmail.com>, Jason Gustafson <jason@confluent.io>
2018-02-20 13:39:45 -08:00
Konstantine Karantasis b79e11bb51 MINOR: Redirect response code in Connect's RestClient to logs instead of stdout
Sending the response code of an http request issued via `RestClient` in Connect to stdout seems like a unconventional choice.

This PR redirects the responds code with a message in the logs at DEBUG level (usually the same level as the one that the caller of `RestClient.httpRequest` uses.

This fix will also fix system tests that broke by outputting this response code to stdout.

Author: Konstantine Karantasis <konstantine@confluent.io>

Reviewers: Randall Hauch <rhauch@gmail.com>, Damian Guy <damian.guy@gmail.com>

Closes #4591 from kkonstantine/MINOR-Redirect-response-code-in-Connect-RestClient-to-logs-instead-of-stdout
2018-02-20 17:15:31 +00:00
Konstantine Karantasis ee352be9c8 MINOR: Fix bug introduced by adding batch.size without default in FileStreamSourceConnector (#4579)
https://github.com/apache/kafka/pull/4356 added `batch.size` config property to `FileStreamSourceConnector` but the property was added as required without a default in config definition (`ConfigDef`). This results in validation error during connector startup. 

Unit tests were added for both `FileStreamSourceConnector` and `FileStreamSinkConnector` to avoid such issues in the future.

Reviewers: Randall Hauch <rhauch@gmail.com>, Jason Gustafson <jason@confluent.io>
2018-02-16 18:30:12 -08:00
Robert Yokota 3af13967db KAFKA-6503: Parallelize plugin scanning
This is a small change to parallelize plugin scanning.  This may help in some environments where otherwise plugin scanning is slow.

Author: Robert Yokota <rayokota@gmail.com>

Reviewers: Konstantine Karantasis <konstantine@confluent.io>, Randall Hauch <rhauch@gmail.com>, Ewen Cheslack-Postava <ewen@confluent.io>

Closes #4561 from rayokota/K6503-improve-plugin-scanning
2018-02-14 16:24:05 -08:00
ConcurrencyPractitioner d902b6b4b6 KAFKA-5996; JsonConverter generates Mismatching schema DataException (#4523)
JsonConverter should use object equality rather than reference equality in `convertToJson`.

Reviewers: Bartlomiej Tartanus <bartektartanus@gmail.com>, Randall Hauch <rhauch@gmail.com>, Jason Gustafson <jason@confluent.io>
2018-02-14 08:46:22 -08:00
Benedict Jin 2693e9be74 MINOR: Misc improvements on runtime / storage / metrics / config parts (#4525)
Reviewers: Guozhang Wang <wangguoz@gmail.com>
2018-02-13 16:39:21 -08:00
Randall Hauch a68a8b6b6c KAFKA-6511; Corrected connect list/map parsing logic (#4516)
Corrected the parsing of invalid list values. A list can only be parsed if it contains elements that have a common type, and a map can only be parsed if it contains keys with a common type and values with a common type.

Reviewers: Arjun Satish <arjun@confluent.io>, Magesh Nandakumar <magesh.n.kumar@gmail.com>, Konstantine Karantasis <konstantine@confluent.io>, Jason Gustafson <jason@confluent.io>
2018-02-13 11:44:12 -08:00
Jeremy Custenborder 9f4fe7679e KAFKA-5550; Connect Struct.put() should include the field name if validation fails (#3507)
Changed call to use the overload of ConnectSchema.validateValue() method with the field name passed in. Ensure that field in put call is not null.

Reviewers: Randall Hauch <rhauch@gmail.com>, Jason Gustafson <jason@confluent.io>
2018-02-12 09:16:25 -08:00
Randall Hauch 976a3b0cc8 KAFKA-6513: Corrected how Converters and HeaderConverters are instantiated and configured
The commits for KIP-145 (KAFKA-5142) changed how the Connect workers instantiate and configure the Converters, and also added the ability to do the same for the new HeaderConverters. However, the last few commits removed the default value for the `converter.type` property for Converters and HeaderConverters, and this broke how the internal converters were being created.

This change corrects the behavior so that the `converter.type` property is always set by the worker (or by the Plugins class), which means the existing Converter implementations will not have to do this. The built-in JsonConverter, ByteArrayConverter, and StringConverter also implement HeaderConverter which implements Configurable, but the Worker and Plugins methods do not yet use the `Configurable.configure(Map)` method and instead still use the `Converter.configure(Map,boolean)`.

Several tests were modified, and a new PluginsTest was added to verify the new behavior in Plugins for instantiating and configuring the Converter and HeaderConverter instances.

Author: Randall Hauch <rhauch@gmail.com>

Reviewers: Konstantine Karantasis <konstantine@confluent.io>, Ewen Cheslack-Postava <ewen@confluent.io>

Closes #4512 from rhauch/kafka-6513
2018-02-09 15:47:44 -08:00
Wladimir Schmidt c32860b224 MINOR: exchange redundant Collections.addAll with parameterized constructor (#4521)
* Exchange manual copy to collection with Collections.addAll call
* Exchange redundant Collections.addAll with parameterized constructor call

Reviewers: Guozhang Wang <wangguoz@gmail.com>
2018-02-06 11:33:49 -08:00
Robert Yokota 7d70a427b2 KAFKA-6504: Ensure uniqueness of connect task metric sensor creation (#4514)
This change ensures that when sensors are created, they are unique to the metric group associated with the task that created them. Previously the sensors were being shared between task metric groups, causing incorrect metrics.

Reviewers: Randall Hauch <rhauch@gmail.com>, Jason Gustafson <jason@confluent.io>
2018-02-06 09:18:37 -08:00
Jeff Klukas eb3fef760e KAFKA-6253: Improve sink connector topic regex validation
KAFKA-3073 added topic regex support for sink connectors. The addition requires that you only specify one of topics or topics.regex settings. This is being validated in one place, but not during submission of connectors. This PR adds validation at `AbstractHerder.validateConnectorConfig` and `WorkerConnector.initialize`.

This adds a test of the new behavior to `AbstractHerderTest`.

Author: Jeff Klukas <jeff@klukas.net>

Reviewers: Randall Hauch <rhauch@gmail.com>, Ewen Cheslack-Postava <ewen@confluent.io>

Closes #4251 from jklukas/connect-topics-validation
2018-02-05 09:46:07 -08:00
Konstantine Karantasis 17aaff3606 KAFKA-6288: Broken symlink interrupts scanning of the plugin path
Submitting a fail safe fix for rare IOExceptions on symbolic links.

The fix is submitted without a test case since it does seem easy to reproduce such type of failures (just having a broken symbolic link does not reproduce the issue) and it's considered pretty low risk.

If accepted, needs to be ported at least to 1.0, if not 0.11

Author: Konstantine Karantasis <konstantine@confluent.io>

Reviewers: Randall Hauch <rhauch@gmail.com>, Ewen Cheslack-Postava <ewen@confluent.io>

Closes #4481 from kkonstantine/KAFKA-6288-Broken-symlink-interrupts-scanning-the-plugin-path
2018-02-04 14:57:25 -08:00
Gunnar Morling f9b56d680b KAFKA-6515; Adding toString() method to o.a.k.connect.data.Field (#4509) 2018-02-03 13:27:02 -08:00
Randall Hauch 4c48942f9d KAFKA-5142: Add Connect support for message headers (KIP-145)
**[KIP-145](https://cwiki.apache.org/confluence/display/KAFKA/KIP-145+-+Expose+Record+Headers+in+Kafka+Connect) has been accepted, and this PR implements KIP-145 except without the SMTs.**

Changed the Connect API and runtime to support message headers as described in [KIP-145](https://cwiki.apache.org/confluence/display/KAFKA/KIP-145+-+Expose+Record+Headers+in+Kafka+Connect).

The new `Header` interface defines an immutable representation of a Kafka header (key-value pair) with support for the Connect value types and schemas. This interface provides methods for easily converting between many of the built-in primitive, structured, and logical data types.

The new `Headers` interface defines an ordered collection of headers and is used to track all headers associated with a `ConnectRecord` (and thus `SourceRecord` and `SinkRecord`). This does allow multiple headers with the same key. The `Headers` contains methods for adding, removing, finding, and modifying headers. Convenience methods allow connectors and transforms to easily use and modify the headers for a record.

A new `HeaderConverter` interface is also defined to enable the Connect runtime framework to be able to serialize and deserialize headers between the in-memory representation and Kafka’s byte[] representation. A new `SimpleHeaderConverter` implementation has been added, and this serializes to strings and deserializes by inferring the schemas (`Struct` header values are serialized without the schemas, so they can only be deserialized as `Map` instances without a schema.) The `StringConverter`, `JsonConverter`, and `ByteArrayConverter` have all been extended to also be `HeaderConverter` implementations. Each connector can be configured with a different header converter, although by default the `SimpleHeaderConverter` is used to serialize header values as strings without schemas.

Unit and integration tests are added for `ConnectHeader` and `ConnectHeaders`, the two implementation classes for headers. Additional test methods are added for the methods added to the `Converter` implementations. Finally, the `ConnectRecord` object is already used heavily, so only limited tests need to be added while quite a few of the existing tests already cover the changes.

Author: Randall Hauch <rhauch@gmail.com>

Reviewers: Arjun Satish <arjun@confluent.io>, Ted Yu <yuzhihong@gmail.com>, Magesh Nandakumar <magesh.n.kumar@gmail.com>, Konstantine Karantasis <konstantine@confluent.io>, Ewen Cheslack-Postava <ewen@confluent.io>

Closes #4319 from rhauch/kafka-5142-b
2018-01-31 10:40:24 -08:00
Soenke Liebau 530bc59de2 KAFKA-4930: Enforce set of legal characters for connector names (KIP-212)
…to check for empty connector name and illegal characters in connector name. This also fixes  KAFKA-4938 by removing the check for slashes in connector name from ConnectorsResource.

Author: Ewen Cheslack-Postava <me@ewencp.org>
Author: Soenke Liebau <soenke.liebau@opencore.com>

Reviewers: Gwen Shapira <cshapi@gmail.com>, Viktor Somogyi <viktor.somogyi@cloudera.com>, Randall Hauch <rhauch@gmail.com>, Ewen Cheslack-Postava <ewen@confluent.io>

Closes #2755 from soenkeliebau/KAFKA-4930
2018-01-31 08:49:23 -08:00
Jakub Scholz 3431be2aeb KAFKA-4029: SSL support for Connect REST API (KIP-208)
This PR implements the JIRA issue [KAFKA-4029: SSL support for Connect REST API](https://issues.apache.org/jira/browse/KAFKA-4029) / [KIP-208](https://cwiki.apache.org/confluence/display/KAFKA/KIP-208%3A+Add+SSL+support+to+Kafka+Connect+REST+interface).

Summary of the main changes:
- Jetty `HttpClient` is used as HTTP client instead of the one shipped with Java. That allows to keep the SSL configuration for Server and Client be in single place (both use the Jetty `SslContextFactory`). It also has much richer configuration than the JDK client (it is easier to configure things such as supported cipher suites etc.).
- The `RestServer` class has been broker into 3 parts. `RestServer` contains the server it self. `RestClient` contains the HTTP client used for forwarding requests etc. and `SSLUtils` contain some helper classes for configuring SSL. One of the reasons for this was Findbugs complaining about the class complexity.
- A new method `valuesWithPrefixAllOrNothing` has been added to `AbstractConfig` to make it easier to handle the situation that we want to use either only the prefixed SSL options or only the non-prefixed. But not mixed them.

Author: Jakub Scholz <www@scholzj.com>

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

Closes #4429 from scholzj/kip-208
2018-01-30 15:09:40 -08:00
Konstantine Karantasis b4165522b3 KAFKA-6148; ClassCastException in connectors that include kafka-clients packages (#4457)
Exclusion for packages that need not be loaded in isolation needs to be extended to all the `org.apache.kafka` packages (that do not belong to transforms and the other whitelisted packages). Most notably, this refers to any classes in `kafka-clients` package. 

Reviewers: Randall Hauch <rhauch@gmail.com>, Jason Gustafson <jason@confluent.io>
2018-01-30 08:45:07 -08:00
Scott d706c87cb2 MINOR: a single doc typo (#2969) 2018-01-26 11:14:57 -08:00
Konstantine Karantasis 993d3c727e KAFKA-6467; Enforce layout of dependencies within a connect plugin to be deterministic (#4459)
Adds alphanumeric ordering of dependencies as they added to a Connect plugin's class loader path. 

This makes the layout of the dependencies consistent across systems and deployments. Dependencies should still, in principle, not include conflicts and ideally order should not matter. 

Reviewers: Randall Hauch <rhauch@gmail.com>, Jason Gustafson <jason@confluent.io>
2018-01-26 08:39:57 -08:00
Charly Molter aa42a11dfd KAFKA-6180; Add a Validator for NonNull configurations and remove redundant null checks on lists (#4188) 2018-01-24 21:06:44 -08:00
Konstantine Karantasis 0fa52644de KAFKA-6277: Ensure loadClass for plugin class loaders is thread-safe.
`loadClass` needs to be synchronized to protect subsequent calls to `defineClass`.

Details in the javadoc of this PR as well as here too: https://docs.oracle.com/javase/7/docs/technotes/guides/lang/cl-mt.html

/cc ewencp rhauch

Author: Konstantine Karantasis <konstantine@confluent.io>

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

Closes #4428 from kkonstantine/KAFKA-6277-Make-loadClass-thread-safe-for-class-loaders-of-Connect-plugins
2018-01-21 19:41:20 -08:00
Gunnar Morling 8cd563d478 KAFKA-6456; JavaDoc clarification for Connect SourceTask#poll() (#4432)
Making clear that implementations of poll() shouldn't block indefinitely in order to allow the task instance to transition to PAUSED state.

Reviewers:  Randall Hauch <rhauch@gmail.com>, Jason Gustafson <jason@confluent.io>
2018-01-18 09:56:22 -08:00
Gavrie Philipson 936e81afcb KAFKA-6250: Use existing Kafka Connect internal topics without requiring ACL (#4247)
When using Kafka Connect with a cluster that doesn't allow the user to
create topics (due to ACL configuration), Connect fails when trying to
create its internal topics even if these topics already exist. This is
incorrect behavior according to the documentation, which mentions that
R/W access should be enough.

This happens specifically when using Aiven Kafka, which does not permit
creation of topics via the Kafka Admin Client API.

The patch ignores the returned error, similar to the behavior for older
brokers that don't support the API.
2018-01-11 15:52:50 -08:00
Filipe Agapito 697a4af35a KAFKA-6363: Use MockAdminClient for any unit tests that depend on AdminClient (#4371)
* Implement MockAdminClient.deleteTopics
* Use MockAdminClient instead of MockKafkaAdminClientEnv in StreamsResetterTest
* Rename MockKafkaAdminClientEnv to AdminClientUnitTestEnv
* Use MockAdminClient instead of MockKafkaAdminClientEnv in TopicAdminTest
* Rename KafkaAdminClient to AdminClientUnitTestEnv in KafkaAdminClientTest.java
* Migrate StreamThreadTest to MockAdminClient
* Fix style errors
* Address review comments
* Fix MockAdminClient call

Reviewers: Matthias J. Sax <matthias@confluent.io>, Konstantine Karantasis <konstantine@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
2018-01-08 11:58:56 -08:00
Arjun Satish 3b03c9d348 KAFKA-6252; Close the metric group to clean up any existing metrics
We are closing the metricGroups created in a Worker, Source task and Sink task before populating them with new metrics. This helps in cases where an Exception is thrown when previously created groups were not cleaned up correctly.

Signed-off-by: Arjun Satish <arjunconfluent.io>

Author: Arjun Satish <arjun@confluent.io>

Reviewers: Randall Hauch <rhauch@gmail.com>, Konstantine Karantasis <konstantine@confluent.io>, Ewen Cheslack-Postava <ewen@confluent.io>

Closes #4397 from wicknicks/KAFKA-6252
2018-01-05 21:24:39 -08:00
Study ecf0ab42c1 KAFKA-4335: Add batch.size to FileStreamSource connector to prevent OOM
When the source file of `FileStreamSource` is a large file, `FileStreamSourceTask.poll()` will result in OOM. This pull request added `batch.size` parameter which can restrict the poll size.

*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: Study <ph.study@gmail.com>

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

Closes #4356 from phstudy/KAFKA-4335
2018-01-05 18:31:53 -08:00
Ewen Cheslack-Postava bfb272c5cd KAFKA-6311: Expose Kafka cluster ID in Connect REST API (KIP-238) (#4314)
Reviewers: Konstantine Karantasis <konstantine@confluent.io>, Randall Hauch <rhauch@gmail.com>, Jason Gustafson <jason@confluent.io>
2018-01-05 07:52:50 -08:00
Koen De Groote 96df93522f Replace Arrays.asList with Collections.singletonList where possible (#4368)
Reviewers: Ismael Juma <ismael@juma.me.uk>
2018-01-01 13:07:32 +00:00
Koen De Groote 2bc780b959 MINOR: Use EnumMap/EnumSet if possible (#3919)
They are more efficient than HashMap/HashSet.

Reviewers: Ismael Juma <ismael@juma.me.uk>
2017-12-29 12:39:18 +00:00
Tobias Gies 68712dcdec KAFKA-6308; Connect Struct should use deepEquals/deepHashCode
This changes the Struct's equals and hashCode method to use Arrays#deepEquals and Arrays#deepHashCode, respectively. This resolves a problem where two structs with values of type byte[] would not be considered equal even though the byte arrays' contents are equal. By using deepEquals, the byte arrays' contents are compared instead of ther identity.

Since this changes the behavior of the equals method for byte array values, the behavior of hashCode must change alongside it to ensure the methods still fulfill the general contract of "equal objects must have equal hashCodes".

Test rationale:
All existing unit tests for equals were untouched and continue to work. A new test method was added to verify the behavior of equals and hashCode for Struct instances that contain a byte array value. I verify the reflixivity and transitivity of equals as well as the fact that equal Structs have equal hashCodes
and not-equal structs do not have equal hashCodes.

Author: Tobias Gies <tobias.gies@trivago.com>
Author: Tobias Gies <tobias@tobiasgies.de>

Reviewers: Randall Hauch <rhauch@gmail.com>, Jason Gustafson <jason@confluent.io>

Closes #4293 from tobiasgies/feature/kafka-6308-deepequals
2017-12-14 15:26:20 -08:00
Colin P. Mccabe 616321bcb6 KAFKA-6102; Consolidate MockTime implementations between connect and clients
Author: Colin P. Mccabe <cmccabe@confluent.io>

Reviewers: Randall Hauch <rhauch@gmail.com>, Jason Gustafson <jason@confluent.io>

Closes #4105 from cmccabe/KAFKA-6102
2017-12-14 14:50:34 -08:00
Soenke Liebau 5a2960f811 KAFKA-5563: Standardize validation and substitution of connector names in REST API connector configs
…from config to own function and added check to create connector call.

Author: Soenke Liebau <soenke.liebau@opencore.com>

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

Closes #4230 from soenkeliebau/KAFKA-5563
2017-11-25 17:50:17 -08:00
Arjun Satish c5f31fe384 KAFKA-4827: Correctly encode special chars while creating URI objects
Signed-off-by: Arjun Satish <arjunconfluent.io>

Author: Arjun Satish <arjun@confluent.io>

Reviewers: Randall Hauch <rhauch@gmail.com>, Ewen Cheslack-Postava <ewen@confluent.io>

Closes #4205 from wicknicks/KAFKA-4827
2017-11-22 09:54:06 -08:00
Jeff Klukas 049342e440 KAFKA-3073: Add topic regex support for Connect sinks
There are more methods that had to be touched than I anticipated when writing [the KIP](https://cwiki.apache.org/confluence/display/KAFKA/KIP-215%3A+Add+topic+regex+support+for+Connect+sinks).

The implementation here is now complete and includes a test that verifies that there's a call to `consumer.subscribe(Pattern, RebalanceHandler)` when `topics.regex` is provided.

Author: Jeff Klukas <jeff@klukas.net>

Reviewers: Randall Hauch <rhauch@gmail.com>, Ewen Cheslack-Postava <ewen@confluent.io>

Closes #4151 from jklukas/connect-topics.regex
2017-11-21 16:01:16 -08:00
tedyu fd8eb268d6 KAFKA-6168: Connect Schema comparison is slow for large schemas
Re-arrange order of comparisons in equals() to evaluate non-composite fields first
Cache hash code

Author: tedyu <yuzhihong@gmail.com>

Reviewers: Randall Hauch <rhauch@gmail.com>, Konstantine Karantasis <konstantine@confluent.io>, Ewen Cheslack-Postava <ewen@confluent.io>

Closes #4176 from tedyu/trunk
2017-11-21 14:26:32 -08:00
Ewen Cheslack-Postava f0276f5ca2 MINOR: Log unexpected exceptions in Connect REST calls that generate 500s at a higher log level
The ConnectExceptionMapper was originally intended to handle ConnectException errors for some expected cases where we just want to always convert them to a certain response and the ExceptionMapper was the easiest way to do that uniformly across the API. However, in the case that it's not an expected subclass, we should log the information at the error level so the user can track down the cause of the error.

This is only an initial improvement. We should probably also add a more general ExceptionMapper to handle other exceptions we may not have caught and converted to ConnectException.

Author: Ewen Cheslack-Postava <me@ewencp.org>

Reviewers: Randall Hauch <rhauch@gmail.com>, Jason Gustafson <jason@confluent.io>

Closes #4227 from ewencp/better-connect-error-logging
2017-11-20 20:29:33 -08:00
sachinbhalekar 0026121543 KAFKA-6218: Optimize condition in if statement to reduce the number of comparisons
Changed the condition in **if** statement
**(schema.name() == null || !(schema.name().equals(LOGICAL_NAME)))** which
requires two comparisons in worst case with
**(!LOGICAL_NAME.equals(schema.name()))**  which requires single comparison
in all cases and _avoids null pointer exception.
![kafka_optimize_if](https://user-images.githubusercontent.com/32234013/32872271-afe0b954-ca3a-11e7-838d-6a3bc416b807.JPG)
_

Author: sachinbhalekar <sachinbansibhalekar@gmail.com>
Author: sachinbhalekar <32234013+sachinbhalekar@users.noreply.github.com>

Reviewers: Randall Hauch <rhauch@gmail.com>, Ewen Cheslack-Postava <ewen@confluent.io>

Closes #4225 from sachinbhalekar/trunk
2017-11-16 15:04:27 -08:00
Adem Efe Gencer 86062e9a78 KAFKA-6157; Fix repeated words words in JavaDoc and comments.
Author: Adem Efe Gencer <agencer@linkedin.com>

Reviewers: Jiangjie Qin <becket.qin@gmail.com>

Closes #4170 from efeg/bug/typoFix
2017-11-05 18:00:43 -08:00
Richard Yu 7fe88e8bd9 KAFKA-5212; Consumer ListOffsets request can starve group heartbeats
Author: Richard Yu <richardyu@Richards-Air.attlocal.net>

Reviewers: Jason Gustafson <jason@confluent.io>

Closes #4110 from ConcurrencyPractitioner/trunk
2017-10-30 11:31:36 -07:00
Konstantine Karantasis 5ec6765bdb KAFKA-6087: Scanning plugin.path needs to support relative symlinks.
Author: Konstantine Karantasis <konstantine@confluent.io>

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

Closes #4092 from kkonstantine/KAFKA-6087-Scanning-plugin.path-needs-to-support-relative-symlinks
2017-10-19 14:24:57 -07:00
Ewen Cheslack-Postava 7d6ca52a27 MINOR: Push JMX metric name mangling into the JmxReporter (KIP-190 follow up)
Author: Ewen Cheslack-Postava <me@ewencp.org>

Reviewers: Rajini Sivaram <rajinisivaram@googlemail.com>, Ismael Juma <ismael@juma.me.uk>

Closes #3980 from ewencp/dont-mangle-names
2017-10-11 17:32:40 -04:00
Konstantine Karantasis 974d6fec93 KAFKA-5953: Register all jdbc drivers available in plugin and class paths
Author: Konstantine Karantasis <konstantine@confluent.io>

Reviewers: Randall Hauch <rhauch@gmail.com>, Ewen Cheslack-Postava <ewen@confluent.io>

Closes #4030 from kkonstantine/KAFKA-5953-Connect-classloader-isolation-may-be-broken-for-JDBC-drivers
2017-10-06 11:42:04 -07:00