Skip to content
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

[QTL]Cached lookup module for JDBC connectors. #2819

Merged
merged 1 commit into from
Sep 16, 2016
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 @@ -49,5 +49,26 @@ public String getPassword()
public String toString() {
return this.getClass().getCanonicalName();
}

@Override
public boolean equals(Object o)
{
if (this == o) {
return true;
}
if (!(o instanceof DefaultPasswordProvider)) {
return false;
}

DefaultPasswordProvider that = (DefaultPasswordProvider) o;

return getPassword() != null ? getPassword().equals(that.getPassword()) : that.getPassword() == null;
}

@Override
public int hashCode()
{
return getPassword() != null ? getPassword().hashCode() : 0;
}
}

Original file line number Diff line number Diff line change
Expand Up @@ -86,4 +86,47 @@ public String toString()
", passwordProvider=" + passwordProvider +
'}';
}

@Override
public boolean equals(Object o)
Copy link
Contributor

Choose a reason for hiding this comment

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

please add proper tests for this addition.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

done

{
if (this == o) {
return true;
}
if (!(o instanceof MetadataStorageConnectorConfig)) {
return false;
}

MetadataStorageConnectorConfig that = (MetadataStorageConnectorConfig) o;

if (isCreateTables() != that.isCreateTables()) {
return false;
}
if (getPort() != that.getPort()) {
return false;
}
if (getHost() != null ? !getHost().equals(that.getHost()) : that.getHost() != null) {
return false;
}
if (getConnectURI() != null ? !getConnectURI().equals(that.getConnectURI()) : that.getConnectURI() != null) {
return false;
}
if (getUser() != null ? !getUser().equals(that.getUser()) : that.getUser() != null) {
return false;
}
return passwordProvider != null ? passwordProvider.equals(that.passwordProvider) : that.passwordProvider == null;
Copy link
Contributor

Choose a reason for hiding this comment

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

What kind of problems might this cause if PasswordProvider does not properly implement equals?

Also making @himanshug aware of this

Copy link
Contributor Author

Choose a reason for hiding this comment

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

ser/desr will fail


}

@Override
public int hashCode()
{
int result = (isCreateTables() ? 1 : 0);
result = 31 * result + (getHost() != null ? getHost().hashCode() : 0);
result = 31 * result + getPort();
result = 31 * result + (getConnectURI() != null ? getConnectURI().hashCode() : 0);
result = 31 * result + (getUser() != null ? getUser().hashCode() : 0);
result = 31 * result + (passwordProvider != null ? passwordProvider.hashCode() : 0);
return result;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,55 @@
import org.junit.Assert;
import org.junit.Test;

public class MetadataStorageConnectorConfigTest {
import java.io.IOException;

public class MetadataStorageConnectorConfigTest
{

private MetadataStorageConnectorConfig createMetadataStorageConfig(
boolean createTables,
String host,
int port,
String connectURI,
String user,
String pwdString
)
throws IOException
{
return jsonMapper.readValue(
"{" +
"\"createTables\": \"" + createTables + "\"," +
"\"host\": \"" + host + "\"," +
"\"port\": \"" + port + "\"," +
"\"connectURI\": \"" + connectURI + "\"," +
"\"user\": \"" + user + "\"," +
"\"password\": " + pwdString +
"}",
MetadataStorageConnectorConfig.class
);
}

@Test
public void testEquals() throws IOException
{
MetadataStorageConnectorConfig metadataStorageConnectorConfig = createMetadataStorageConfig(
true,
"testHost",
4000,
"url",
"user",
"\"nothing\""
);
MetadataStorageConnectorConfig metadataStorageConnectorConfig2 = createMetadataStorageConfig(
true,
"testHost",
4000,
"url",
"user",
"\"nothing\""
);
Assert.assertTrue(metadataStorageConnectorConfig.equals(metadataStorageConnectorConfig2));
}

private static final ObjectMapper jsonMapper = new ObjectMapper();

Expand All @@ -37,7 +85,8 @@ public void testMetadaStorageConnectionConfigSimplePassword() throws Exception
"connectURI",
"user",
"\"nothing\"",
"nothing");
"nothing"
);
}

@Test
Expand All @@ -50,29 +99,31 @@ public void testMetadaStorageConnectionConfigWithDefaultProviderPassword() throw
"connectURI",
"user",
"{\"type\":\"default\",\"password\":\"nothing\"}",
"nothing");
"nothing"
);
}

