Skip to content

Commit

Permalink
KYLIN-3376 Some improvements for lookup table - query change
Browse files Browse the repository at this point in the history
  • Loading branch information
allenma authored and shaofengshi committed Jun 1, 2018
1 parent 777f8fa commit 5a96f8b
Show file tree
Hide file tree
Showing 19 changed files with 1,224 additions and 60 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -75,7 +75,7 @@ public static CapabilityResult check(CubeInstance cube, SQLDigest digest) {
}
} else {
//for non query-on-facttable
if (cube.getSegments().get(0).getSnapshots().containsKey(digest.factTable)) {
if (cube.getSegments().get(0).getSnapshots().containsKey(digest.factTable) || cube.getSnapshots().containsKey(digest.factTable)) {

Set<TblColRef> dimCols = Sets.newHashSet(cube.getModel().findFirstTable(digest.factTable).getColumns());

Expand Down
5 changes: 5 additions & 0 deletions core-dictionary/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -38,6 +38,11 @@
<artifactId>kylin-core-metadata</artifactId>
</dependency>

<dependency>
<groupId>org.rocksdb</groupId>
<artifactId>rocksdbjni</artifactId>
</dependency>

<!-- Env & Test -->
<dependency>
<groupId>org.apache.kylin</groupId>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@

import java.io.IOException;
import java.util.Comparator;
import java.util.Iterator;

import org.apache.kylin.common.util.DateFormat;
import org.apache.kylin.metadata.datatype.DataType;
Expand All @@ -31,7 +32,7 @@
* @author yangli9
*
*/
public class LookupStringTable extends LookupTable<String> {
public class LookupStringTable extends LookupTable<String> implements ILookupTable{

private static final Comparator<String> dateStrComparator = new Comparator<String>() {
@Override
Expand Down Expand Up @@ -109,4 +110,13 @@ public Class<?> getType() {
return String.class;
}

@Override
public Iterator<String[]> iterator() {
return data.values().iterator();
}

@Override
public void close() throws IOException {
// do nothing
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -22,10 +22,10 @@
import java.util.ArrayList;
import java.util.Collection;
import java.util.Comparator;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;

import org.apache.commons.io.IOUtils;
import org.apache.kylin.common.util.Array;
Expand Down Expand Up @@ -53,7 +53,7 @@ public LookupTable(TableDesc tableDesc, String[] keyColumns, IReadableTable tabl
this.tableDesc = tableDesc;
this.keyColumns = keyColumns;
this.table = table;
this.data = new ConcurrentHashMap<Array<T>, T[]>();
this.data = new HashMap<Array<T>, T[]>();
init();
}

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,83 @@
/*
* 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.kylin.dict.lookup.cache;

import java.io.File;

import org.apache.commons.io.FileUtils;
import org.apache.kylin.dict.lookup.ILookupTable;
import org.apache.kylin.dict.lookup.cache.RocksDBLookupRowEncoder.KV;
import org.apache.kylin.metadata.model.TableDesc;
import org.rocksdb.CompactionStyle;
import org.rocksdb.CompressionType;
import org.rocksdb.Options;
import org.rocksdb.RocksDB;
import org.rocksdb.RocksDBException;
import org.rocksdb.util.SizeUnit;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

public class RocksDBLookupBuilder {
private static final Logger logger = LoggerFactory.getLogger(RocksDBLookupBuilder.class);

static {
RocksDB.loadLibrary();
}
private Options options;
private String dbPath;
private TableDesc tableDesc;
private RocksDBLookupRowEncoder encoder;
private int writeBatchSize;

public RocksDBLookupBuilder(TableDesc tableDesc, String[] keyColumns, String dbPath) {
this.tableDesc = tableDesc;
this.encoder = new RocksDBLookupRowEncoder(tableDesc, keyColumns);
this.dbPath = dbPath;
this.writeBatchSize = 500;
this.options = new Options();
options.setCreateIfMissing(true).setWriteBufferSize(8 * SizeUnit.KB).setMaxWriteBufferNumber(3)
.setMaxBackgroundCompactions(5).setCompressionType(CompressionType.SNAPPY_COMPRESSION)
.setCompactionStyle(CompactionStyle.UNIVERSAL);

}

public void build(ILookupTable srcLookupTable) {
File dbFolder = new File(dbPath);
if (dbFolder.exists()) {
logger.info("remove rocksdb folder:{} to rebuild table cache:{}", dbPath, tableDesc.getIdentity());
FileUtils.deleteQuietly(dbFolder);
} else {
logger.info("create new rocksdb folder:{} for table cache:{}", dbPath, tableDesc.getIdentity());
dbFolder.mkdirs();
}
logger.info("start to build lookup table:{} to rocks db:{}", tableDesc.getIdentity(), dbPath);
try (RocksDB rocksDB = RocksDB.open(options, dbPath)) {
// todo use batch may improve write performance
for (String[] row : srcLookupTable) {
KV kv = encoder.encode(row);

rocksDB.put(kv.getKey(), kv.getValue());
}
} catch (RocksDBException e) {
throw new RuntimeException("error when write data to rocks db", e);
}

logger.info("source table:{} has been written to rocks db:{}", tableDesc.getIdentity(), dbPath);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,70 @@
/*
* 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.kylin.dict.lookup.cache;

import org.apache.kylin.dict.lookup.AbstractLookupRowEncoder;
import org.apache.kylin.dict.lookup.cache.RocksDBLookupRowEncoder.KV;
import org.apache.kylin.metadata.model.TableDesc;

/**
* encode/decode original table row to rocksDB KV
*
*/
public class RocksDBLookupRowEncoder extends AbstractLookupRowEncoder<KV>{

public RocksDBLookupRowEncoder(TableDesc tableDesc, String[] keyColumns) {
super(tableDesc, keyColumns);
}

public KV encode(String[] row) {
String[] keys = getKeyData(row);
String[] values = getValueData(row);
byte[] encodeKey = encodeStringsWithLenPfx(keys, false);
byte[] encodeValue = encodeStringsWithLenPfx(values, true);

return new KV(encodeKey, encodeValue);
}

public String[] decode(KV kv) {
String[] result = new String[columnsNum];

decodeFromLenPfxBytes(kv.key, keyIndexes, result);
decodeFromLenPfxBytes(kv.value, valueIndexes, result);

return result;
}

public static class KV {
private byte[] key;
private byte[] value;

public KV(byte[] key, byte[] value) {
this.key = key;
this.value = value;
}

public byte[] getKey() {
return key;
}

public byte[] getValue() {
return value;
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,113 @@
/*
* 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.kylin.dict.lookup.cache;

import java.io.IOException;
import java.util.Iterator;

import org.apache.kylin.common.util.Array;
import org.apache.kylin.dict.lookup.ILookupTable;
import org.apache.kylin.dict.lookup.cache.RocksDBLookupRowEncoder.KV;
import org.apache.kylin.metadata.model.TableDesc;
import org.rocksdb.Options;
import org.rocksdb.RocksDB;
import org.rocksdb.RocksDBException;
import org.rocksdb.RocksIterator;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

public class RocksDBLookupTable implements ILookupTable {
private static final Logger logger = LoggerFactory.getLogger(RocksDBLookupTable.class);
static {
RocksDB.loadLibrary();
}

private TableDesc tableDesc;
private RocksDB rocksDB;
private Options options;

private RocksDBLookupRowEncoder rowEncoder;

public RocksDBLookupTable(TableDesc tableDesc, String[] keyColumns, String dbPath) {
this.tableDesc = tableDesc;
this.options = new Options();
this.rowEncoder = new RocksDBLookupRowEncoder(tableDesc, keyColumns);
try {
this.rocksDB = RocksDB.openReadOnly(options, dbPath);
} catch (RocksDBException e) {
throw new RuntimeException("cannot open rocks db in path:" + dbPath, e);
}
}

@Override
public String[] getRow(Array<String> key) {
byte[] encodeKey = rowEncoder.encodeStringsWithLenPfx(key.data, false);
try {
byte[] value = rocksDB.get(encodeKey);
if (value == null) {
return null;
}
return rowEncoder.decode(new KV(encodeKey, value));
} catch (RocksDBException e) {
throw new RuntimeException("error when get key from rocksdb", e);
}
}

@Override
public Iterator<String[]> iterator() {
final RocksIterator rocksIterator = rocksDB.newIterator();
rocksIterator.seekToFirst();

return new Iterator<String[]>() {
int counter;
@Override
public boolean hasNext() {
boolean valid = rocksIterator.isValid();
if (!valid) {
rocksIterator.close();
}
return valid;
}

@Override
public String[] next() {
counter ++;
if (counter % 100000 == 0) {
logger.info("scanned {} rows from rocksDB", counter);
}
String[] result = rowEncoder.decode(new KV(rocksIterator.key(), rocksIterator.value()));
rocksIterator.next();
return result;
}

@Override
public void remove() {
throw new UnsupportedOperationException("not support operation");
}
};
}

@Override
public void close() throws IOException {
options.close();
if (rocksDB != null) {
rocksDB.close();
}
}
}
Loading

0 comments on commit 5a96f8b

Please sign in to comment.