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
@@ -0,0 +1,39 @@
/*
* 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.druid.metadata;

import com.fasterxml.jackson.annotation.JsonSubTypes;
import com.fasterxml.jackson.annotation.JsonTypeInfo;
import org.apache.druid.guice.annotations.ExtensionPoint;

import java.util.Map;

/**
* This is used to get [secure] configuration in various places in an extensible way.
*/
@ExtensionPoint
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = MapStringDynamicConfigProvider.class)
@JsonSubTypes(value = {
@JsonSubTypes.Type(name = "mapString", value = MapStringDynamicConfigProvider.class),
})
public interface DynamicConfigProvider<T>
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: should we annotate PasswordProvider as @Deprecated in favor of this class?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I didn't mark that deprecated because users wanting to have their own extension for db password must still implement a PasswordProvider ,
however, for Druid devs, any new credential/extensible-config type thing must use DynamicConfigProvider ... so PasswordProvider is "deprecated" in that sense.
will mark it deprecated and add few comments.

{
Map<String, T> getConfig();
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,47 @@
/*
* 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.druid.metadata;

import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.collect.ImmutableMap;

import java.util.Map;

public class MapStringDynamicConfigProvider implements DynamicConfigProvider<String>
{
private final ImmutableMap<String, String> config;

@JsonCreator
public MapStringDynamicConfigProvider(
@JsonProperty("config") Map<String, String> config
)
{
this.config = ImmutableMap.copyOf(config);
}


@Override
@JsonProperty
public Map<String, String> getConfig()
{
return config;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,13 @@

/**
* Implement this for different ways to (optionally securely) access secrets.
*
* Any further use case of extensible configuration/secrets must use {@link DynamicConfigProvider} interface. Users
* may still implement this interface for existing use cases till https://github.com/apache/druid/issues/9351 is
* resolved.
*
*/
@Deprecated
@ExtensionPoint
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = DefaultPasswordProvider.class)
@JsonSubTypes(value = {
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,44 @@
/*
* 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.druid.metadata;

import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.ImmutableMap;
import org.junit.Assert;
import org.junit.Test;

public class MapStringDynamicConfigProviderTest
{
@Test
public void testSerde() throws Exception
{
DynamicConfigProvider<String> original = new MapStringDynamicConfigProvider(ImmutableMap.of("k", "v"));

ObjectMapper jsonMapper = new ObjectMapper();

MapStringDynamicConfigProvider recreated = (MapStringDynamicConfigProvider) jsonMapper.readValue(
jsonMapper.writeValueAsString(original),
DynamicConfigProvider.class
);

Assert.assertEquals(1, recreated.getConfig().size());
Assert.assertEquals("v", recreated.getConfig().get("k"));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,7 @@
import org.apache.druid.indexing.seekablestream.common.StreamException;
import org.apache.druid.indexing.seekablestream.common.StreamPartition;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.metadata.DynamicConfigProvider;
import org.apache.druid.metadata.PasswordProvider;
import org.apache.kafka.clients.consumer.ConsumerRecord;
import org.apache.kafka.clients.consumer.KafkaConsumer;
Expand Down Expand Up @@ -192,16 +193,30 @@ public static void addConsumerPropertiesFromConfig(
// Extract passwords before SSL connection to Kafka
for (Map.Entry<String, Object> entry : consumerProperties.entrySet()) {
String propertyKey = entry.getKey();
if (propertyKey.equals(KafkaSupervisorIOConfig.TRUST_STORE_PASSWORD_KEY)
|| propertyKey.equals(KafkaSupervisorIOConfig.KEY_STORE_PASSWORD_KEY)
|| propertyKey.equals(KafkaSupervisorIOConfig.KEY_PASSWORD_KEY)) {
PasswordProvider configPasswordProvider = configMapper.convertValue(
entry.getValue(),
PasswordProvider.class
);
properties.setProperty(propertyKey, configPasswordProvider.getPassword());
} else {
properties.setProperty(propertyKey, String.valueOf(entry.getValue()));

if (!KafkaSupervisorIOConfig.DRUID_DYNAMIC_CONFIG_PROVIDER_KEY.equals(propertyKey)) {
if (propertyKey.equals(KafkaSupervisorIOConfig.TRUST_STORE_PASSWORD_KEY)
|| propertyKey.equals(KafkaSupervisorIOConfig.KEY_STORE_PASSWORD_KEY)
|| propertyKey.equals(KafkaSupervisorIOConfig.KEY_PASSWORD_KEY)) {
PasswordProvider configPasswordProvider = configMapper.convertValue(
entry.getValue(),
PasswordProvider.class
);
properties.setProperty(propertyKey, configPasswordProvider.getPassword());
} else {
properties.setProperty(propertyKey, String.valueOf(entry.getValue()));
}
}
}

// Additional DynamicConfigProvider based extensible support for all consumer properties
Object dynamicConfigProviderJson = consumerProperties.get(KafkaSupervisorIOConfig.DRUID_DYNAMIC_CONFIG_PROVIDER_KEY);
if (dynamicConfigProviderJson != null) {
DynamicConfigProvider dynamicConfigProvider = configMapper.convertValue(dynamicConfigProviderJson, DynamicConfigProvider.class);
Map<String, String> dynamicConfig = dynamicConfigProvider.getConfig();

for (Map.Entry<String, String> e : dynamicConfig.entrySet()) {
properties.setProperty(e.getKey(), e.getValue());
}
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@

public class KafkaSupervisorIOConfig extends SeekableStreamSupervisorIOConfig
{
public static final String DRUID_DYNAMIC_CONFIG_PROVIDER_KEY = "druid.dynamic.config.provider";
public static final String BOOTSTRAP_SERVERS_KEY = "bootstrap.servers";
public static final String TRUST_STORE_PASSWORD_KEY = "ssl.truststore.password";
public static final String KEY_STORE_PASSWORD_KEY = "ssl.keystore.password";
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,10 +24,13 @@
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet;
import org.apache.curator.test.TestingCluster;
import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorIOConfig;
import org.apache.druid.indexing.kafka.test.TestBroker;
import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord;
import org.apache.druid.indexing.seekablestream.common.StreamPartition;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.metadata.DynamicConfigProvider;
import org.apache.druid.metadata.MapStringDynamicConfigProvider;
import org.apache.druid.segment.TestHelper;
import org.apache.kafka.clients.producer.KafkaProducer;
import org.apache.kafka.clients.producer.ProducerRecord;
Expand All @@ -43,6 +46,7 @@
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Properties;
import java.util.Set;
import java.util.concurrent.ExecutionException;
import java.util.stream.Collectors;
Expand Down Expand Up @@ -582,6 +586,33 @@ public void getEarliestSequenceNumberWhenPartitionIsEmptyAndUseLatestOffsetShoul
Assert.assertEquals(new Long(0), recordSupplier.getEarliestSequenceNumber(streamPartition));
}

@Test
public void testAddConsumerPropertiesFromConfig()
{
DynamicConfigProvider dynamicConfigProvider = new MapStringDynamicConfigProvider(
ImmutableMap.of("kafka.prop.2", "value.2", KafkaSupervisorIOConfig.TRUST_STORE_PASSWORD_KEY, "pwd2")
);

Properties properties = new Properties();

Map<String, Object> consumerProperties = ImmutableMap.of(
KafkaSupervisorIOConfig.TRUST_STORE_PASSWORD_KEY, "pwd1",
"kafka.prop.1", "value.1",
"druid.dynamic.config.provider", OBJECT_MAPPER.convertValue(dynamicConfigProvider, Map.class)
);

KafkaRecordSupplier.addConsumerPropertiesFromConfig(
properties,
OBJECT_MAPPER,
consumerProperties
);

Assert.assertEquals(3, properties.size());
Assert.assertEquals("value.1", properties.getProperty("kafka.prop.1"));
Assert.assertEquals("value.2", properties.getProperty("kafka.prop.2"));
Assert.assertEquals("pwd2", properties.getProperty(KafkaSupervisorIOConfig.TRUST_STORE_PASSWORD_KEY));
}

private void insertData() throws ExecutionException, InterruptedException
{
try (final KafkaProducer<byte[], byte[]> kafkaProducer = kafkaServer.newProducer()) {
Expand Down