private void testMetadataStorageConnectionConfig(
boolean createTables,
String host,
int port,
String connectURI,
String user,
String pwdString,
String pwd
) throws Exception
boolean createTables,
String host,
int port,
String connectURI,
String user,
String pwdString,
String pwd
) throws Exception
{
MetadataStorageConnectorConfig config = jsonMapper.readValue(
"{" +
"\"createTables\": \"" + createTables + "\"," +
"\"host\": \"" + host + "\"," +
"\"port\": \"" + port + "\"," +
"\"connectURI\": \"" + connectURI + "\"," +
"\"user\": \"" + user + "\"," +
"\"password\": " + pwdString +
"}",
MetadataStorageConnectorConfig.class);
"{" +
"\"createTables\": \"" + createTables + "\"," +
"\"host\": \"" + host + "\"," +
"\"port\": \"" + port + "\"," +
"\"connectURI\": \"" + connectURI + "\"," +
"\"user\": \"" + user + "\"," +
"\"password\": " + pwdString +
"}",
MetadataStorageConnectorConfig.class
);

Assert.assertEquals(host, config.getHost());
Assert.assertEquals(port, config.getPort());
Expand Down
129 changes: 129 additions & 0 deletions docs/content/development/extensions-core/druid-lookups.md
Original file line number Diff line number Diff line change
@@ -0,0 +1,129 @@
---
layout: doc_page
---
# Cached Lookup Module
Copy link
Contributor

Choose a reason for hiding this comment

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

this will not render correctly without


layout: doc_page

Copy link
Contributor Author

Choose a reason for hiding this comment

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

thanks fixed !


<div class="note info">Please note that this is an experimental module and the development/testing still at early stage. Feel free to try it and give us your feedback.</div>

## Description
This module provides a per-lookup caching mechanism for JDBC data sources.
The main goal of this cache is to speed up the access to a high latency lookup sources and to provide a caching isolation for every lookup source.
Thus user can define various caching strategies or and implementation per lookup, even if the source is the same.
Copy link
Contributor

Choose a reason for hiding this comment

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

Could you include something about whether it is intended that users could use this together with lookups-cached-global? Would be good to answer questions like:

  • Do they conflict with each other or can they be loaded simultaneously?
  • Does it make sense to run them side by side while migrating from one to the other?
  • Does it make sense to run them side by side long term?

Copy link
Contributor

Choose a reason for hiding this comment

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

Could you also include how to set up this extension? Something like:

Make sure to [include](../../operations/including-extensions.html) `druid-lookups-cached-single` as an extension.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

good point will add

Copy link
Contributor Author

Choose a reason for hiding this comment

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

done

This module can be used side to side with other lookup module like the global cached lookup module.

To use this extension please make sure to [include](../../operations/including-extensions.html) `druid-lookups-cached-single` as an extension.

## Architecture
Generally speaking this module can be divided into two main component, namely, the data fetcher layer and caching layer.

### Data Fetcher layer

First part is the data fetcher layer API `DataFetcher`, that exposes a set of fetch methods to fetch data from the actual Lookup dimension source.
For instance `JdbcDataFetcher` provides an implementation of `DataFetcher` that can be used to fetch key/value from a RDBMS via JDBC driver.
If you need new type of data fetcher, all you need to do, is to implement the interface `DataFetcher` and load it via another druid module.
### Caching layer

This extension comes with two different caching strategies. First strategy is a poll based and the second is a load based.
#### Poll lookup cache

The poll strategy cache strategy will fetch and swap all the pair of key/values periodically from the lookup source.
Hence, user should make sure that the cache can fit all the data.
The current implementation provides 2 type of poll cache, the first is onheap (uses immutable map), while the second uses MapBD based offheap map.
User can also implement a different lookup polling cache by implementing `PollingCacheFactory` and `PollingCache` interfaces.

#### Loading lookup
Loading cache strategy will load the key\value pair upon request on the key it self, the general algorithm is load key if absent.
Once the key/value pair is loaded eviction will occur according to the cache eviction policy.
Copy link
Contributor

Choose a reason for hiding this comment

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

extra ' '

Copy link
Contributor Author

Choose a reason for hiding this comment

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

not sure where ?

This module comes with two loading lookup implementation, the first is onheap backed by a Guava cache implementation, the second is MapDB offheap implementation.
Both implementations offer various eviction strategies.
Same for Loading cache, developer can implement a new type of loading cache by implementing `LookupLoadingCache` interface.

## Configuration and Operation:


### Polling Lookup

**Note that the current implementation of `offHeapPolling` and `onHeapPolling` will create two caches one to lookup value based on key and the other to reverse lookup the key from value**

|Field|Type|Description|Required|default|
|-----|----|-----------|--------|-------|
|dataFetcher|Json object|Specifies the lookup data fetcher type to use in order to fetch data|yes|null|
|cacheFactory|Json Object|Cache factory implementation|no |onHeapPolling|
|pollPeriod|Period|polling period |no |null (poll once)|


##### Example of Polling On-heap Lookup
This example demonstrates a polling cache that will update its on-heap cache every 10 minutes
```json
{
"type":"pollingLookup",
"pollPeriod":"PT10M",
"dataFetcher":{ "type":"jdbcDataFetcher", "connectorConfig":"jdbc://mysql://localhost:3306/my_data_base", "table":"lookup_table_name", "keyColumn":"key_column_name", "valueColumn": "value_column_name"},
"cacheFactory":{"type":"onHeapPolling"}
}

```

