mirror of https://github.com/apache/druid.git
fix more things
This commit is contained in:
parent
cc18bdeb0d
commit
9f75df63f3
|
@ -18,11 +18,6 @@
|
|||
<groupId>io.druid</groupId>
|
||||
<artifactId>druid-api</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>io.druid</groupId>
|
||||
<artifactId>druid-processing</artifactId>
|
||||
<version>${project.parent.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.rabbitmq</groupId>
|
||||
<artifactId>amqp-client</artifactId>
|
||||
|
@ -45,5 +40,11 @@
|
|||
<artifactId>commons-cli</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>io.druid</groupId>
|
||||
<artifactId>druid-processing</artifactId>
|
||||
<version>${project.parent.version}</version>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
</dependencies>
|
||||
</project>
|
|
@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonCreator;
|
|||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.rabbitmq.client.ConnectionFactory;
|
||||
import com.rabbitmq.client.LongString;
|
||||
|
||||
import java.util.Map;
|
||||
|
||||
|
@ -37,6 +38,24 @@ public class JacksonifiedConnectionFactory extends ConnectionFactory
|
|||
return new JacksonifiedConnectionFactory(null, 0, null, null, null, null, 0, 0, 0, 0, null);
|
||||
}
|
||||
|
||||
private static Map<String, Object> getSerializableClientProperties(final Map<String, Object> clientProperties)
|
||||
{
|
||||
return Maps.transformEntries(
|
||||
clientProperties,
|
||||
new Maps.EntryTransformer<String, Object, Object>()
|
||||
{
|
||||
@Override
|
||||
public Object transformEntry(String key, Object value)
|
||||
{
|
||||
if (value instanceof LongString) {
|
||||
return value.toString();
|
||||
}
|
||||
return value;
|
||||
}
|
||||
}
|
||||
);
|
||||
}
|
||||
|
||||
private final String host;
|
||||
private final int port;
|
||||
private final String username;
|
||||
|
@ -164,19 +183,9 @@ public class JacksonifiedConnectionFactory extends ConnectionFactory
|
|||
}
|
||||
|
||||
@JsonProperty("clientProperties")
|
||||
public Map<String, Object> getClientPropertiesForSerde()
|
||||
public Map<String, Object> getSerializableClientProperties()
|
||||
{
|
||||
return Maps.transformEntries(
|
||||
super.getClientProperties(),
|
||||
new Maps.EntryTransformer<String, Object, Object>()
|
||||
{
|
||||
@Override
|
||||
public Object transformEntry(String key, Object value)
|
||||
{
|
||||
return value.toString();
|
||||
}
|
||||
}
|
||||
);
|
||||
return getSerializableClientProperties(clientProperties);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -206,7 +215,12 @@ public class JacksonifiedConnectionFactory extends ConnectionFactory
|
|||
if (requestedHeartbeat != that.requestedHeartbeat) {
|
||||
return false;
|
||||
}
|
||||
if (clientProperties != null ? Maps.difference(clientProperties, that.clientProperties).areEqual() : that.clientProperties != null) {
|
||||
if (clientProperties != null
|
||||
? !Maps.difference(
|
||||
getSerializableClientProperties(clientProperties),
|
||||
getSerializableClientProperties(that.clientProperties)
|
||||
).areEqual()
|
||||
: that.clientProperties != null) {
|
||||
return false;
|
||||
}
|
||||
if (host != null ? !host.equals(that.host) : that.host != null) {
|
||||
|
|
Loading…
Reference in New Issue