Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -77,7 +77,7 @@ public void testFileGroupLookUpManyEntriesWithSameStartValue() {
}

/**
* Tests for many duplicte entries in the tree.
* Tests for many duplicate entries in the tree.
*/
@Test
public void testFileGroupLookUpManyDulicateEntries() {
Expand Down
8 changes: 4 additions & 4 deletions hudi-utilities/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -261,22 +261,22 @@
<dependency>
<groupId>io.confluent</groupId>
<artifactId>kafka-avro-serializer</artifactId>
<version>3.0.0</version>
<version>${confluent.version}</version>
</dependency>
<dependency>
<groupId>io.confluent</groupId>
<artifactId>common-config</artifactId>
<version>3.0.0</version>
<version>${confluent.version}</version>
</dependency>
<dependency>
<groupId>io.confluent</groupId>
<artifactId>common-utils</artifactId>
<version>3.0.0</version>
<version>${confluent.version}</version>
</dependency>
<dependency>
<groupId>io.confluent</groupId>
<artifactId>kafka-schema-registry-client</artifactId>
<version>3.0.0</version>
<version>${confluent.version}</version>
</dependency>

<dependency>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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.
Expand All @@ -48,19 +54,49 @@ public static class Config {
"hoodie.deltastreamer.schemaprovider.registry.targetUrl";
}

private static String fetchSchemaFromRegistry(String registryUrl) throws IOException {
URL registry = new URL(registryUrl);
/**
* The method takes the provided url {@code registryUrl} and gets the schema from the schema registry using that url.
* If the caller provides userInfo credentials in the url (e.g "https://foo:bar@schemaregistry.org") then the credentials
* are extracted the url using the Matcher and the extracted credentials are set on the request as an Authorization
* header.
* @param registryUrl
* @return the Schema in String form.
* @throws IOException
*/
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();
}

protected void setAuthorizationHeader(String creds, HttpURLConnection connection) {
String encodedAuth = Base64.getEncoder().encodeToString(creds.getBytes(StandardCharsets.UTF_8));
connection.setRequestProperty("Authorization", "Basic " + encodedAuth);
}

protected InputStream getStream(HttpURLConnection connection) throws IOException {
return connection.getInputStream();
Copy link
Contributor

Choose a reason for hiding this comment

The 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.

Copy link
Contributor Author

@jhsb25 jhsb25 Jun 29, 2021

Choose a reason for hiding this comment

The 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));
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -96,7 +96,7 @@ private IndexedRecord createExtendUserRecord() {
}

/**
* Tests {@link KafkaAvroSchemaDeserializer#deserialize(boolean, String, Boolean, byte[], Schema)}.
* Tests {@link KafkaAvroSchemaDeserializer#deserialize(Boolean, String, Boolean, byte[], Schema)}.
*/
@Test
public void testKafkaAvroSchemaDeserializer() {
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,111 @@
/*
* 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 TestSchemaRegistryProvider {

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");
}};
}

private 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());
}

private SchemaRegistryProvider getUnderTest(TypedProperties props) throws IOException {
InputStream is = new ByteArrayInputStream(json.getBytes(StandardCharsets.UTF_8));
SchemaRegistryProvider spyUnderTest = Mockito.spy(new SchemaRegistryProvider(props, null));
Mockito.doReturn(is).when(spyUnderTest).getStream(Mockito.any());
return spyUnderTest;
}

@Test
public void testGetSourceSchemaShouldRequestSchemaWithCreds() throws IOException {
SchemaRegistryProvider spyUnderTest = getUnderTest(getProps());
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 {
SchemaRegistryProvider spyUnderTest = getUnderTest(getProps());
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");
SchemaRegistryProvider spyUnderTest = getUnderTest(props);
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");
SchemaRegistryProvider spyUnderTest = getUnderTest(props);
Schema actual = spyUnderTest.getTargetSchema();
assertNotNull(actual);
assertEquals(actual, getExpectedSchema(json));
verify(spyUnderTest, times(0)).setAuthorizationHeader(Mockito.any(), Mockito.any());
}
}
8 changes: 4 additions & 4 deletions packaging/hudi-integ-test-bundle/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -615,25 +615,25 @@
<dependency>
<groupId>io.confluent</groupId>
<artifactId>kafka-avro-serializer</artifactId>
<version>3.0.0</version>
<version>${confluent.version}</version>
</dependency>

<dependency>
<groupId>io.confluent</groupId>
<artifactId>common-config</artifactId>
<version>3.0.0</version>
<version>${confluent.version}</version>
</dependency>

<dependency>
<groupId>io.confluent</groupId>
<artifactId>common-utils</artifactId>
<version>3.0.0</version>
<version>${confluent.version}</version>
</dependency>

<dependency>
<groupId>io.confluent</groupId>
<artifactId>kafka-schema-registry-client</artifactId>
<version>3.0.0</version>
<version>${confluent.version}</version>
</dependency>

<dependency>
Expand Down
1 change: 1 addition & 0 deletions packaging/hudi-utilities-bundle/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -93,6 +93,7 @@
<include>com.twitter:bijection-avro_${scala.binary.version}</include>
<include>com.twitter:bijection-core_${scala.binary.version}</include>
<include>io.confluent:kafka-avro-serializer</include>
<include>io.confluent:kafka-schema-serializer</include>
<include>io.confluent:common-config</include>
<include>io.confluent:common-utils</include>
<include>io.confluent:kafka-schema-registry-client</include>
Expand Down
1 change: 1 addition & 0 deletions pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -89,6 +89,7 @@
<fasterxml.jackson.dataformat.yaml.version>2.7.4</fasterxml.jackson.dataformat.yaml.version>
<fasterxml.spark3.version>2.10.0</fasterxml.spark3.version>
<kafka.version>2.0.0</kafka.version>
<confluent.version>5.3.4</confluent.version>
<glassfish.version>2.17</glassfish.version>
<parquet.version>1.10.1</parquet.version>
<junit.jupiter.version>5.7.0-M1</junit.jupiter.version>
Expand Down