##### Example Polling Off-heap Lookup
This example demonstrates an off-heap lookup that will be cached once and never swapped `(pollPeriod == null)`

```json
{
"type":"pollingLookup",
"dataFetcher":{ "type":"jdbcDataFetcher", "connectorConfig":"jdbc://mysql://localhost:3306/my_data_base", "table":"lookup_table_name", "keyColumn":"key_column_name", "valueColumn": "value_column_name"},
"cacheFactory":{"type":"offHeapPolling"}
}

```


### Loading lookup

|Field|Type|Description|Required|default|
|-----|----|-----------|--------|-------|
|dataFetcher|Json object|Specifies the lookup data fetcher type to use in order to fetch data|yes|null|
|loadingCacheSpec|Json Object|Lookup cache spec implementation|yes |null|
|reverseLoadingCacheSpec|Json Object| Reverse lookup cache implementation|yes |null|


##### Example Loading On-heap Guava

Guava cache configuration spec.

|Field|Type|Description|Required|default|
|-----|----|-----------|--------|-------|
|concurrencyLevel|int|Allowed concurrency among update operations|no|4|
Copy link
Contributor

Choose a reason for hiding this comment

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

Suggest making the default 1

Copy link
Contributor

Choose a reason for hiding this comment

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

Ah nevermind, 4 is the guava default.

|initialCapacity|int|Initial capacity size|no |null|
|maximumSize|long| Specifies the maximum number of entries the cache may contain.|no |null (infinite capacity)|
|expireAfterAccess|long| Specifies the eviction time after last read in milliseconds.|no |null (No read-time-based eviction when set to null)|
|expireAfterWrite|long| Specifies the eviction time after last write in milliseconds.|no |null (No write-time-based eviction when set to null)|

```json
{
"type":"loadingLookup",
"dataFetcher":{ "type":"jdbcDataFetcher", "connectorConfig":"jdbc://mysql://localhost:3306/my_data_base", "table":"lookup_table_name", "keyColumn":"key_column_name", "valueColumn": "value_column_name"},
"loadingCacheSpec":{"type":"guava"},
"reverseLoadingCacheSpec":{"type":"guava", "maximumSize":500000, "expireAfterAccess":100000, "expireAfterAccess":10000}
}
```

##### Example Loading Off-heap MapDB

Off heap cache is backed by [MapDB](http://www.mapdb.org/) implementation. MapDB is using direct memory as memory pool, please take that into account when limiting the JVM direct memory setup.

|Field|Type|Description|Required|default|
|-----|----|-----------|--------|-------|
|maxStoreSize|double|maximal size of store in GB, if store is larger entries will start expiring|no |0|
|maxEntriesSize|long| Specifies the maximum number of entries the cache may contain.|no |0 (infinite capacity)|
|expireAfterAccess|long| Specifies the eviction time after last read in milliseconds.|no |0 (No read-time-based eviction when set to null)|
|expireAfterWrite|long| Specifies the eviction time after last write in milliseconds.|no |0 (No write-time-based eviction when set to null)|


```json
{
"type":"loadingLookup",
"dataFetcher":{ "type":"jdbcDataFetcher", "connectorConfig":"jdbc://mysql://localhost:3306/my_data_base", "table":"lookup_table_name", "keyColumn":"key_column_name", "valueColumn": "value_column_name"},
"loadingCacheSpec":{"type":"mapDb", "maxEntriesSize":100000},
"reverseLoadingCacheSpec":{"type":"mapDb", "maxStoreSize":5, "expireAfterAccess":100000, "expireAfterAccess":10000}
}
```
1 change: 1 addition & 0 deletions docs/content/development/extensions.md
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@ Core extensions are maintained by Druid committers.
|druid-kafka-eight|Kafka ingest firehose (high level consumer).|[link](../development/extensions-core/kafka-eight-firehose.html)|
|druid-kafka-extraction-namespace|Kafka-based namespaced lookup. Requires namespace lookup extension.|[link](../development/extensions-core/kafka-extraction-namespace.html)|
|druid-lookups-cached-global|A module for [lookups](../querying/lookups.html) providing a jvm-global eager caching for lookups. It provides JDBC and URI implementations for fetching lookup data.|[link](../development/extensions-core/lookups-cached-global.html)|
|druid-lookups-cached-single| Per lookup caching module to support the use cases where a lookup need to be isolated from the global pool of lookups |[link](../development/extensions-core/druid-lookups.html)|
|druid-s3-extensions|Interfacing with data in AWS S3, and using S3 as deep storage.|[link](../development/extensions-core/s3.html)|
|druid-stats|Statistics related module including variance and standard deviation.|[link](../development/extensions-core/stats.html)|
|mysql-metadata-storage|MySQL metadata store.|[link](../development/extensions-core/mysql.html)|
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -169,6 +169,6 @@ public Timestamp withHandle(Handle handle) throws Exception
}
);
return update.getTime();
}

}
}
Loading