-
Notifications
You must be signed in to change notification settings - Fork 2.5k
[HUDI-1996] Adding functionality to allow the providing of basic auth creds for confluent cloud schema registry #3097
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from 9 commits
b0bd45f
42b1949
4e29d4f
7f9268b
6865391
3e3d2c2
5cfa5f1
75d4a71
10c79ac
8ff2467
3d16c8d
488e268
ca83cba
10ded84
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -28,8 +28,14 @@ | |
| import org.apache.spark.api.java.JavaSparkContext; | ||
|
|
||
| import java.io.IOException; | ||
| import java.io.InputStream; | ||
| import java.net.HttpURLConnection; | ||
| import java.net.URL; | ||
| import java.nio.charset.StandardCharsets; | ||
| import java.util.Base64; | ||
| import java.util.Collections; | ||
| import java.util.regex.Matcher; | ||
| import java.util.regex.Pattern; | ||
|
|
||
| /** | ||
| * Obtains latest schema from the Confluent/Kafka schema-registry. | ||
|
|
@@ -48,19 +54,40 @@ public static class Config { | |
| "hoodie.deltastreamer.schemaprovider.registry.targetUrl"; | ||
| } | ||
|
|
||
| private static String fetchSchemaFromRegistry(String registryUrl) throws IOException { | ||
| URL registry = new URL(registryUrl); | ||
| public String fetchSchemaFromRegistry(String registryUrl) throws IOException { | ||
| URL registry; | ||
| HttpURLConnection connection; | ||
| Matcher matcher = Pattern.compile("://(.*?)@").matcher(registryUrl); | ||
| if (matcher.find()) { | ||
| String creds = matcher.group(1); | ||
| String urlWithoutCreds = registryUrl.replace(creds + "@", ""); | ||
| registry = new URL(urlWithoutCreds); | ||
| connection = (HttpURLConnection) registry.openConnection(); | ||
| setAuthorizationHeader(matcher.group(1), connection); | ||
| } else { | ||
| registry = new URL(registryUrl); | ||
| connection = (HttpURLConnection) registry.openConnection(); | ||
| } | ||
| ObjectMapper mapper = new ObjectMapper(); | ||
| JsonNode node = mapper.readTree(registry.openStream()); | ||
| JsonNode node = mapper.readTree(getStream(connection)); | ||
| return node.get("schema").asText(); | ||
| } | ||
|
|
||
| public void setAuthorizationHeader(String creds, HttpURLConnection connection) { | ||
|
||
| String encodedAuth = Base64.getEncoder().encodeToString(creds.getBytes(StandardCharsets.UTF_8)); | ||
| connection.setRequestProperty("Authorization", "Basic " + encodedAuth); | ||
| } | ||
|
|
||
| public InputStream getStream(HttpURLConnection connection) throws IOException { | ||
| return connection.getInputStream(); | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. do we need a method for one line code. also, I don't see it being used in more than 1 place.
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. this is implemented like this for testing purposes, in order to allow for mocking/spying of the InputStream. I've made the method protected. |
||
| } | ||
|
|
||
| public SchemaRegistryProvider(TypedProperties props, JavaSparkContext jssc) { | ||
| super(props, jssc); | ||
| DataSourceUtils.checkRequiredProperties(props, Collections.singletonList(Config.SRC_SCHEMA_REGISTRY_URL_PROP)); | ||
| } | ||
|
|
||
| private static Schema getSchema(String registryUrl) throws IOException { | ||
| private Schema getSchema(String registryUrl) throws IOException { | ||
| return new Schema.Parser().parse(fetchSchemaFromRegistry(registryUrl)); | ||
| } | ||
|
|
||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,116 @@ | ||
| /* | ||
| * Licensed to the Apache Software Foundation (ASF) under one | ||
| * or more contributor license agreements. See the NOTICE file | ||
| * distributed with this work for additional information | ||
| * regarding copyright ownership. The ASF licenses this file | ||
| * to you under the Apache License, Version 2.0 (the | ||
| * "License"); you may not use this file except in compliance | ||
| * with the License. You may obtain a copy of the License at | ||
| * | ||
| * http://www.apache.org/licenses/LICENSE-2.0 | ||
| * | ||
| * Unless required by applicable law or agreed to in writing, software | ||
| * distributed under the License is distributed on an "AS IS" BASIS, | ||
| * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
| * See the License for the specific language governing permissions and | ||
| * limitations under the License. | ||
| */ | ||
|
|
||
| package org.apache.hudi.utilities.schema; | ||
|
|
||
| import com.fasterxml.jackson.databind.JsonNode; | ||
| import com.fasterxml.jackson.databind.ObjectMapper; | ||
| import org.apache.avro.Schema; | ||
| import org.apache.hudi.common.config.TypedProperties; | ||
| import org.junit.jupiter.api.Test; | ||
| import org.mockito.Mockito; | ||
|
|
||
| import java.io.ByteArrayInputStream; | ||
| import java.io.IOException; | ||
| import java.io.InputStream; | ||
| import java.net.HttpURLConnection; | ||
| import java.nio.charset.StandardCharsets; | ||
|
|
||
| import static org.junit.jupiter.api.Assertions.assertEquals; | ||
| import static org.junit.jupiter.api.Assertions.assertNotNull; | ||
| import static org.mockito.ArgumentMatchers.eq; | ||
| import static org.mockito.Mockito.times; | ||
| import static org.mockito.Mockito.verify; | ||
|
|
||
| class SchemaRegistryProviderTest { | ||
|
||
|
|
||
| private final String basicAuth = "foo:bar"; | ||
|
|
||
| private final String json = "{\"schema\":\"{\\\"type\\\": \\\"record\\\", \\\"namespace\\\": \\\"example\\\", " | ||
| + "\\\"name\\\": \\\"FullName\\\",\\\"fields\\\": [{ \\\"name\\\": \\\"first\\\", \\\"type\\\": " | ||
| + "\\\"string\\\" }]}\"}"; | ||
|
|
||
| private TypedProperties getProps() { | ||
| return new TypedProperties() {{ | ||
| put("hoodie.deltastreamer.schemaprovider.registry.baseUrl", "http://" + basicAuth + "@localhost"); | ||
| put("hoodie.deltastreamer.schemaprovider.registry.urlSuffix", "-value"); | ||
| put("hoodie.deltastreamer.schemaprovider.registry.url", "http://foo:bar@localhost"); | ||
| put("hoodie.deltastreamer.source.kafka.topic", "foo"); | ||
| }}; | ||
| } | ||
|
|
||
| Schema getExpectedSchema(String response) throws IOException { | ||
| ObjectMapper mapper = new ObjectMapper(); | ||
| JsonNode node = mapper.readTree(new ByteArrayInputStream(response.getBytes(StandardCharsets.UTF_8))); | ||
| return (new Schema.Parser()).parse(node.get("schema").asText()); | ||
| } | ||
|
|
||
| @Test | ||
| public void testGetSourceSchemaShouldRequestSchemaWithCreds() throws IOException { | ||
| InputStream is = new ByteArrayInputStream(json.getBytes(StandardCharsets.UTF_8)); | ||
| SchemaRegistryProvider underTest = new SchemaRegistryProvider(getProps(), null); | ||
| SchemaRegistryProvider spyUnderTest = Mockito.spy(underTest); | ||
| Mockito.doReturn(is).when(spyUnderTest).getStream(Mockito.any()); | ||
| Schema actual = spyUnderTest.getSourceSchema(); | ||
| assertNotNull(actual); | ||
| assertEquals(actual, getExpectedSchema(json)); | ||
| verify(spyUnderTest, times(1)).setAuthorizationHeader(eq(basicAuth), | ||
| Mockito.any(HttpURLConnection.class)); | ||
| } | ||
|
|
||
| @Test | ||
| public void testGetTargetSchemaShouldRequestSchemaWithCreds() throws IOException { | ||
| InputStream is = new ByteArrayInputStream(json.getBytes(StandardCharsets.UTF_8)); | ||
|
||
| SchemaRegistryProvider underTest = new SchemaRegistryProvider(getProps(), null); | ||
| SchemaRegistryProvider spyUnderTest = Mockito.spy(underTest); | ||
| Mockito.doReturn(is).when(spyUnderTest).getStream(Mockito.any()); | ||
| Schema actual = spyUnderTest.getTargetSchema(); | ||
| assertNotNull(actual); | ||
| assertEquals(actual, getExpectedSchema(json)); | ||
| verify(spyUnderTest, times(1)).setAuthorizationHeader(eq(basicAuth), | ||
| Mockito.any(HttpURLConnection.class)); | ||
| } | ||
|
|
||
| @Test | ||
| public void testGetSourceSchemaShouldRequestSchemaWithoutCreds() throws IOException { | ||
| TypedProperties props = getProps(); | ||
| props.put("hoodie.deltastreamer.schemaprovider.registry.url", "http://localhost"); | ||
| InputStream is = new ByteArrayInputStream(json.getBytes(StandardCharsets.UTF_8)); | ||
| SchemaRegistryProvider underTest = new SchemaRegistryProvider(props, null); | ||
| SchemaRegistryProvider spyUnderTest = Mockito.spy(underTest); | ||
| Mockito.doReturn(is).when(spyUnderTest).getStream(Mockito.any()); | ||
| Schema actual = spyUnderTest.getSourceSchema(); | ||
| assertNotNull(actual); | ||
| assertEquals(actual, getExpectedSchema(json)); | ||
| verify(spyUnderTest, times(0)).setAuthorizationHeader(Mockito.any(), Mockito.any()); | ||
| } | ||
|
|
||
| @Test | ||
| public void testGetTargetSchemaShouldRequestSchemaWithoutCreds() throws IOException { | ||
| TypedProperties props = getProps(); | ||
| props.put("hoodie.deltastreamer.schemaprovider.registry.url", "http://localhost"); | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. same comment as above.
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. done |
||
| InputStream is = new ByteArrayInputStream(json.getBytes(StandardCharsets.UTF_8)); | ||
| SchemaRegistryProvider underTest = new SchemaRegistryProvider(props, null); | ||
| SchemaRegistryProvider spyUnderTest = Mockito.spy(underTest); | ||
| Mockito.doReturn(is).when(spyUnderTest).getStream(Mockito.any()); | ||
| Schema actual = spyUnderTest.getTargetSchema(); | ||
| assertNotNull(actual); | ||
| assertEquals(actual, getExpectedSchema(json)); | ||
| verify(spyUnderTest, times(0)).setAuthorizationHeader(Mockito.any(), Mockito.any()); | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -615,25 +615,25 @@ | |
| <dependency> | ||
| <groupId>io.confluent</groupId> | ||
| <artifactId>kafka-avro-serializer</artifactId> | ||
| <version>3.0.0</version> | ||
| <version>5.3.4</version> | ||
| </dependency> | ||
|
|
||
| <dependency> | ||
| <groupId>io.confluent</groupId> | ||
| <artifactId>common-config</artifactId> | ||
| <version>3.0.0</version> | ||
| <version>5.3.4</version> | ||
|
||
| </dependency> | ||
|
|
||
| <dependency> | ||
| <groupId>io.confluent</groupId> | ||
| <artifactId>common-utils</artifactId> | ||
| <version>3.0.0</version> | ||
| <version>5.3.4</version> | ||
| </dependency> | ||
|
|
||
| <dependency> | ||
| <groupId>io.confluent</groupId> | ||
| <artifactId>kafka-schema-registry-client</artifactId> | ||
| <version>3.0.0</version> | ||
| <version>5.3.4</version> | ||
| </dependency> | ||
|
|
||
| <dependency> | ||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.