http://git-wip-us.apache.org/repos/asf/kylin/blob/6b6aa313/invertedindex/src/main/java/org/apache/kylin/invertedindex/model/IIDesc.java ---------------------------------------------------------------------- diff --git a/invertedindex/src/main/java/org/apache/kylin/invertedindex/model/IIDesc.java b/invertedindex/src/main/java/org/apache/kylin/invertedindex/model/IIDesc.java deleted file mode 100644 index f10712a..0000000 --- a/invertedindex/src/main/java/org/apache/kylin/invertedindex/model/IIDesc.java +++ /dev/null @@ -1,372 +0,0 @@ -/* - * 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.invertedindex.model; - -import java.security.MessageDigest; -import java.security.NoSuchAlgorithmException; -import java.util.ArrayList; -import java.util.BitSet; -import java.util.HashSet; -import java.util.List; - -import org.apache.commons.net.util.Base64; -import org.apache.kylin.common.KylinConfig; -import org.apache.kylin.common.persistence.ResourceStore; -import org.apache.kylin.common.persistence.RootPersistentEntity; -import org.apache.kylin.common.util.Bytes; -import org.apache.kylin.common.util.JsonUtil; -import org.apache.kylin.common.util.StringUtil; -import org.apache.kylin.metadata.MetadataConstants; -import org.apache.kylin.metadata.MetadataManager; -import org.apache.kylin.metadata.model.ColumnDesc; -import org.apache.kylin.metadata.model.DataModelDesc; -import org.apache.kylin.metadata.model.FunctionDesc; -import org.apache.kylin.metadata.model.MeasureDesc; -import org.apache.kylin.metadata.model.ParameterDesc; -import org.apache.kylin.metadata.model.TableDesc; -import org.apache.kylin.metadata.model.TblColRef; - -import com.fasterxml.jackson.annotation.JsonAutoDetect; -import com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.fasterxml.jackson.core.JsonProcessingException; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Lists; -import com.google.common.collect.Sets; - -/** - * @author yangli9 - */ -@JsonAutoDetect(fieldVisibility = Visibility.NONE, getterVisibility = Visibility.NONE, isGetterVisibility = Visibility.NONE, setterVisibility = Visibility.NONE) -public class IIDesc extends RootPersistentEntity { - - public static final String HBASE_FAMILY = "f"; - public static final String HBASE_QUALIFIER = "c"; - public static final byte[] HBASE_FAMILY_BYTES = Bytes.toBytes(HBASE_FAMILY); - public static final byte[] HBASE_QUALIFIER_BYTES = Bytes.toBytes(HBASE_QUALIFIER); - - private KylinConfig config; - private DataModelDesc model; - - @JsonProperty("name") - private String name; - @JsonProperty("model_name") - private String modelName; - @JsonProperty("timestamp_dimension") - private String timestampDimension; - @JsonProperty("value_dimensions") - private List<IIDimension> valueDimensions; - @JsonProperty("metrics") - private String[] metricNames; - @JsonProperty("sharding") - private short sharding = 1; // parallelism - @JsonProperty("slice_size") - private int sliceSize = 50000; // no. rows - @JsonProperty("signature") - private String signature; - - // computed - private List<TableDesc> allTables = Lists.newArrayList(); - private List<TblColRef> allColumns = Lists.newArrayList(); - private List<TblColRef> allDimensions = Lists.newArrayList(); - private int tsCol; - private int[] valueCols; - private int[] metricsCols; - private BitSet metricsColSet; - private List<MeasureDesc> measureDescs; - - public void init(MetadataManager metadataManager) { - - config = metadataManager.getConfig(); - - if (this.modelName == null || this.modelName.length() == 0) { - throw new RuntimeException("The cubeDesc '" + this.getName() + "' doesn't have data model specified."); - } - - this.model = MetadataManager.getInstance(config).getDataModelDesc(this.modelName); - - if (this.model == null) { - throw new RuntimeException("No data model found with name '" + modelName + "'."); - } - - timestampDimension = timestampDimension.toUpperCase(); - - // capitalize - IIDimension.capicalizeStrings(valueDimensions); - StringUtil.toUpperCaseArray(metricNames, metricNames); - - // retrieve all columns and all tables, and make available measure to ii - HashSet<String> allTableNames = Sets.newHashSet(); - measureDescs = Lists.newArrayList(); - measureDescs.add(makeCountMeasure()); - for (IIDimension iiDimension : valueDimensions) { - TableDesc tableDesc = this.getTableDesc(iiDimension.getTable()); - for (String column : iiDimension.getColumns()) { - ColumnDesc columnDesc = tableDesc.findColumnByName(column); - TblColRef tcr = new TblColRef(columnDesc); - allColumns.add(tcr); - allDimensions.add(tcr); - measureDescs.add(makeHLLMeasure(columnDesc, "hllc10")); - } - - if (!allTableNames.contains(tableDesc.getIdentity())) { - allTableNames.add(tableDesc.getIdentity()); - allTables.add(tableDesc); - } - } - for (String column : metricNames) { - TableDesc tableDesc = this.getTableDesc(this.getFactTableName()); - ColumnDesc columnDesc = tableDesc.findColumnByName(column); - allColumns.add(new TblColRef(columnDesc)); - measureDescs.add(makeNormalMeasure("SUM", columnDesc)); - measureDescs.add(makeNormalMeasure("MIN", columnDesc)); - measureDescs.add(makeNormalMeasure("MAX", columnDesc)); - if (!allTableNames.contains(tableDesc.getIdentity())) { - allTableNames.add(tableDesc.getIdentity()); - allTables.add(tableDesc); - } - } - - // indexing for each type of columns - valueCols = new int[IIDimension.getColumnCount(valueDimensions)]; - metricsCols = new int[metricNames.length]; - metricsColSet = new BitSet(this.getTableDesc(this.getFactTableName()).getColumnCount()); - - int totalIndex = 0; - for (int i = 0; i < valueCols.length; ++i, ++totalIndex) { - valueCols[i] = totalIndex; - } - for (int i = 0; i < metricsCols.length; ++i, ++totalIndex) { - metricsCols[i] = totalIndex; - metricsColSet.set(totalIndex); - } - - // partitioning column - tsCol = -1; - for (int i = 0; i < allColumns.size(); ++i) { - TblColRef col = allColumns.get(i); - - if (col.isSameAs(this.getFactTableName(), this.timestampDimension)) { - tsCol = i; - break; - } - } - if (tsCol < 0) - throw new RuntimeException("timestamp_dimension is not in valueDimensions"); - } - - private TableDesc getTableDesc(String tableName) { - return MetadataManager.getInstance(this.config).getTableDesc(tableName); - } - - public String getResourcePath() { - return getIIDescResourcePath(name); - } - - public static String getIIDescResourcePath(String descName) { - return ResourceStore.II_DESC_RESOURCE_ROOT + "/" + descName + MetadataConstants.FILE_SURFIX; - } - - public List<MeasureDesc> getMeasures() { - return measureDescs; - } - - public List<FunctionDesc> listAllFunctions() { - List<FunctionDesc> functions = new ArrayList<FunctionDesc>(); - for (MeasureDesc m : measureDescs) { - functions.add(m.getFunction()); - } - return functions; - } - - private MeasureDesc makeNormalMeasure(String func, ColumnDesc columnDesc) { - String columnName = columnDesc.getName(); - String returnType = columnDesc.getTypeName(); - MeasureDesc measureDesc = new MeasureDesc(); - FunctionDesc f1 = new FunctionDesc(); - f1.setExpression(func); - ParameterDesc p1 = new ParameterDesc(); - p1.setType("column"); - p1.setValue(columnName); - p1.setColRefs(ImmutableList.of(new TblColRef(columnDesc))); - f1.setParameter(p1); - f1.setReturnType(returnType); - if (f1.isSum() && f1.getReturnDataType().isIntegerFamily()) { - f1.setReturnType("bigint"); - } - - measureDesc.setFunction(f1); - return measureDesc; - } - - /** - * - * @param hllType represents the presision - */ - private MeasureDesc makeHLLMeasure(ColumnDesc columnDesc, String hllType) { - String columnName = columnDesc.getName(); - MeasureDesc measureDesc = new MeasureDesc(); - FunctionDesc f1 = new FunctionDesc(); - f1.setExpression("COUNT_DISTINCT"); - ParameterDesc p1 = new ParameterDesc(); - p1.setType("column"); - p1.setValue(columnName); - p1.setColRefs(ImmutableList.of(new TblColRef(columnDesc))); - f1.setParameter(p1); - f1.setReturnType(hllType); - measureDesc.setFunction(f1); - return measureDesc; - } - - private MeasureDesc makeCountMeasure() { - MeasureDesc measureDesc = new MeasureDesc(); - FunctionDesc f1 = new FunctionDesc(); - f1.setExpression("COUNT"); - ParameterDesc p1 = new ParameterDesc(); - p1.setType("constant"); - p1.setValue("1"); - f1.setParameter(p1); - f1.setReturnType("bigint"); - measureDesc.setFunction(f1); - return measureDesc; - } - - /** - * at first stage the only table in II is fact table, tables - * - * @return - */ - public List<TableDesc> listTables() { - return allTables; - } - - public List<TblColRef> listAllColumns() { - return allColumns; - } - - public List<TblColRef> listAllDimensions() { - return allDimensions; - } - - public TblColRef findColumnRef(String table, String column) { - ColumnDesc columnDesc = this.getTableDesc(table).findColumnByName(column); - return new TblColRef(columnDesc); - } - - public int findColumn(TblColRef col) { - return this.allColumns.indexOf(col); - } - - public KylinConfig getConfig() { - return config; - } - - public String getName() { - return name; - } - - public String getModelName() { - return modelName; - } - - public void setModelName(String modelName) { - this.modelName = modelName; - } - - public DataModelDesc getModel() { - return model; - } - - public void setModel(DataModelDesc model) { - this.model = model; - } - - public int getTimestampColumn() { - return tsCol; - } - - public int[] getValueColumns() { - return valueCols; - } - - public int[] getMetricsColumns() { - return metricsCols; - } - - public short getSharding() { - return sharding; - } - - public int getSliceSize() { - return sliceSize; - } - - public String getSignature() { - return signature; - } - - public void setSignature(String signature) { - this.signature = signature; - } - - public boolean isMetricsCol(TblColRef col) { - if (!col.getTable().equalsIgnoreCase(this.getFactTableName())) - return false; - return isMetricsCol(this.findColumn(col)); - } - - public boolean isMetricsCol(int index) { - return metricsColSet.get(index); - } - - /** - * the returned fact table name is guaranteed to be in the form of db.table - * - * @return - */ - public String getFactTableName() { - return this.model.getFactTable().toUpperCase(); - } - - public String getTimestampDimension() { - return timestampDimension; - } - - public void setName(String name) { - this.name = name; - } - - public String calculateSignature() { - MessageDigest md = null; - try { - md = MessageDigest.getInstance("MD5"); - StringBuilder sigString = new StringBuilder(); - sigString.append(this.name).append("|").append(this.getFactTableName()).append("|").append(timestampDimension).append("|").append("|").append(JsonUtil.writeValueAsString(valueDimensions)).append("|").append(JsonUtil.writeValueAsString(this.metricNames)).append("|").append(sharding).append("|").append(sliceSize); - - byte[] signature = md.digest(sigString.toString().getBytes()); - return new String(Base64.encodeBase64(signature)); - } catch (NoSuchAlgorithmException e) { - throw new RuntimeException("Failed to calculate signature"); - } catch (JsonProcessingException e) { - throw new RuntimeException("Failed to calculate signature"); - } - - } - -}
http://git-wip-us.apache.org/repos/asf/kylin/blob/6b6aa313/invertedindex/src/main/java/org/apache/kylin/invertedindex/model/IIDimension.java ---------------------------------------------------------------------- diff --git a/invertedindex/src/main/java/org/apache/kylin/invertedindex/model/IIDimension.java b/invertedindex/src/main/java/org/apache/kylin/invertedindex/model/IIDimension.java deleted file mode 100644 index 56e3692..0000000 --- a/invertedindex/src/main/java/org/apache/kylin/invertedindex/model/IIDimension.java +++ /dev/null @@ -1,71 +0,0 @@ -/* - * 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.invertedindex.model; - -import java.util.List; - -import org.apache.kylin.common.util.StringUtil; - -import com.fasterxml.jackson.annotation.JsonAutoDetect; -import com.fasterxml.jackson.annotation.JsonProperty; - -/** - * Created by Hongbin Ma(Binmahone) on 12/26/14. - */ -@JsonAutoDetect(fieldVisibility = JsonAutoDetect.Visibility.NONE, getterVisibility = JsonAutoDetect.Visibility.NONE, isGetterVisibility = JsonAutoDetect.Visibility.NONE, setterVisibility = JsonAutoDetect.Visibility.NONE) -public class IIDimension { - @JsonProperty("table") - private String table; - @JsonProperty("columns") - private String[] columns; - - public String getTable() { - return table; - } - - public void setTable(String table) { - this.table = table; - } - - public String[] getColumns() { - return columns; - } - - public void setColumns(String[] columns) { - this.columns = columns; - } - - public static void capicalizeStrings(List<IIDimension> dimensions) { - if (dimensions != null) { - for (IIDimension iiDimension : dimensions) { - iiDimension.setTable(iiDimension.getTable().toUpperCase()); - StringUtil.toUpperCaseArray(iiDimension.getColumns(), iiDimension.getColumns()); - } - } - } - - public static int getColumnCount(List<IIDimension> iiDimensions) { - int count = 0; - for (IIDimension iiDimension : iiDimensions) { - count += iiDimension.getColumns().length; - } - return count; - } - -} http://git-wip-us.apache.org/repos/asf/kylin/blob/6b6aa313/invertedindex/src/main/java/org/apache/kylin/invertedindex/model/IIKeyValueCodec.java ---------------------------------------------------------------------- diff --git a/invertedindex/src/main/java/org/apache/kylin/invertedindex/model/IIKeyValueCodec.java b/invertedindex/src/main/java/org/apache/kylin/invertedindex/model/IIKeyValueCodec.java deleted file mode 100644 index df24703..0000000 --- a/invertedindex/src/main/java/org/apache/kylin/invertedindex/model/IIKeyValueCodec.java +++ /dev/null @@ -1,253 +0,0 @@ -/* - * 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.invertedindex.model; - -import java.util.ArrayList; -import java.util.Collection; -import java.util.Iterator; -import java.util.List; - -import org.apache.hadoop.hbase.io.ImmutableBytesWritable; -import org.apache.kylin.common.util.BytesUtil; -import org.apache.kylin.common.util.Pair; -import org.apache.kylin.invertedindex.index.BitMapContainer; -import org.apache.kylin.invertedindex.index.ColumnValueContainer; -import org.apache.kylin.invertedindex.index.CompressedValueContainer; -import org.apache.kylin.invertedindex.index.Slice; -import org.apache.kylin.invertedindex.index.TableRecordInfoDigest; - -import com.google.common.collect.Lists; - -/** - * @author yangli9 - */ -public class IIKeyValueCodec { - - public static final int SHARD_LEN = 2; - public static final int TIMEPART_LEN = 8; - public static final int COLNO_LEN = 2; - - private TableRecordInfoDigest infoDigest; - - public IIKeyValueCodec(TableRecordInfoDigest digest) { - this.infoDigest = digest; - } - - public Collection<Pair<ImmutableBytesWritable, ImmutableBytesWritable>> encodeKeyValue(Slice slice) { - ArrayList<Pair<ImmutableBytesWritable, ImmutableBytesWritable>> result = Lists.newArrayList(); - ColumnValueContainer[] containers = slice.getColumnValueContainers(); - for (int col = 0; col < containers.length; col++) { - if (containers[col] instanceof BitMapContainer) { - collectKeyValues(slice, col, (BitMapContainer) containers[col], result); - } else if (containers[col] instanceof CompressedValueContainer) { - collectKeyValues(slice, col, (CompressedValueContainer) containers[col], result); - } else { - throw new IllegalArgumentException("Unkown container class " + containers[col].getClass()); - } - } - return result; - } - - private void collectKeyValues(Slice slice, int col, CompressedValueContainer container, // - ArrayList<Pair<ImmutableBytesWritable, ImmutableBytesWritable>> result) { - ImmutableBytesWritable key = encodeKey(slice.getShard(), slice.getTimestamp(), col, -1); - ImmutableBytesWritable value = container.toBytes(); - result.add(new Pair<ImmutableBytesWritable, ImmutableBytesWritable>(key, value)); - } - - private void collectKeyValues(Slice slice, int col, BitMapContainer container, // - ArrayList<Pair<ImmutableBytesWritable, ImmutableBytesWritable>> result) { - List<ImmutableBytesWritable> values = container.toBytes(); - for (int v = 0; v < values.size(); v++) { - ImmutableBytesWritable key = encodeKey(slice.getShard(), slice.getTimestamp(), col, v); - result.add(new Pair<ImmutableBytesWritable, ImmutableBytesWritable>(key, values.get(v))); - } - } - - ImmutableBytesWritable encodeKey(short shard, long timestamp, int col, int colValue) { - byte[] bytes = new byte[20]; - int len = encodeKey(shard, timestamp, col, colValue, bytes, 0); - return new ImmutableBytesWritable(bytes, 0, len); - } - - int encodeKey(short shard, long timestamp, int col, int colValue, byte[] buf, int offset) { - int i = offset; - - BytesUtil.writeUnsigned(shard, buf, i, SHARD_LEN); - i += SHARD_LEN; - BytesUtil.writeLong(timestamp, buf, i, TIMEPART_LEN); - i += TIMEPART_LEN; - - BytesUtil.writeUnsigned(col, buf, i, COLNO_LEN); - i += COLNO_LEN; - - if (colValue >= 0) { - int colLen = infoDigest.length(col); - BytesUtil.writeUnsigned(colValue, buf, i, colLen); - i += colLen; - } - - return i - offset; - } - - public Iterable<Slice> decodeKeyValue(Iterable<Pair<ImmutableBytesWritable, ImmutableBytesWritable>> kvs) { - return new Decoder(infoDigest, kvs); - } - - private static class Decoder implements Iterable<Slice> { - - TableRecordInfoDigest info; - Iterator<Pair<ImmutableBytesWritable, ImmutableBytesWritable>> iterator; - - Slice next = null; - short curShard = Short.MIN_VALUE; - long curSliceTimestamp = Long.MIN_VALUE; - int curCol = -1; - int curColValue = -1; - short lastShard = Short.MIN_VALUE; - long lastSliceTimestamp = Long.MIN_VALUE; - int lastCol = -1; - ColumnValueContainer[] containers = null; - List<ImmutableBytesWritable> bitMapValues = Lists.newArrayList(); - - Decoder(TableRecordInfoDigest info, Iterable<Pair<ImmutableBytesWritable, ImmutableBytesWritable>> kvs) { - this.info = info; - this.iterator = kvs.iterator(); - } - - private void goToNext() { - if (next != null) { // was not fetched - return; - } - - // NOTE the input keys are ordered - while (next == null && iterator.hasNext()) { - Pair<ImmutableBytesWritable, ImmutableBytesWritable> kv = iterator.next(); - ImmutableBytesWritable k = kv.getFirst(); - ImmutableBytesWritable v = kv.getSecond(); - decodeKey(k); - - if (curShard != lastShard || curSliceTimestamp != lastSliceTimestamp) { - makeNext(); - } - consumeCurrent(v); - } - if (next == null) { - makeNext(); - } - } - - private void decodeKey(ImmutableBytesWritable k) { - byte[] buf = k.get(); - int i = k.getOffset(); - - curShard = (short) BytesUtil.readUnsigned(buf, i, SHARD_LEN); - i += SHARD_LEN; - curSliceTimestamp = BytesUtil.readLong(buf, i, TIMEPART_LEN); - i += TIMEPART_LEN; - - curCol = BytesUtil.readUnsigned(buf, i, COLNO_LEN); - i += COLNO_LEN; - - if (i - k.getOffset() < k.getLength()) { - // bitmap - int colLen = info.length(curCol); - curColValue = BytesUtil.readUnsigned(buf, i, colLen); - i += colLen; - } else { - // value list - curColValue = -1; - } - } - - private void consumeCurrent(ImmutableBytesWritable v) { - if (curCol != lastCol && bitMapValues.size() > 0) { // end of a - // bitmap - // container - addBitMapContainer(lastCol); - } - if (curColValue < 0) { - CompressedValueContainer c = new CompressedValueContainer(info, curCol, 0); - c.fromBytes(v); - addContainer(curCol, c); - } else { - assert curColValue == bitMapValues.size(); - // make a copy, the value object from caller is typically reused - // through iteration - bitMapValues.add(new ImmutableBytesWritable(v)); - } - - lastShard = curShard; - lastSliceTimestamp = curSliceTimestamp; - lastCol = curCol; - } - - private void makeNext() { - if (bitMapValues.isEmpty() == false) { - addBitMapContainer(lastCol); - } - if (containers != null) { - next = new Slice(info, lastShard, lastSliceTimestamp, containers); - } - lastSliceTimestamp = Long.MIN_VALUE; - lastCol = -1; - containers = null; - bitMapValues.clear(); - } - - private void addBitMapContainer(int col) { - BitMapContainer c = new BitMapContainer(info, col); - c.fromBytes(bitMapValues); - addContainer(col, c); - bitMapValues.clear(); - } - - private void addContainer(int col, ColumnValueContainer c) { - if (containers == null) { - containers = new ColumnValueContainer[info.getColumnCount()]; - } - containers[col] = c; - } - - @Override - public Iterator<Slice> iterator() { - return new Iterator<Slice>() { - @Override - public boolean hasNext() { - goToNext(); - return next != null; - } - - @Override - public Slice next() { - Slice result = next; - next = null; - return result; - } - - @Override - public void remove() { - throw new UnsupportedOperationException(); - } - }; - } - - } - -} http://git-wip-us.apache.org/repos/asf/kylin/blob/6b6aa313/invertedindex/src/main/java/org/apache/kylin/invertedindex/tools/IICLI.java ---------------------------------------------------------------------- diff --git a/invertedindex/src/main/java/org/apache/kylin/invertedindex/tools/IICLI.java b/invertedindex/src/main/java/org/apache/kylin/invertedindex/tools/IICLI.java deleted file mode 100644 index bfcc7d5..0000000 --- a/invertedindex/src/main/java/org/apache/kylin/invertedindex/tools/IICLI.java +++ /dev/null @@ -1,106 +0,0 @@ -/* - * 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.invertedindex.tools; - -import java.io.IOException; -import java.util.Iterator; - -import org.apache.commons.io.IOUtils; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.io.ImmutableBytesWritable; -import org.apache.hadoop.io.SequenceFile; -import org.apache.hadoop.io.SequenceFile.Reader; -import org.apache.kylin.common.KylinConfig; -import org.apache.kylin.common.util.HadoopUtil; -import org.apache.kylin.common.util.Pair; -import org.apache.kylin.invertedindex.IIInstance; -import org.apache.kylin.invertedindex.IIManager; -import org.apache.kylin.invertedindex.index.RawTableRecord; -import org.apache.kylin.invertedindex.index.Slice; -import org.apache.kylin.invertedindex.index.TableRecord; -import org.apache.kylin.invertedindex.index.TableRecordInfo; -import org.apache.kylin.invertedindex.model.IIKeyValueCodec; - -/** - * @author yangli9 - */ -public class IICLI { - - public static void main(String[] args) throws IOException { - Configuration hconf = HadoopUtil.getCurrentConfiguration(); - IIManager mgr = IIManager.getInstance(KylinConfig.getInstanceFromEnv()); - - String iiName = args[0]; - IIInstance ii = mgr.getII(iiName); - - String path = args[1]; - System.out.println("Reading from " + path + " ..."); - - TableRecordInfo info = new TableRecordInfo(ii.getFirstSegment()); - IIKeyValueCodec codec = new IIKeyValueCodec(info.getDigest()); - int count = 0; - for (Slice slice : codec.decodeKeyValue(readSequenceKVs(hconf, path))) { - for (RawTableRecord rec : slice) { - System.out.printf(new TableRecord(rec, info).toString()); - count++; - } - } - System.out.println("Total " + count + " records"); - } - - public static Iterable<Pair<ImmutableBytesWritable, ImmutableBytesWritable>> readSequenceKVs(Configuration hconf, String path) throws IOException { - final Reader reader = new Reader(hconf, SequenceFile.Reader.file(new Path(path))); - return new Iterable<Pair<ImmutableBytesWritable, ImmutableBytesWritable>>() { - @Override - public Iterator<Pair<ImmutableBytesWritable, ImmutableBytesWritable>> iterator() { - return new Iterator<Pair<ImmutableBytesWritable, ImmutableBytesWritable>>() { - ImmutableBytesWritable k = new ImmutableBytesWritable(); - ImmutableBytesWritable v = new ImmutableBytesWritable(); - Pair<ImmutableBytesWritable, ImmutableBytesWritable> pair = new Pair<ImmutableBytesWritable, ImmutableBytesWritable>(k, v); - - @Override - public boolean hasNext() { - boolean hasNext = false; - try { - hasNext = reader.next(k, v); - } catch (IOException e) { - throw new RuntimeException(e); - } finally { - if (hasNext == false) { - IOUtils.closeQuietly(reader); - } - } - return hasNext; - } - - @Override - public Pair<ImmutableBytesWritable, ImmutableBytesWritable> next() { - return pair; - } - - @Override - public void remove() { - throw new UnsupportedOperationException(); - } - }; - } - }; - } -} http://git-wip-us.apache.org/repos/asf/kylin/blob/6b6aa313/invertedindex/src/test/java/org/apache/kylin/invertedindex/invertedindex/IIDescManagerTest.java ---------------------------------------------------------------------- diff --git a/invertedindex/src/test/java/org/apache/kylin/invertedindex/invertedindex/IIDescManagerTest.java b/invertedindex/src/test/java/org/apache/kylin/invertedindex/invertedindex/IIDescManagerTest.java deleted file mode 100644 index 87723fd..0000000 --- a/invertedindex/src/test/java/org/apache/kylin/invertedindex/invertedindex/IIDescManagerTest.java +++ /dev/null @@ -1,105 +0,0 @@ -/* - * 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.invertedindex.invertedindex; - -import java.io.IOException; -import java.util.UUID; - -import org.apache.kylin.common.util.LocalFileMetadataTestCase; -import org.apache.kylin.invertedindex.IIDescManager; -import org.apache.kylin.invertedindex.model.IIDesc; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; - -/** - * Created by shaoshi on 1/30/15. - */ -public class IIDescManagerTest extends LocalFileMetadataTestCase { - - public static final String TEST_II_DESC_NAME = "test_kylin_ii_desc"; - - @Before - public void setup() { - createTestMetadata(); - } - - @After - public void clean() { - this.cleanupTestMetadata(); - } - - @Test - public void testCRUD() throws IOException { - IIDescManager mgr = IIDescManager.getInstance(getTestConfig()); - - String newDescName = "Copy_of_" + TEST_II_DESC_NAME; - - try { - IIDesc testRecord = mgr.getIIDesc(newDescName); - if (testRecord != null) - mgr.removeIIDesc(testRecord); - } catch (IOException e) { - // just ensure the old one is removed - } - - Assert.assertNull(mgr.getIIDesc(newDescName)); - IIDesc desc = mgr.getIIDesc(TEST_II_DESC_NAME); - - desc.setName(newDescName); - desc.setLastModified(0); - - mgr.createIIDesc(desc); - - desc = mgr.getIIDesc(newDescName); - - Assert.assertNotNull(desc); - - mgr.updateIIDesc(desc); // this will trigger cache wipe; please ignore the HTTP error in logs. - - mgr.removeIIDesc(desc); - - Assert.assertNull(mgr.getIIDesc(newDescName)); - - } - - @Test - public void testReload() throws IOException { - IIDescManager mgr = IIDescManager.getInstance(getTestConfig()); - - IIDesc desc = mgr.getIIDesc(TEST_II_DESC_NAME); - - // do some modification - desc.setUuid(UUID.randomUUID().toString()); - - IIDesc newDesc = mgr.getIIDesc(TEST_II_DESC_NAME); - - Assert.assertEquals(desc, newDesc); - - // reload the cache - mgr.reloadIIDesc(TEST_II_DESC_NAME); - - newDesc = mgr.getIIDesc(TEST_II_DESC_NAME); - - Assert.assertNotEquals(desc, newDesc); - - } - -} http://git-wip-us.apache.org/repos/asf/kylin/blob/6b6aa313/invertedindex/src/test/java/org/apache/kylin/invertedindex/invertedindex/IIDescTest.java ---------------------------------------------------------------------- diff --git a/invertedindex/src/test/java/org/apache/kylin/invertedindex/invertedindex/IIDescTest.java b/invertedindex/src/test/java/org/apache/kylin/invertedindex/invertedindex/IIDescTest.java deleted file mode 100644 index d5e29ec..0000000 --- a/invertedindex/src/test/java/org/apache/kylin/invertedindex/invertedindex/IIDescTest.java +++ /dev/null @@ -1,69 +0,0 @@ -/* - * 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.invertedindex.invertedindex; - -import java.io.IOException; - -import org.apache.kylin.common.util.JsonUtil; -import org.apache.kylin.common.util.LocalFileMetadataTestCase; -import org.apache.kylin.invertedindex.IIDescManager; -import org.apache.kylin.invertedindex.model.IIDesc; -import org.apache.kylin.metadata.model.DataModelDesc; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; - -/** - * Created by shaoshi on 1/30/15. - */ -public class IIDescTest extends LocalFileMetadataTestCase { - - @Before - public void setup() { - this.createTestMetadata(); - - } - - @After - public void clear() { - this.cleanupTestMetadata(); - } - - @Test - public void testGetIIDesc() { - - IIDesc iiDesc = IIDescManager.getInstance(getTestConfig()).getIIDesc("test_kylin_ii_desc"); - DataModelDesc model = iiDesc.getModel(); - Assert.assertNotNull(iiDesc); - Assert.assertNotNull(model); - - } - - @Test - public void testSerialization() throws IOException { - IIDesc iiDesc = IIDescManager.getInstance(getTestConfig()).getIIDesc("test_kylin_ii_desc"); - String str = JsonUtil.writeValueAsIndentString(iiDesc); - System.out.println(str); - @SuppressWarnings("unused") - IIDesc desc2 = JsonUtil.readValue(str, IIDesc.class); - - Assert.assertEquals(iiDesc, desc2); - } -} http://git-wip-us.apache.org/repos/asf/kylin/blob/6b6aa313/invertedindex/src/test/java/org/apache/kylin/invertedindex/invertedindex/IIInstanceTest.java ---------------------------------------------------------------------- diff --git a/invertedindex/src/test/java/org/apache/kylin/invertedindex/invertedindex/IIInstanceTest.java b/invertedindex/src/test/java/org/apache/kylin/invertedindex/invertedindex/IIInstanceTest.java deleted file mode 100644 index 50af8a4..0000000 --- a/invertedindex/src/test/java/org/apache/kylin/invertedindex/invertedindex/IIInstanceTest.java +++ /dev/null @@ -1,80 +0,0 @@ -/* - * 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.invertedindex.invertedindex; - -import java.io.IOException; -import java.util.List; - -import org.apache.kylin.common.util.LocalFileMetadataTestCase; -import org.apache.kylin.common.util.Dictionary; -import org.apache.kylin.invertedindex.IIDescManager; -import org.apache.kylin.invertedindex.IIInstance; -import org.apache.kylin.invertedindex.IIManager; -import org.apache.kylin.invertedindex.model.IIDesc; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; - -/** - * Created by shaoshi on 2/5/15. - */ -public class IIInstanceTest extends LocalFileMetadataTestCase { - @Before - public void setup() { - createTestMetadata(); - } - - @After - public void clean() { - cleanupTestMetadata(); - } - - @Test - public void testGetIIsByDesc() throws IOException { - IIManager mgr = IIManager.getInstance(getTestConfig()); - - List<IIInstance> iiInstances = mgr.getIIsByDesc("test_kylin_ii_desc"); - - Assert.assertTrue(iiInstances.size() > 0); - - IIInstance instance = iiInstances.get(0); - - Dictionary dict = mgr.getDictionary(instance.getFirstSegment(), instance.getDescriptor().findColumnRef("DEFAULT.TEST_KYLIN_FACT", "LSTG_SITE_ID")); - - Assert.assertNotNull(dict); - } - - @Test - public void testCreateIIInstance() throws IOException { - - IIDesc iiDesc = IIDescManager.getInstance(getTestConfig()).getIIDesc("test_kylin_ii_desc"); - - IIInstance ii = IIInstance.create("new_ii", "default", iiDesc); - - IIManager iiMgr = IIManager.getInstance(getTestConfig()); - - List<IIInstance> allIIList = iiMgr.listAllIIs(); - - iiMgr.createII(ii); - - Assert.assertNotNull(iiMgr.getII("new_ii")); - } - -} http://git-wip-us.apache.org/repos/asf/kylin/blob/6b6aa313/invertedindex/src/test/java/org/apache/kylin/invertedindex/invertedindex/InvertedIndexLocalTest.java ---------------------------------------------------------------------- diff --git a/invertedindex/src/test/java/org/apache/kylin/invertedindex/invertedindex/InvertedIndexLocalTest.java b/invertedindex/src/test/java/org/apache/kylin/invertedindex/invertedindex/InvertedIndexLocalTest.java deleted file mode 100644 index cfa4ba6..0000000 --- a/invertedindex/src/test/java/org/apache/kylin/invertedindex/invertedindex/InvertedIndexLocalTest.java +++ /dev/null @@ -1,232 +0,0 @@ -/* - * 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.invertedindex.invertedindex; - -import static org.junit.Assert.assertEquals; - -import java.io.File; -import java.io.FileInputStream; -import java.io.IOException; -import java.util.Collections; -import java.util.Comparator; -import java.util.HashSet; -import java.util.List; - -import org.apache.commons.io.IOUtils; -import org.apache.hadoop.hbase.io.ImmutableBytesWritable; -import org.apache.kylin.common.util.BytesUtil; -import org.apache.kylin.common.util.LocalFileMetadataTestCase; -import org.apache.kylin.common.util.Pair; -import org.apache.kylin.common.util.Dictionary; -import org.apache.kylin.invertedindex.IIInstance; -import org.apache.kylin.invertedindex.IIManager; -import org.apache.kylin.invertedindex.index.BitMapContainer; -import org.apache.kylin.invertedindex.index.CompressedValueContainer; -import org.apache.kylin.invertedindex.index.RawTableRecord; -import org.apache.kylin.invertedindex.index.ShardingSliceBuilder; -import org.apache.kylin.invertedindex.index.Slice; -import org.apache.kylin.invertedindex.index.TableRecord; -import org.apache.kylin.invertedindex.index.TableRecordInfo; -import org.apache.kylin.invertedindex.model.IIKeyValueCodec; -import org.junit.After; -import org.junit.Before; -import org.junit.Ignore; -import org.junit.Test; - -import com.google.common.collect.Lists; - -public class InvertedIndexLocalTest extends LocalFileMetadataTestCase { - - IIInstance ii; - TableRecordInfo info; - - @Before - public void setUp() throws Exception { - this.createTestMetadata(); - this.ii = IIManager.getInstance(getTestConfig()).getII("test_kylin_ii"); - this.info = new TableRecordInfo(ii.getFirstSegment()); - } - - @After - public void after() throws Exception { - this.cleanupTestMetadata(); - } - - @Test - @Ignore - public void testBitMapContainer() { - // create container - BitMapContainer container = new BitMapContainer(info.getDigest(), 0); - Dictionary<String> dict = info.dict(0); - for (int v = dict.getMinId(); v <= dict.getMaxId(); v++) { - container.append(v); - } - container.append(Dictionary.NULL_ID[dict.getSizeOfId()]); - container.closeForChange(); - - // copy by serialization - List<ImmutableBytesWritable> bytes = container.toBytes(); - BitMapContainer container2 = new BitMapContainer(info.getDigest(), 0); - container2.fromBytes(bytes); - - // check the copy - int i = 0; - for (int v = dict.getMinId(); v <= dict.getMaxId(); v++) { - int value = container2.getValueIntAt(i++); - assertEquals(v, value); - } - assertEquals(Dictionary.NULL_ID[dict.getSizeOfId()], container2.getValueIntAt(i++)); - assertEquals(container, container2); - } - - @Test - public void testCompressedValueContainer() { - // create container - CompressedValueContainer container = new CompressedValueContainer(info.getDigest(), 0, 500); - Dictionary<String> dict = info.dict(0); - - byte[] buf = new byte[dict.getSizeOfId()]; - ImmutableBytesWritable bytes = new ImmutableBytesWritable(buf); - - for (int v = dict.getMinId(); v <= dict.getMaxId(); v++) { - BytesUtil.writeUnsigned(v, buf, 0, dict.getSizeOfId()); - container.append(bytes); - } - BytesUtil.writeUnsigned(Dictionary.NULL_ID[dict.getSizeOfId()], buf, 0, dict.getSizeOfId()); - container.append(bytes); - container.closeForChange(); - - // copy by serialization - ImmutableBytesWritable copy = container.toBytes(); - CompressedValueContainer container2 = new CompressedValueContainer(info.getDigest(), 0, 500); - container2.fromBytes(copy); - - // check the copy - int i = 0; - for (int v = dict.getMinId(); v <= dict.getMaxId(); v++) { - container2.getValueAt(i++, bytes); - int value = BytesUtil.readUnsigned(bytes.get(), bytes.getOffset(), bytes.getLength()); - assertEquals(v, value); - } - container2.getValueAt(i++, bytes); - int value = BytesUtil.readUnsigned(bytes.get(), bytes.getOffset(), bytes.getLength()); - assertEquals(Dictionary.NULL_ID[dict.getSizeOfId()], value); - assertEquals(container, container2); - } - - @Test - public void testCodec() throws IOException { - List<TableRecord> records = loadRecordsSorted(); - System.out.println(records.size() + " records"); - List<Slice> slices = buildTimeSlices(records); - System.out.println(slices.size() + " slices"); - - IIKeyValueCodec codec = new IIKeyValueCodec(info.getDigest()); - List<Pair<ImmutableBytesWritable, ImmutableBytesWritable>> kvs = encodeKVs(codec, slices); - System.out.println(kvs.size() + " KV pairs"); - - List<Slice> slicesCopy = decodeKVs(codec, kvs); - assertEquals(slices, slicesCopy); - - List<TableRecord> recordsCopy = iterateRecords(slicesCopy); - assertEquals(new HashSet<TableRecord>(records), new HashSet<TableRecord>(recordsCopy)); - dump(recordsCopy); - } - - private List<TableRecord> loadRecordsSorted() throws IOException { - File file = new File(LOCALMETA_TEST_DATA, "data/flatten_data_for_ii.csv"); - FileInputStream in = new FileInputStream(file); - List<String> lines = IOUtils.readLines(in, "UTF-8"); - in.close(); - - List<TableRecord> records = Lists.newArrayList(); - for (String line : lines) { - String[] fields = line.split(","); - TableRecord rec = info.createTableRecord(); - for (int col = 0; col < fields.length; col++) { - rec.setValueString(col, fields[col]); - } - records.add(rec); - } - - Collections.sort(records, new Comparator<TableRecord>() { - @Override - public int compare(TableRecord a, TableRecord b) { - long x = a.getTimestamp() - b.getTimestamp(); - if (x > 0) - return 1; - else if (x == 0) - return 0; - else - return -1; - } - }); - - return records; - } - - private List<Slice> buildTimeSlices(List<TableRecord> records) throws IOException { - ShardingSliceBuilder builder = new ShardingSliceBuilder(info); - List<Slice> slices = Lists.newArrayList(); - for (TableRecord rec : records) { - Slice slice = builder.append(rec); - if (slice != null) - slices.add(slice); - } - List<Slice> finals = builder.close(); - slices.addAll(finals); - - Collections.sort(slices); - return slices; - } - - private List<Pair<ImmutableBytesWritable, ImmutableBytesWritable>> encodeKVs(IIKeyValueCodec codec, List<Slice> slices) { - - List<Pair<ImmutableBytesWritable, ImmutableBytesWritable>> kvs = Lists.newArrayList(); - for (Slice slice : slices) { - kvs.addAll(codec.encodeKeyValue(slice)); - } - return kvs; - } - - private List<Slice> decodeKVs(IIKeyValueCodec codec, List<Pair<ImmutableBytesWritable, ImmutableBytesWritable>> kvs) { - List<Slice> slices = Lists.newArrayList(); - for (Slice slice : codec.decodeKeyValue(kvs)) { - slices.add(slice); - } - return slices; - } - - private List<TableRecord> iterateRecords(List<Slice> slices) { - List<TableRecord> records = Lists.newArrayList(); - for (Slice slice : slices) { - for (RawTableRecord rec : slice) { - records.add(new TableRecord((RawTableRecord) rec.clone(), info)); - } - } - return records; - } - - private void dump(Iterable<TableRecord> records) { - for (TableRecord rec : records) { - System.out.println(rec.toString()); - } - } - -} http://git-wip-us.apache.org/repos/asf/kylin/blob/6b6aa313/invertedindex/src/test/java/org/apache/kylin/invertedindex/invertedindex/LZFTest.java ---------------------------------------------------------------------- diff --git a/invertedindex/src/test/java/org/apache/kylin/invertedindex/invertedindex/LZFTest.java b/invertedindex/src/test/java/org/apache/kylin/invertedindex/invertedindex/LZFTest.java deleted file mode 100644 index d900f55..0000000 --- a/invertedindex/src/test/java/org/apache/kylin/invertedindex/invertedindex/LZFTest.java +++ /dev/null @@ -1,50 +0,0 @@ -/* - * 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.invertedindex.invertedindex; - -import java.io.IOException; - -import org.apache.hadoop.hbase.io.ImmutableBytesWritable; -import org.junit.Test; - -import com.ning.compress.lzf.LZFDecoder; -import com.ning.compress.lzf.LZFEncoder; - -/** - * Created by Hongbin Ma(Binmahone) on 2/6/15. - */ -public class LZFTest { - @Test - public void test() throws IOException { - - byte[] raw = new byte[] { 1, 2, 3, 3, 2, 23 }; - byte[] data = LZFEncoder.encode(raw); - - byte[] data2 = new byte[data.length * 2]; - java.lang.System.arraycopy(data, 0, data2, 0, data.length); - ImmutableBytesWritable bytes = new ImmutableBytesWritable(); - bytes.set(data2, 0, data.length); - - try { - byte[] uncompressed = LZFDecoder.decode(bytes.get(), bytes.getOffset(), bytes.getLength()); - } catch (IOException e) { - throw new RuntimeException("LZF decode failure", e); - } - } -} http://git-wip-us.apache.org/repos/asf/kylin/blob/6b6aa313/jdbc/.settings/org.eclipse.core.resources.prefs ---------------------------------------------------------------------- diff --git a/jdbc/.settings/org.eclipse.core.resources.prefs b/jdbc/.settings/org.eclipse.core.resources.prefs deleted file mode 100644 index 365bbd6..0000000 --- a/jdbc/.settings/org.eclipse.core.resources.prefs +++ /dev/null @@ -1,5 +0,0 @@ -eclipse.preferences.version=1 -encoding//src/main/java=UTF-8 -encoding//src/main/resources=UTF-8 -encoding//src/test/java=UTF-8 -encoding/<project>=UTF-8 http://git-wip-us.apache.org/repos/asf/kylin/blob/6b6aa313/jdbc/.settings/org.eclipse.jdt.core.prefs ---------------------------------------------------------------------- diff --git a/jdbc/.settings/org.eclipse.jdt.core.prefs b/jdbc/.settings/org.eclipse.jdt.core.prefs deleted file mode 100644 index a903301..0000000 --- a/jdbc/.settings/org.eclipse.jdt.core.prefs +++ /dev/null @@ -1,379 +0,0 @@ -eclipse.preferences.version=1 -org.eclipse.jdt.core.compiler.annotation.inheritNullAnnotations=disabled -org.eclipse.jdt.core.compiler.annotation.missingNonNullByDefaultAnnotation=ignore -org.eclipse.jdt.core.compiler.annotation.nonnull=org.eclipse.jdt.annotation.NonNull -org.eclipse.jdt.core.compiler.annotation.nonnullbydefault=org.eclipse.jdt.annotation.NonNullByDefault -org.eclipse.jdt.core.compiler.annotation.nullable=org.eclipse.jdt.annotation.Nullable -org.eclipse.jdt.core.compiler.annotation.nullanalysis=disabled -org.eclipse.jdt.core.compiler.codegen.inlineJsrBytecode=enabled -org.eclipse.jdt.core.compiler.codegen.targetPlatform=1.7 -org.eclipse.jdt.core.compiler.codegen.unusedLocal=preserve -org.eclipse.jdt.core.compiler.compliance=1.7 -org.eclipse.jdt.core.compiler.debug.lineNumber=generate -org.eclipse.jdt.core.compiler.debug.localVariable=generate -org.eclipse.jdt.core.compiler.debug.sourceFile=generate -org.eclipse.jdt.core.compiler.problem.annotationSuperInterface=warning -org.eclipse.jdt.core.compiler.problem.assertIdentifier=error -org.eclipse.jdt.core.compiler.problem.autoboxing=ignore -org.eclipse.jdt.core.compiler.problem.comparingIdentical=warning -org.eclipse.jdt.core.compiler.problem.deadCode=warning -org.eclipse.jdt.core.compiler.problem.deprecation=warning -org.eclipse.jdt.core.compiler.problem.deprecationInDeprecatedCode=disabled -org.eclipse.jdt.core.compiler.problem.deprecationWhenOverridingDeprecatedMethod=disabled -org.eclipse.jdt.core.compiler.problem.discouragedReference=warning -org.eclipse.jdt.core.compiler.problem.emptyStatement=ignore -org.eclipse.jdt.core.compiler.problem.enumIdentifier=error -org.eclipse.jdt.core.compiler.problem.explicitlyClosedAutoCloseable=ignore -org.eclipse.jdt.core.compiler.problem.fallthroughCase=ignore -org.eclipse.jdt.core.compiler.problem.fatalOptionalError=disabled -org.eclipse.jdt.core.compiler.problem.fieldHiding=ignore -org.eclipse.jdt.core.compiler.problem.finalParameterBound=warning -org.eclipse.jdt.core.compiler.problem.finallyBlockNotCompletingNormally=warning -org.eclipse.jdt.core.compiler.problem.forbiddenReference=warning -org.eclipse.jdt.core.compiler.problem.hiddenCatchBlock=warning -org.eclipse.jdt.core.compiler.problem.includeNullInfoFromAsserts=disabled -org.eclipse.jdt.core.compiler.problem.incompatibleNonInheritedInterfaceMethod=warning -org.eclipse.jdt.core.compiler.problem.incompleteEnumSwitch=warning -org.eclipse.jdt.core.compiler.problem.indirectStaticAccess=ignore -org.eclipse.jdt.core.compiler.problem.localVariableHiding=ignore -org.eclipse.jdt.core.compiler.problem.methodWithConstructorName=warning -org.eclipse.jdt.core.compiler.problem.missingDefaultCase=ignore -org.eclipse.jdt.core.compiler.problem.missingDeprecatedAnnotation=ignore -org.eclipse.jdt.core.compiler.problem.missingEnumCaseDespiteDefault=disabled -org.eclipse.jdt.core.compiler.problem.missingHashCodeMethod=ignore -org.eclipse.jdt.core.compiler.problem.missingOverrideAnnotation=ignore -org.eclipse.jdt.core.compiler.problem.missingOverrideAnnotationForInterfaceMethodImplementation=enabled -org.eclipse.jdt.core.compiler.problem.missingSerialVersion=warning -org.eclipse.jdt.core.compiler.problem.missingSynchronizedOnInheritedMethod=ignore -org.eclipse.jdt.core.compiler.problem.noEffectAssignment=warning -org.eclipse.jdt.core.compiler.problem.noImplicitStringConversion=warning -org.eclipse.jdt.core.compiler.problem.nonExternalizedStringLiteral=ignore -org.eclipse.jdt.core.compiler.problem.nonnullParameterAnnotationDropped=warning -org.eclipse.jdt.core.compiler.problem.nullAnnotationInferenceConflict=error -org.eclipse.jdt.core.compiler.problem.nullReference=warning -org.eclipse.jdt.core.compiler.problem.nullSpecViolation=error -org.eclipse.jdt.core.compiler.problem.nullUncheckedConversion=warning -org.eclipse.jdt.core.compiler.problem.overridingPackageDefaultMethod=warning -org.eclipse.jdt.core.compiler.problem.parameterAssignment=ignore -org.eclipse.jdt.core.compiler.problem.possibleAccidentalBooleanAssignment=ignore -org.eclipse.jdt.core.compiler.problem.potentialNullReference=ignore -org.eclipse.jdt.core.compiler.problem.potentiallyUnclosedCloseable=ignore -org.eclipse.jdt.core.compiler.problem.rawTypeReference=warning -org.eclipse.jdt.core.compiler.problem.redundantNullAnnotation=warning -org.eclipse.jdt.core.compiler.problem.redundantNullCheck=ignore -org.eclipse.jdt.core.compiler.problem.redundantSpecificationOfTypeArguments=ignore -org.eclipse.jdt.core.compiler.problem.redundantSuperinterface=ignore -org.eclipse.jdt.core.compiler.problem.reportMethodCanBePotentiallyStatic=ignore -org.eclipse.jdt.core.compiler.problem.reportMethodCanBeStatic=ignore -org.eclipse.jdt.core.compiler.problem.specialParameterHidingField=disabled -org.eclipse.jdt.core.compiler.problem.staticAccessReceiver=warning -org.eclipse.jdt.core.compiler.problem.suppressOptionalErrors=disabled -org.eclipse.jdt.core.compiler.problem.suppressWarnings=enabled -org.eclipse.jdt.core.compiler.problem.syntacticNullAnalysisForFields=disabled -org.eclipse.jdt.core.compiler.problem.syntheticAccessEmulation=ignore -org.eclipse.jdt.core.compiler.problem.typeParameterHiding=warning -org.eclipse.jdt.core.compiler.problem.unavoidableGenericTypeProblems=enabled -org.eclipse.jdt.core.compiler.problem.uncheckedTypeOperation=warning -org.eclipse.jdt.core.compiler.problem.unclosedCloseable=warning -org.eclipse.jdt.core.compiler.problem.undocumentedEmptyBlock=ignore -org.eclipse.jdt.core.compiler.problem.unhandledWarningToken=warning -org.eclipse.jdt.core.compiler.problem.unnecessaryElse=ignore -org.eclipse.jdt.core.compiler.problem.unnecessaryTypeCheck=ignore -org.eclipse.jdt.core.compiler.problem.unqualifiedFieldAccess=ignore -org.eclipse.jdt.core.compiler.problem.unusedDeclaredThrownException=ignore -org.eclipse.jdt.core.compiler.problem.unusedDeclaredThrownExceptionExemptExceptionAndThrowable=enabled -org.eclipse.jdt.core.compiler.problem.unusedDeclaredThrownExceptionIncludeDocCommentReference=enabled -org.eclipse.jdt.core.compiler.problem.unusedDeclaredThrownExceptionWhenOverriding=disabled -org.eclipse.jdt.core.compiler.problem.unusedImport=warning -org.eclipse.jdt.core.compiler.problem.unusedLabel=warning -org.eclipse.jdt.core.compiler.problem.unusedLocal=warning -org.eclipse.jdt.core.compiler.problem.unusedObjectAllocation=ignore -org.eclipse.jdt.core.compiler.problem.unusedParameter=ignore -org.eclipse.jdt.core.compiler.problem.unusedParameterIncludeDocCommentReference=enabled -org.eclipse.jdt.core.compiler.problem.unusedParameterWhenImplementingAbstract=disabled -org.eclipse.jdt.core.compiler.problem.unusedParameterWhenOverridingConcrete=disabled -org.eclipse.jdt.core.compiler.problem.unusedPrivateMember=warning -org.eclipse.jdt.core.compiler.problem.unusedTypeParameter=ignore -org.eclipse.jdt.core.compiler.problem.unusedWarningToken=warning -org.eclipse.jdt.core.compiler.problem.varargsArgumentNeedCast=warning -org.eclipse.jdt.core.compiler.source=1.7 -org.eclipse.jdt.core.formatter.align_type_members_on_columns=false -org.eclipse.jdt.core.formatter.alignment_for_arguments_in_allocation_expression=16 -org.eclipse.jdt.core.formatter.alignment_for_arguments_in_annotation=0 -org.eclipse.jdt.core.formatter.alignment_for_arguments_in_enum_constant=16 -org.eclipse.jdt.core.formatter.alignment_for_arguments_in_explicit_constructor_call=16 -org.eclipse.jdt.core.formatter.alignment_for_arguments_in_method_invocation=16 -org.eclipse.jdt.core.formatter.alignment_for_arguments_in_qualified_allocation_expression=16 -org.eclipse.jdt.core.formatter.alignment_for_assignment=0 -org.eclipse.jdt.core.formatter.alignment_for_binary_expression=16 -org.eclipse.jdt.core.formatter.alignment_for_compact_if=16 -org.eclipse.jdt.core.formatter.alignment_for_conditional_expression=80 -org.eclipse.jdt.core.formatter.alignment_for_enum_constants=0 -org.eclipse.jdt.core.formatter.alignment_for_expressions_in_array_initializer=16 -org.eclipse.jdt.core.formatter.alignment_for_method_declaration=0 -org.eclipse.jdt.core.formatter.alignment_for_multiple_fields=16 -org.eclipse.jdt.core.formatter.alignment_for_parameters_in_constructor_declaration=16 -org.eclipse.jdt.core.formatter.alignment_for_parameters_in_method_declaration=16 -org.eclipse.jdt.core.formatter.alignment_for_resources_in_try=80 -org.eclipse.jdt.core.formatter.alignment_for_selector_in_method_invocation=16 -org.eclipse.jdt.core.formatter.alignment_for_superclass_in_type_declaration=16 -org.eclipse.jdt.core.formatter.alignment_for_superinterfaces_in_enum_declaration=16 -org.eclipse.jdt.core.formatter.alignment_for_superinterfaces_in_type_declaration=16 -org.eclipse.jdt.core.formatter.alignment_for_throws_clause_in_constructor_declaration=16 -org.eclipse.jdt.core.formatter.alignment_for_throws_clause_in_method_declaration=16 -org.eclipse.jdt.core.formatter.alignment_for_union_type_in_multicatch=16 -org.eclipse.jdt.core.formatter.blank_lines_after_imports=1 -org.eclipse.jdt.core.formatter.blank_lines_after_package=1 -org.eclipse.jdt.core.formatter.blank_lines_before_field=0 -org.eclipse.jdt.core.formatter.blank_lines_before_first_class_body_declaration=0 -org.eclipse.jdt.core.formatter.blank_lines_before_imports=1 -org.eclipse.jdt.core.formatter.blank_lines_before_member_type=1 -org.eclipse.jdt.core.formatter.blank_lines_before_method=1 -org.eclipse.jdt.core.formatter.blank_lines_before_new_chunk=1 -org.eclipse.jdt.core.formatter.blank_lines_before_package=0 -org.eclipse.jdt.core.formatter.blank_lines_between_import_groups=1 -org.eclipse.jdt.core.formatter.blank_lines_between_type_declarations=1 -org.eclipse.jdt.core.formatter.brace_position_for_annotation_type_declaration=end_of_line -org.eclipse.jdt.core.formatter.brace_position_for_anonymous_type_declaration=end_of_line -org.eclipse.jdt.core.formatter.brace_position_for_array_initializer=end_of_line -org.eclipse.jdt.core.formatter.brace_position_for_block=end_of_line -org.eclipse.jdt.core.formatter.brace_position_for_block_in_case=end_of_line -org.eclipse.jdt.core.formatter.brace_position_for_constructor_declaration=end_of_line -org.eclipse.jdt.core.formatter.brace_position_for_enum_constant=end_of_line -org.eclipse.jdt.core.formatter.brace_position_for_enum_declaration=end_of_line -org.eclipse.jdt.core.formatter.brace_position_for_method_declaration=end_of_line -org.eclipse.jdt.core.formatter.brace_position_for_switch=end_of_line -org.eclipse.jdt.core.formatter.brace_position_for_type_declaration=end_of_line -org.eclipse.jdt.core.formatter.comment.clear_blank_lines_in_block_comment=false -org.eclipse.jdt.core.formatter.comment.clear_blank_lines_in_javadoc_comment=false -org.eclipse.jdt.core.formatter.comment.format_block_comments=false -org.eclipse.jdt.core.formatter.comment.format_header=false -org.eclipse.jdt.core.formatter.comment.format_html=true -org.eclipse.jdt.core.formatter.comment.format_javadoc_comments=false -org.eclipse.jdt.core.formatter.comment.format_line_comments=false -org.eclipse.jdt.core.formatter.comment.format_source_code=true -org.eclipse.jdt.core.formatter.comment.indent_parameter_description=true -org.eclipse.jdt.core.formatter.comment.indent_root_tags=true -org.eclipse.jdt.core.formatter.comment.insert_new_line_before_root_tags=insert -org.eclipse.jdt.core.formatter.comment.insert_new_line_for_parameter=insert -org.eclipse.jdt.core.formatter.comment.line_length=80 -org.eclipse.jdt.core.formatter.comment.new_lines_at_block_boundaries=true -org.eclipse.jdt.core.formatter.comment.new_lines_at_javadoc_boundaries=true -org.eclipse.jdt.core.formatter.comment.preserve_white_space_between_code_and_line_comments=false -org.eclipse.jdt.core.formatter.compact_else_if=true -org.eclipse.jdt.core.formatter.continuation_indentation=2 -org.eclipse.jdt.core.formatter.continuation_indentation_for_array_initializer=2 -org.eclipse.jdt.core.formatter.disabling_tag=@formatter\:off -org.eclipse.jdt.core.formatter.enabling_tag=@formatter\:on -org.eclipse.jdt.core.formatter.format_guardian_clause_on_one_line=false -org.eclipse.jdt.core.formatter.format_line_comment_starting_on_first_column=true -org.eclipse.jdt.core.formatter.indent_body_declarations_compare_to_annotation_declaration_header=true -org.eclipse.jdt.core.formatter.indent_body_declarations_compare_to_enum_constant_header=true -org.eclipse.jdt.core.formatter.indent_body_declarations_compare_to_enum_declaration_header=true -org.eclipse.jdt.core.formatter.indent_body_declarations_compare_to_type_header=true -org.eclipse.jdt.core.formatter.indent_breaks_compare_to_cases=true -org.eclipse.jdt.core.formatter.indent_empty_lines=false -org.eclipse.jdt.core.formatter.indent_statements_compare_to_block=true -org.eclipse.jdt.core.formatter.indent_statements_compare_to_body=true -org.eclipse.jdt.core.formatter.indent_switchstatements_compare_to_cases=true -org.eclipse.jdt.core.formatter.indent_switchstatements_compare_to_switch=false -org.eclipse.jdt.core.formatter.indentation.size=4 -org.eclipse.jdt.core.formatter.insert_new_line_after_annotation_on_field=insert -org.eclipse.jdt.core.formatter.insert_new_line_after_annotation_on_local_variable=insert -org.eclipse.jdt.core.formatter.insert_new_line_after_annotation_on_method=insert -org.eclipse.jdt.core.formatter.insert_new_line_after_annotation_on_package=insert -org.eclipse.jdt.core.formatter.insert_new_line_after_annotation_on_parameter=do not insert -org.eclipse.jdt.core.formatter.insert_new_line_after_annotation_on_type=insert -org.eclipse.jdt.core.formatter.insert_new_line_after_label=do not insert -org.eclipse.jdt.core.formatter.insert_new_line_after_opening_brace_in_array_initializer=do not insert -org.eclipse.jdt.core.formatter.insert_new_line_at_end_of_file_if_missing=do not insert -org.eclipse.jdt.core.formatter.insert_new_line_before_catch_in_try_statement=do not insert -org.eclipse.jdt.core.formatter.insert_new_line_before_closing_brace_in_array_initializer=do not insert -org.eclipse.jdt.core.formatter.insert_new_line_before_else_in_if_statement=do not insert -org.eclipse.jdt.core.formatter.insert_new_line_before_finally_in_try_statement=do not insert -org.eclipse.jdt.core.formatter.insert_new_line_before_while_in_do_statement=do not insert -org.eclipse.jdt.core.formatter.insert_new_line_in_empty_annotation_declaration=insert -org.eclipse.jdt.core.formatter.insert_new_line_in_empty_anonymous_type_declaration=insert -org.eclipse.jdt.core.formatter.insert_new_line_in_empty_block=insert -org.eclipse.jdt.core.formatter.insert_new_line_in_empty_enum_constant=insert -org.eclipse.jdt.core.formatter.insert_new_line_in_empty_enum_declaration=insert -org.eclipse.jdt.core.formatter.insert_new_line_in_empty_method_body=insert -org.eclipse.jdt.core.formatter.insert_new_line_in_empty_type_declaration=insert -org.eclipse.jdt.core.formatter.insert_space_after_and_in_type_parameter=insert -org.eclipse.jdt.core.formatter.insert_space_after_assignment_operator=insert -org.eclipse.jdt.core.formatter.insert_space_after_at_in_annotation=do not insert -org.eclipse.jdt.core.formatter.insert_space_after_at_in_annotation_type_declaration=do not insert -org.eclipse.jdt.core.formatter.insert_space_after_binary_operator=insert -org.eclipse.jdt.core.formatter.insert_space_after_closing_angle_bracket_in_type_arguments=insert -org.eclipse.jdt.core.formatter.insert_space_after_closing_angle_bracket_in_type_parameters=insert -org.eclipse.jdt.core.formatter.insert_space_after_closing_brace_in_block=insert -org.eclipse.jdt.core.formatter.insert_space_after_closing_paren_in_cast=insert -org.eclipse.jdt.core.formatter.insert_space_after_colon_in_assert=insert -org.eclipse.jdt.core.formatter.insert_space_after_colon_in_case=insert -org.eclipse.jdt.core.formatter.insert_space_after_colon_in_conditional=insert -org.eclipse.jdt.core.formatter.insert_space_after_colon_in_for=insert -org.eclipse.jdt.core.formatter.insert_space_after_colon_in_labeled_statement=insert -org.eclipse.jdt.core.formatter.insert_space_after_comma_in_allocation_expression=insert -org.eclipse.jdt.core.formatter.insert_space_after_comma_in_annotation=insert -org.eclipse.jdt.core.formatter.insert_space_after_comma_in_array_initializer=insert -org.eclipse.jdt.core.formatter.insert_space_after_comma_in_constructor_declaration_parameters=insert -org.eclipse.jdt.core.formatter.insert_space_after_comma_in_constructor_declaration_throws=insert -org.eclipse.jdt.core.formatter.insert_space_after_comma_in_enum_constant_arguments=insert -org.eclipse.jdt.core.formatter.insert_space_after_comma_in_enum_declarations=insert -org.eclipse.jdt.core.formatter.insert_space_after_comma_in_explicitconstructorcall_arguments=insert -org.eclipse.jdt.core.formatter.insert_space_after_comma_in_for_increments=insert -org.eclipse.jdt.core.formatter.insert_space_after_comma_in_for_inits=insert -org.eclipse.jdt.core.formatter.insert_space_after_comma_in_method_declaration_parameters=insert -org.eclipse.jdt.core.formatter.insert_space_after_comma_in_method_declaration_throws=insert -org.eclipse.jdt.core.formatter.insert_space_after_comma_in_method_invocation_arguments=insert -org.eclipse.jdt.core.formatter.insert_space_after_comma_in_multiple_field_declarations=insert -org.eclipse.jdt.core.formatter.insert_space_after_comma_in_multiple_local_declarations=insert -org.eclipse.jdt.core.formatter.insert_space_after_comma_in_parameterized_type_reference=insert -org.eclipse.jdt.core.formatter.insert_space_after_comma_in_superinterfaces=insert -org.eclipse.jdt.core.formatter.insert_space_after_comma_in_type_arguments=insert -org.eclipse.jdt.core.formatter.insert_space_after_comma_in_type_parameters=insert -org.eclipse.jdt.core.formatter.insert_space_after_ellipsis=insert -org.eclipse.jdt.core.formatter.insert_space_after_opening_angle_bracket_in_parameterized_type_reference=do not insert -org.eclipse.jdt.core.formatter.insert_space_after_opening_angle_bracket_in_type_arguments=do not insert -org.eclipse.jdt.core.formatter.insert_space_after_opening_angle_bracket_in_type_parameters=do not insert -org.eclipse.jdt.core.formatter.insert_space_after_opening_brace_in_array_initializer=insert -org.eclipse.jdt.core.formatter.insert_space_after_opening_bracket_in_array_allocation_expression=do not insert -org.eclipse.jdt.core.formatter.insert_space_after_opening_bracket_in_array_reference=do not insert -org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_annotation=do not insert -org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_cast=do not insert -org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_catch=do not insert -org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_constructor_declaration=do not insert -org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_enum_constant=do not insert -org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_for=do not insert -org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_if=do not insert -org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_method_declaration=do not insert -org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_method_invocation=do not insert -org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_parenthesized_expression=do not insert -org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_switch=do not insert -org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_synchronized=do not insert -org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_try=do not insert -org.eclipse.jdt.core.formatter.insert_space_after_opening_paren_in_while=do not insert -org.eclipse.jdt.core.formatter.insert_space_after_postfix_operator=do not insert -org.eclipse.jdt.core.formatter.insert_space_after_prefix_operator=do not insert -org.eclipse.jdt.core.formatter.insert_space_after_question_in_conditional=insert -org.eclipse.jdt.core.formatter.insert_space_after_question_in_wildcard=do not insert -org.eclipse.jdt.core.formatter.insert_space_after_semicolon_in_for=insert -org.eclipse.jdt.core.formatter.insert_space_after_semicolon_in_try_resources=insert -org.eclipse.jdt.core.formatter.insert_space_after_unary_operator=do not insert -org.eclipse.jdt.core.formatter.insert_space_before_and_in_type_parameter=insert -org.eclipse.jdt.core.formatter.insert_space_before_assignment_operator=insert -org.eclipse.jdt.core.formatter.insert_space_before_at_in_annotation_type_declaration=insert -org.eclipse.jdt.core.formatter.insert_space_before_binary_operator=insert -org.eclipse.jdt.core.formatter.insert_space_before_closing_angle_bracket_in_parameterized_type_reference=do not insert -org.eclipse.jdt.core.formatter.insert_space_before_closing_angle_bracket_in_type_arguments=do not insert -org.eclipse.jdt.core.formatter.insert_space_before_closing_angle_bracket_in_type_parameters=do not insert -org.eclipse.jdt.core.formatter.insert_space_before_closing_brace_in_array_initializer=insert -org.eclipse.jdt.core.formatter.insert_space_before_closing_bracket_in_array_allocation_expression=do not insert -org.eclipse.jdt.core.formatter.insert_space_before_closing_bracket_in_array_reference=do not insert -org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_annotation=do not insert -org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_cast=do not insert -org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_catch=do not insert -org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_constructor_declaration=do not insert -org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_enum_constant=do not insert -org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_for=do not insert -org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_if=do not insert -org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_method_declaration=do not insert -org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_method_invocation=do not insert -org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_parenthesized_expression=do not insert -org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_switch=do not insert -org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_synchronized=do not insert -org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_try=do not insert -org.eclipse.jdt.core.formatter.insert_space_before_closing_paren_in_while=do not insert -org.eclipse.jdt.core.formatter.insert_space_before_colon_in_assert=insert -org.eclipse.jdt.core.formatter.insert_space_before_colon_in_case=do not insert -org.eclipse.jdt.core.formatter.insert_space_before_colon_in_conditional=insert -org.eclipse.jdt.core.formatter.insert_space_before_colon_in_default=do not insert -org.eclipse.jdt.core.formatter.insert_space_before_colon_in_for=insert -org.eclipse.jdt.core.formatter.insert_space_before_colon_in_labeled_statement=do not insert -org.eclipse.jdt.core.formatter.insert_space_before_comma_in_allocation_expression=do not insert -org.eclipse.jdt.core.formatter.insert_space_before_comma_in_annotation=do not insert -org.eclipse.jdt.core.formatter.insert_space_before_comma_in_array_initializer=do not insert -org.eclipse.jdt.core.formatter.insert_space_before_comma_in_constructor_declaration_parameters=do not insert -org.eclipse.jdt.core.formatter.insert_space_before_comma_in_constructor_declaration_throws=do not insert -org.eclipse.jdt.core.formatter.insert_space_before_comma_in_enum_constant_arguments=do not insert -org.eclipse.jdt.core.formatter.insert_space_before_comma_in_enum_declarations=do not insert -org.eclipse.jdt.core.formatter.insert_space_before_comma_in_explicitconstructorcall_arguments=do not insert -org.eclipse.jdt.core.formatter.insert_space_before_comma_in_for_increments=do not insert -org.eclipse.jdt.core.formatter.insert_space_before_comma_in_for_inits=do not insert -org.eclipse.jdt.core.formatter.insert_space_before_comma_in_method_declaration_parameters=do not insert -org.eclipse.jdt.core.formatter.insert_space_before_comma_in_method_declaration_throws=do not insert -org.eclipse.jdt.core.formatter.insert_space_before_comma_in_method_invocation_arguments=do not insert -org.eclipse.jdt.core.formatter.insert_space_before_comma_in_multiple_field_declarations=do not insert -org.eclipse.jdt.core.formatter.insert_space_before_comma_in_multiple_local_declarations=do not insert -org.eclipse.jdt.core.formatter.insert_space_before_comma_in_parameterized_type_reference=do not insert -org.eclipse.jdt.core.formatter.insert_space_before_comma_in_superinterfaces=do not insert -org.eclipse.jdt.core.formatter.insert_space_before_comma_in_type_arguments=do not insert -org.eclipse.jdt.core.formatter.insert_space_before_comma_in_type_parameters=do not insert -org.eclipse.jdt.core.formatter.insert_space_before_ellipsis=do not insert -org.eclipse.jdt.core.formatter.insert_space_before_opening_angle_bracket_in_parameterized_type_reference=do not insert -org.eclipse.jdt.core.formatter.insert_space_before_opening_angle_bracket_in_type_arguments=do not insert -org.eclipse.jdt.core.formatter.insert_space_before_opening_angle_bracket_in_type_parameters=do not insert -org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_annotation_type_declaration=insert -org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_anonymous_type_declaration=insert -org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_array_initializer=insert -org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_block=insert -org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_constructor_declaration=insert -org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_enum_constant=insert -org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_enum_declaration=insert -org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_method_declaration=insert -org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_switch=insert -org.eclipse.jdt.core.formatter.insert_space_before_opening_brace_in_type_declaration=insert -org.eclipse.jdt.core.formatter.insert_space_before_opening_bracket_in_array_allocation_expression=do not insert -org.eclipse.jdt.core.formatter.insert_space_before_opening_bracket_in_array_reference=do not insert -org.eclipse.jdt.core.formatter.insert_space_before_opening_bracket_in_array_type_reference=do not insert -org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_annotation=do not insert -org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_annotation_type_member_declaration=do not insert -org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_catch=insert -org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_constructor_declaration=do not insert -org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_enum_constant=do not insert -org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_for=insert -org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_if=insert -org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_method_declaration=do not insert -org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_method_invocation=do not insert -org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_parenthesized_expression=do not insert -org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_switch=insert -org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_synchronized=insert -org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_try=insert -org.eclipse.jdt.core.formatter.insert_space_before_opening_paren_in_while=insert -org.eclipse.jdt.core.formatter.insert_space_before_parenthesized_expression_in_return=insert -org.eclipse.jdt.core.formatter.insert_space_before_parenthesized_expression_in_throw=insert -org.eclipse.jdt.core.formatter.insert_space_before_postfix_operator=do not insert -org.eclipse.jdt.core.formatter.insert_space_before_prefix_operator=do not insert -org.eclipse.jdt.core.formatter.insert_space_before_question_in_conditional=insert -org.eclipse.jdt.core.formatter.insert_space_before_question_in_wildcard=do not insert -org.eclipse.jdt.core.formatter.insert_space_before_semicolon=do not insert -org.eclipse.jdt.core.formatter.insert_space_before_semicolon_in_for=do not insert -org.eclipse.jdt.core.formatter.insert_space_before_semicolon_in_try_resources=do not insert -org.eclipse.jdt.core.formatter.insert_space_before_unary_operator=do not insert -org.eclipse.jdt.core.formatter.insert_space_between_brackets_in_array_type_reference=do not insert -org.eclipse.jdt.core.formatter.insert_space_between_empty_braces_in_array_initializer=do not insert -org.eclipse.jdt.core.formatter.insert_space_between_empty_brackets_in_array_allocation_expression=do not insert -org.eclipse.jdt.core.formatter.insert_space_between_empty_parens_in_annotation_type_member_declaration=do not insert -org.eclipse.jdt.core.formatter.insert_space_between_empty_parens_in_constructor_declaration=do not insert -org.eclipse.jdt.core.formatter.insert_space_between_empty_parens_in_enum_constant=do not insert -org.eclipse.jdt.core.formatter.insert_space_between_empty_parens_in_method_declaration=do not insert -org.eclipse.jdt.core.formatter.insert_space_between_empty_parens_in_method_invocation=do not insert -org.eclipse.jdt.core.formatter.join_lines_in_comments=true -org.eclipse.jdt.core.formatter.join_wrapped_lines=true -org.eclipse.jdt.core.formatter.keep_else_statement_on_same_line=false -org.eclipse.jdt.core.formatter.keep_empty_array_initializer_on_one_line=false -org.eclipse.jdt.core.formatter.keep_imple_if_on_one_line=false -org.eclipse.jdt.core.formatter.keep_then_statement_on_same_line=false -org.eclipse.jdt.core.formatter.lineSplit=999 -org.eclipse.jdt.core.formatter.never_indent_block_comments_on_first_column=false -org.eclipse.jdt.core.formatter.never_indent_line_comments_on_first_column=false -org.eclipse.jdt.core.formatter.number_of_blank_lines_at_beginning_of_method_body=0 -org.eclipse.jdt.core.formatter.number_of_empty_lines_to_preserve=1 -org.eclipse.jdt.core.formatter.put_empty_statement_on_new_line=true -org.eclipse.jdt.core.formatter.tabulation.char=space -org.eclipse.jdt.core.formatter.tabulation.size=4 -org.eclipse.jdt.core.formatter.use_on_off_tags=false -org.eclipse.jdt.core.formatter.use_tabs_only_for_leading_indentations=false -org.eclipse.jdt.core.formatter.wrap_before_binary_operator=true -org.eclipse.jdt.core.formatter.wrap_before_or_operator_multicatch=true -org.eclipse.jdt.core.formatter.wrap_outer_expressions_when_nested=true http://git-wip-us.apache.org/repos/asf/kylin/blob/6b6aa313/jdbc/.settings/org.eclipse.jdt.ui.prefs ---------------------------------------------------------------------- diff --git a/jdbc/.settings/org.eclipse.jdt.ui.prefs b/jdbc/.settings/org.eclipse.jdt.ui.prefs deleted file mode 100644 index dece0e6..0000000 --- a/jdbc/.settings/org.eclipse.jdt.ui.prefs +++ /dev/null @@ -1,7 +0,0 @@ -eclipse.preferences.version=1 -formatter_profile=_Space Indent & Long Lines -formatter_settings_version=12 -org.eclipse.jdt.ui.ignorelowercasenames=true -org.eclipse.jdt.ui.importorder=java;javax;org;com; -org.eclipse.jdt.ui.ondemandthreshold=99 -org.eclipse.jdt.ui.staticondemandthreshold=1
