You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kylin.apache.org by li...@apache.org on 2016/02/21 12:51:56 UTC

[01/52] [abbrv] kylin git commit: KYLIN-1343 Compile pass, pending IT [Forced Update!]

Repository: kylin
Updated Branches:
  refs/heads/1.x-HBase1.1.3 0cec6fb2b -> 48cec941b (forced update)


http://git-wip-us.apache.org/repos/asf/kylin/blob/df7ae17a/jdbc/src/main/java/org/apache/kylin/jdbc/KylinConnection.java
----------------------------------------------------------------------
diff --git a/jdbc/src/main/java/org/apache/kylin/jdbc/KylinConnection.java b/jdbc/src/main/java/org/apache/kylin/jdbc/KylinConnection.java
index cf5526b..3ae5d29 100644
--- a/jdbc/src/main/java/org/apache/kylin/jdbc/KylinConnection.java
+++ b/jdbc/src/main/java/org/apache/kylin/jdbc/KylinConnection.java
@@ -118,7 +118,7 @@ public class KylinConnection extends AvaticaConnection {
         ArrayList<ColumnMetaData> columns = new ArrayList<ColumnMetaData>();
         Map<String, Object> internalParams = Collections.<String, Object> emptyMap();
 
-        return new Meta.Signature(columns, sql, params, internalParams, CursorFactory.ARRAY);
+        return new Meta.Signature(columns, sql, params, internalParams, CursorFactory.ARRAY, Meta.StatementType.SELECT);
     }
 
     private KylinJdbcFactory factory() {

http://git-wip-us.apache.org/repos/asf/kylin/blob/df7ae17a/jdbc/src/main/java/org/apache/kylin/jdbc/KylinJdbcFactory.java
----------------------------------------------------------------------
diff --git a/jdbc/src/main/java/org/apache/kylin/jdbc/KylinJdbcFactory.java b/jdbc/src/main/java/org/apache/kylin/jdbc/KylinJdbcFactory.java
index 9f9e986..dbfe36d 100644
--- a/jdbc/src/main/java/org/apache/kylin/jdbc/KylinJdbcFactory.java
+++ b/jdbc/src/main/java/org/apache/kylin/jdbc/KylinJdbcFactory.java
@@ -30,6 +30,7 @@ import org.apache.calcite.avatica.AvaticaPreparedStatement;
 import org.apache.calcite.avatica.AvaticaResultSet;
 import org.apache.calcite.avatica.AvaticaResultSetMetaData;
 import org.apache.calcite.avatica.AvaticaStatement;
+import org.apache.calcite.avatica.QueryState;
 import org.apache.calcite.avatica.Meta.Frame;
 import org.apache.calcite.avatica.Meta.Signature;
 import org.apache.calcite.avatica.Meta.StatementHandle;
@@ -93,9 +94,9 @@ public class KylinJdbcFactory implements AvaticaFactory {
     }
 
     @Override
-    public AvaticaResultSet newResultSet(AvaticaStatement statement, Signature signature, TimeZone timeZone, Frame firstFrame) throws SQLException {
+    public AvaticaResultSet newResultSet(AvaticaStatement statement, QueryState state, Signature signature, TimeZone timeZone, Frame firstFrame) throws SQLException {
         AvaticaResultSetMetaData resultSetMetaData = new AvaticaResultSetMetaData(statement, null, signature);
-        return new KylinResultSet(statement, signature, resultSetMetaData, timeZone, firstFrame);
+        return new KylinResultSet(statement, state, signature, resultSetMetaData, timeZone, firstFrame);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/kylin/blob/df7ae17a/jdbc/src/main/java/org/apache/kylin/jdbc/KylinMeta.java
----------------------------------------------------------------------
diff --git a/jdbc/src/main/java/org/apache/kylin/jdbc/KylinMeta.java b/jdbc/src/main/java/org/apache/kylin/jdbc/KylinMeta.java
index 79defcc..6bfd356 100644
--- a/jdbc/src/main/java/org/apache/kylin/jdbc/KylinMeta.java
+++ b/jdbc/src/main/java/org/apache/kylin/jdbc/KylinMeta.java
@@ -30,6 +30,10 @@ import java.util.regex.Pattern;
 import org.apache.calcite.avatica.AvaticaUtils;
 import org.apache.calcite.avatica.ColumnMetaData;
 import org.apache.calcite.avatica.MetaImpl;
+import org.apache.calcite.avatica.MissingResultsException;
+import org.apache.calcite.avatica.NoSuchStatementException;
+import org.apache.calcite.avatica.QueryState;
+import org.apache.calcite.avatica.remote.TypedValue;
 
 import com.google.common.collect.ImmutableList;
 
@@ -56,6 +60,13 @@ public class KylinMeta extends MetaImpl {
         return result;
     }
 
+    // real execution happens in KylinResultSet.execute()
+    @Override
+    public ExecuteResult execute(StatementHandle sh, List<TypedValue> parameterValues, long maxRowCount) throws NoSuchStatementException {
+        final MetaResultSet metaResultSet = MetaResultSet.create(sh.connectionId,  sh.id, false, sh.signature, null);
+        return new ExecuteResult(ImmutableList.of(metaResultSet));
+    }
+
     // mimic from CalciteMetaImpl, real execution happens via callback in KylinResultSet.execute()
     @Override
     public ExecuteResult prepareAndExecute(StatementHandle sh, String sql, long maxRowCount, PrepareCallback callback) {
@@ -91,24 +102,24 @@ public class KylinMeta extends MetaImpl {
     }
 
     @Override
-    public MetaResultSet getTableTypes() {
+    public MetaResultSet getTableTypes(ConnectionHandle ch) {
         return createResultSet(metaTableTypes, MetaTableType.class, "TABLE_TYPE");
     }
 
     @Override
-    public MetaResultSet getCatalogs() {
+    public MetaResultSet getCatalogs(ConnectionHandle ch) {
         List<KMetaCatalog> catalogs = getMetaProject().catalogs;
         return createResultSet(catalogs, KMetaCatalog.class, "TABLE_CAT");
     }
 
     @Override
-    public MetaResultSet getSchemas(String catalog, Pat schemaPattern) {
+    public MetaResultSet getSchemas(ConnectionHandle ch, String catalog, Pat schemaPattern) {
         List<KMetaSchema> schemas = getMetaProject().getSchemas(catalog, schemaPattern);
         return createResultSet(schemas, KMetaSchema.class, "TABLE_SCHEM", "TABLE_CATALOG");
     }
 
     @Override
-    public MetaResultSet getTables(String catalog, Pat schemaPattern, Pat tableNamePattern, List<String> typeList) {
+    public MetaResultSet getTables(ConnectionHandle ch, String catalog, Pat schemaPattern, Pat tableNamePattern, List<String> typeList) {
         List<KMetaTable> tables = getMetaProject().getTables(catalog, schemaPattern, tableNamePattern, typeList);
         return createResultSet(tables, KMetaTable.class, //
                 "TABLE_CAT", //
@@ -124,7 +135,7 @@ public class KylinMeta extends MetaImpl {
     }
 
     @Override
-    public MetaResultSet getColumns(String catalog, Pat schemaPattern, Pat tableNamePattern, Pat columnNamePattern) {
+    public MetaResultSet getColumns(ConnectionHandle ch, String catalog, Pat schemaPattern, Pat tableNamePattern, Pat columnNamePattern) {
         List<KMetaColumn> columns = getMetaProject().getColumns(catalog, schemaPattern, tableNamePattern, columnNamePattern);
         return createResultSet(columns, KMetaColumn.class, //
                 "TABLE_CAT", //
@@ -172,7 +183,7 @@ public class KylinMeta extends MetaImpl {
         }
 
         CursorFactory cursorFactory = CursorFactory.record(clazz, fields, fieldNames);
-        Signature signature = new Signature(columns, "", null, Collections.<String, Object> emptyMap(), cursorFactory);
+        Signature signature = new Signature(columns, "", null, Collections.<String, Object> emptyMap(), cursorFactory, StatementType.SELECT);
         StatementHandle sh = this.createStatement(connection().handle);
         Frame frame = new Frame(0, true, iterable);
 
@@ -355,4 +366,28 @@ public class KylinMeta extends MetaImpl {
         }
     }
 
+    @Override
+    public Frame fetch(StatementHandle h, long offset, int fetchMaxRowCount) throws NoSuchStatementException, MissingResultsException {
+        // TODO Auto-generated method stub
+        return null;
+    }
+
+    @Override
+    public boolean syncResults(StatementHandle sh, QueryState state, long offset) throws NoSuchStatementException {
+        // TODO Auto-generated method stub
+        return false;
+    }
+
+    @Override
+    public void commit(ConnectionHandle ch) {
+        // TODO Auto-generated method stub
+        
+    }
+
+    @Override
+    public void rollback(ConnectionHandle ch) {
+        // TODO Auto-generated method stub
+        
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/df7ae17a/jdbc/src/main/java/org/apache/kylin/jdbc/KylinPreparedStatement.java
----------------------------------------------------------------------
diff --git a/jdbc/src/main/java/org/apache/kylin/jdbc/KylinPreparedStatement.java b/jdbc/src/main/java/org/apache/kylin/jdbc/KylinPreparedStatement.java
index 6552eac..098c3c2 100644
--- a/jdbc/src/main/java/org/apache/kylin/jdbc/KylinPreparedStatement.java
+++ b/jdbc/src/main/java/org/apache/kylin/jdbc/KylinPreparedStatement.java
@@ -35,6 +35,8 @@ public class KylinPreparedStatement extends AvaticaPreparedStatement {
 
     protected KylinPreparedStatement(AvaticaConnection connection, StatementHandle h, Signature signature, int resultSetType, int resultSetConcurrency, int resultSetHoldability) throws SQLException {
         super(connection, h, signature, resultSetType, resultSetConcurrency, resultSetHoldability);
+        if (this.handle.signature == null)
+            this.handle.signature = signature;
     }
 
     protected List<Object> getParameterValues2() {

http://git-wip-us.apache.org/repos/asf/kylin/blob/df7ae17a/jdbc/src/main/java/org/apache/kylin/jdbc/KylinResultSet.java
----------------------------------------------------------------------
diff --git a/jdbc/src/main/java/org/apache/kylin/jdbc/KylinResultSet.java b/jdbc/src/main/java/org/apache/kylin/jdbc/KylinResultSet.java
index 453b9ea..9db53f4 100644
--- a/jdbc/src/main/java/org/apache/kylin/jdbc/KylinResultSet.java
+++ b/jdbc/src/main/java/org/apache/kylin/jdbc/KylinResultSet.java
@@ -27,6 +27,7 @@ import java.util.TimeZone;
 import org.apache.calcite.avatica.AvaticaParameter;
 import org.apache.calcite.avatica.AvaticaResultSet;
 import org.apache.calcite.avatica.AvaticaStatement;
+import org.apache.calcite.avatica.QueryState;
 import org.apache.calcite.avatica.Meta.Frame;
 import org.apache.calcite.avatica.Meta.Signature;
 import org.apache.calcite.avatica.MetaImpl;
@@ -34,8 +35,8 @@ import org.apache.kylin.jdbc.IRemoteClient.QueryResult;
 
 public class KylinResultSet extends AvaticaResultSet {
 
-    public KylinResultSet(AvaticaStatement statement, Signature signature, ResultSetMetaData resultSetMetaData, TimeZone timeZone, Frame firstFrame) {
-        super(statement, signature, resultSetMetaData, timeZone, firstFrame);
+    public KylinResultSet(AvaticaStatement statement, QueryState state, Signature signature, ResultSetMetaData resultSetMetaData, TimeZone timeZone, Frame firstFrame) {
+        super(statement, state, signature, resultSetMetaData, timeZone, firstFrame);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/kylin/blob/df7ae17a/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 374bac9..793a5f2 100644
--- a/pom.xml
+++ b/pom.xml
@@ -88,7 +88,7 @@
         <spring.boot.version>1.2.7.RELEASE</spring.boot.version>
 
         <!-- Calcite Version -->
-        <calcite.version>1.4.0-incubating</calcite.version>
+        <calcite.version>1.6.0</calcite.version>
 
         <!-- Curator.version Version -->
         <curator.version>2.6.0</curator.version>

http://git-wip-us.apache.org/repos/asf/kylin/blob/df7ae17a/query/src/main/java/org/apache/kylin/query/optrule/OLAPJoinRule.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/org/apache/kylin/query/optrule/OLAPJoinRule.java b/query/src/main/java/org/apache/kylin/query/optrule/OLAPJoinRule.java
index dee9154..d247742 100644
--- a/query/src/main/java/org/apache/kylin/query/optrule/OLAPJoinRule.java
+++ b/query/src/main/java/org/apache/kylin/query/optrule/OLAPJoinRule.java
@@ -63,7 +63,7 @@ public class OLAPJoinRule extends ConverterRule {
         try {
             newRel = new OLAPJoinRel(cluster, traitSet, left, right, //
                     info.getEquiCondition(left, right, cluster.getRexBuilder()), //
-                    info.leftKeys, info.rightKeys, join.getJoinType(), join.getVariablesStopped());
+                    info.leftKeys, info.rightKeys, join.getVariablesSet(), join.getJoinType());
         } catch (InvalidRelException e) {
             // Semantic error not possible. Must be a bug. Convert to internal error.
             throw new AssertionError(e);

http://git-wip-us.apache.org/repos/asf/kylin/blob/df7ae17a/query/src/main/java/org/apache/kylin/query/relnode/OLAPAggregateRel.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/org/apache/kylin/query/relnode/OLAPAggregateRel.java b/query/src/main/java/org/apache/kylin/query/relnode/OLAPAggregateRel.java
index 9e75106..3d1ddb3 100644
--- a/query/src/main/java/org/apache/kylin/query/relnode/OLAPAggregateRel.java
+++ b/query/src/main/java/org/apache/kylin/query/relnode/OLAPAggregateRel.java
@@ -26,6 +26,7 @@ import java.util.Map;
 import java.util.Set;
 
 import com.google.common.collect.Lists;
+
 import org.apache.calcite.adapter.enumerable.EnumerableAggregate;
 import org.apache.calcite.adapter.enumerable.EnumerableConvention;
 import org.apache.calcite.adapter.enumerable.EnumerableRel;
@@ -38,6 +39,7 @@ import org.apache.calcite.rel.InvalidRelException;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.Aggregate;
 import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.rel.type.RelDataTypeField;
@@ -122,8 +124,8 @@ public class OLAPAggregateRel extends Aggregate implements OLAPRel {
     }
 
     @Override
-    public RelOptCost computeSelfCost(RelOptPlanner planner) {
-        return super.computeSelfCost(planner).multiplyBy(.05);
+    public RelOptCost computeSelfCost(RelOptPlanner planner, RelMetadataQuery mq) {
+        return super.computeSelfCost(planner, mq).multiplyBy(.05);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/kylin/blob/df7ae17a/query/src/main/java/org/apache/kylin/query/relnode/OLAPFilterRel.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/org/apache/kylin/query/relnode/OLAPFilterRel.java b/query/src/main/java/org/apache/kylin/query/relnode/OLAPFilterRel.java
index 583af9a..667acf8 100644
--- a/query/src/main/java/org/apache/kylin/query/relnode/OLAPFilterRel.java
+++ b/query/src/main/java/org/apache/kylin/query/relnode/OLAPFilterRel.java
@@ -20,6 +20,7 @@ package org.apache.kylin.query.relnode;
 
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Sets;
+
 import org.apache.calcite.adapter.enumerable.EnumerableCalc;
 import org.apache.calcite.adapter.enumerable.EnumerableConvention;
 import org.apache.calcite.adapter.enumerable.EnumerableRel;
@@ -27,6 +28,7 @@ import org.apache.calcite.avatica.util.TimeUnitRange;
 import org.apache.calcite.plan.*;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.Filter;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rex.*;
 import org.apache.calcite.sql.SqlKind;
@@ -220,8 +222,8 @@ public class OLAPFilterRel extends Filter implements OLAPRel {
     }
 
     @Override
-    public RelOptCost computeSelfCost(RelOptPlanner planner) {
-        return super.computeSelfCost(planner).multiplyBy(.05);
+    public RelOptCost computeSelfCost(RelOptPlanner planner, RelMetadataQuery mq) {
+        return super.computeSelfCost(planner, mq).multiplyBy(.05);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/kylin/blob/df7ae17a/query/src/main/java/org/apache/kylin/query/relnode/OLAPJoinRel.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/org/apache/kylin/query/relnode/OLAPJoinRel.java b/query/src/main/java/org/apache/kylin/query/relnode/OLAPJoinRel.java
index 30f059c..c079357 100644
--- a/query/src/main/java/org/apache/kylin/query/relnode/OLAPJoinRel.java
+++ b/query/src/main/java/org/apache/kylin/query/relnode/OLAPJoinRel.java
@@ -43,8 +43,10 @@ import org.apache.calcite.plan.RelTrait;
 import org.apache.calcite.plan.RelTraitSet;
 import org.apache.calcite.rel.InvalidRelException;
 import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.CorrelationId;
 import org.apache.calcite.rel.core.JoinInfo;
 import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory.FieldInfoBuilder;
 import org.apache.calcite.rel.type.RelDataTypeField;
@@ -73,8 +75,8 @@ public class OLAPJoinRel extends EnumerableJoin implements OLAPRel {
 
     public OLAPJoinRel(RelOptCluster cluster, RelTraitSet traits, RelNode left, RelNode right, //
             RexNode condition, ImmutableIntList leftKeys, ImmutableIntList rightKeys, //
-            JoinRelType joinType, Set<String> variablesStopped) throws InvalidRelException {
-        super(cluster, traits, left, right, condition, leftKeys, rightKeys, joinType, variablesStopped);
+            Set<CorrelationId> variablesSet, JoinRelType joinType) throws InvalidRelException {
+        super(cluster, traits, left, right, condition, leftKeys, rightKeys, variablesSet, joinType);
         Preconditions.checkArgument(getConvention() == OLAPRel.CONVENTION);
         this.rowType = getRowType();
         this.isTopJoin = false;
@@ -87,7 +89,7 @@ public class OLAPJoinRel extends EnumerableJoin implements OLAPRel {
         final JoinInfo joinInfo = JoinInfo.of(left, right, condition);
         assert joinInfo.isEqui();
         try {
-            return new OLAPJoinRel(getCluster(), traitSet, left, right, condition, joinInfo.leftKeys, joinInfo.rightKeys, joinType, variablesStopped);
+            return new OLAPJoinRel(getCluster(), traitSet, left, right, condition, joinInfo.leftKeys, joinInfo.rightKeys, variablesSet, joinType);
         } catch (InvalidRelException e) {
             // Semantic error not possible. Must be a bug. Convert to internal error.
             throw new AssertionError(e);
@@ -95,13 +97,13 @@ public class OLAPJoinRel extends EnumerableJoin implements OLAPRel {
     }
 
     @Override
-    public RelOptCost computeSelfCost(RelOptPlanner planner) {
-        return super.computeSelfCost(planner).multiplyBy(.05);
+    public RelOptCost computeSelfCost(RelOptPlanner planner, RelMetadataQuery mq) {
+        return super.computeSelfCost(planner, mq).multiplyBy(.05);
     }
 
     @Override
-    public double getRows() {
-        return super.getRows() * 0.1;
+    public double estimateRowCount(RelMetadataQuery mq) {
+        return super.estimateRowCount(mq) * 0.1;
     }
 
     @Override
@@ -238,8 +240,8 @@ public class OLAPJoinRel extends EnumerableJoin implements OLAPRel {
                     RexNode.class, //
                     ImmutableIntList.class, //
                     ImmutableIntList.class, //
-                    JoinRelType.class, //
-                    Set.class);
+                    Set.class, //
+                    JoinRelType.class);
             constr.setAccessible(true);
         } catch (Exception e) {
             throw new RuntimeException(e);
@@ -251,7 +253,7 @@ public class OLAPJoinRel extends EnumerableJoin implements OLAPRel {
         if (this.hasSubQuery) {
             try {
                 return constr.newInstance(getCluster(), getCluster().traitSetOf(EnumerableConvention.INSTANCE), //
-                        inputs.get(0), inputs.get(1), condition, leftKeys, rightKeys, joinType, variablesStopped);
+                        inputs.get(0), inputs.get(1), condition, leftKeys, rightKeys, variablesSet, joinType);
             } catch (Exception e) {
                 throw new IllegalStateException("Can't create EnumerableJoin!", e);
             }

http://git-wip-us.apache.org/repos/asf/kylin/blob/df7ae17a/query/src/main/java/org/apache/kylin/query/relnode/OLAPLimitRel.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/org/apache/kylin/query/relnode/OLAPLimitRel.java b/query/src/main/java/org/apache/kylin/query/relnode/OLAPLimitRel.java
index f6084ba..c891770 100644
--- a/query/src/main/java/org/apache/kylin/query/relnode/OLAPLimitRel.java
+++ b/query/src/main/java/org/apache/kylin/query/relnode/OLAPLimitRel.java
@@ -31,6 +31,7 @@ import org.apache.calcite.plan.RelTraitSet;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.RelWriter;
 import org.apache.calcite.rel.SingleRel;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rex.RexLiteral;
 import org.apache.calcite.rex.RexNode;
 
@@ -59,8 +60,8 @@ public class OLAPLimitRel extends SingleRel implements OLAPRel {
     }
 
     @Override
-    public RelOptCost computeSelfCost(RelOptPlanner planner) {
-        return super.computeSelfCost(planner).multiplyBy(.05);
+    public RelOptCost computeSelfCost(RelOptPlanner planner, RelMetadataQuery mq) {
+        return super.computeSelfCost(planner, mq).multiplyBy(.05);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/kylin/blob/df7ae17a/query/src/main/java/org/apache/kylin/query/relnode/OLAPProjectRel.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/org/apache/kylin/query/relnode/OLAPProjectRel.java b/query/src/main/java/org/apache/kylin/query/relnode/OLAPProjectRel.java
index f0c7531..6a54767 100644
--- a/query/src/main/java/org/apache/kylin/query/relnode/OLAPProjectRel.java
+++ b/query/src/main/java/org/apache/kylin/query/relnode/OLAPProjectRel.java
@@ -35,6 +35,7 @@ import org.apache.calcite.plan.RelTrait;
 import org.apache.calcite.plan.RelTraitSet;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory.FieldInfoBuilder;
 import org.apache.calcite.rel.type.RelDataTypeField;
@@ -86,8 +87,8 @@ public class OLAPProjectRel extends Project implements OLAPRel {
     }
 
     @Override
-    public RelOptCost computeSelfCost(RelOptPlanner planner) {
-        return super.computeSelfCost(planner).multiplyBy(.05);
+    public RelOptCost computeSelfCost(RelOptPlanner planner, RelMetadataQuery mq) {
+        return super.computeSelfCost(planner, mq).multiplyBy(.05);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/kylin/blob/df7ae17a/query/src/main/java/org/apache/kylin/query/relnode/OLAPSortRel.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/org/apache/kylin/query/relnode/OLAPSortRel.java b/query/src/main/java/org/apache/kylin/query/relnode/OLAPSortRel.java
index 766f2cd..e8b7d17 100644
--- a/query/src/main/java/org/apache/kylin/query/relnode/OLAPSortRel.java
+++ b/query/src/main/java/org/apache/kylin/query/relnode/OLAPSortRel.java
@@ -32,6 +32,7 @@ import org.apache.calcite.rel.RelCollation;
 import org.apache.calcite.rel.RelFieldCollation;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.Sort;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rex.RexNode;
 import org.apache.kylin.metadata.model.MeasureDesc;
 import org.apache.kylin.metadata.model.TblColRef;
@@ -58,8 +59,8 @@ public class OLAPSortRel extends Sort implements OLAPRel {
     }
 
     @Override
-    public RelOptCost computeSelfCost(RelOptPlanner planner) {
-        return super.computeSelfCost(planner).multiplyBy(.05);
+    public RelOptCost computeSelfCost(RelOptPlanner planner, RelMetadataQuery mq) {
+        return super.computeSelfCost(planner, mq).multiplyBy(.05);
     }
 
     @Override
@@ -115,7 +116,7 @@ public class OLAPSortRel extends Sort implements OLAPRel {
 
     @Override
     public EnumerableRel implementEnumerable(List<EnumerableRel> inputs) {
-        return new EnumerableSort(getCluster(), getCluster().traitSetOf(EnumerableConvention.INSTANCE, collation), //
+        return new EnumerableSort(getCluster(), getCluster().traitSetOf(EnumerableConvention.INSTANCE).replace(collation), //
                 sole(inputs), collation, offset, fetch);
     }
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/df7ae17a/query/src/main/java/org/apache/kylin/query/relnode/OLAPTableScan.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/org/apache/kylin/query/relnode/OLAPTableScan.java b/query/src/main/java/org/apache/kylin/query/relnode/OLAPTableScan.java
index d2661b4..48cf662 100644
--- a/query/src/main/java/org/apache/kylin/query/relnode/OLAPTableScan.java
+++ b/query/src/main/java/org/apache/kylin/query/relnode/OLAPTableScan.java
@@ -40,6 +40,7 @@ import org.apache.calcite.plan.volcano.AbstractConverter.ExpandConversionRule;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.RelWriter;
 import org.apache.calcite.rel.core.TableScan;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rel.rules.AggregateExpandDistinctAggregatesRule;
 import org.apache.calcite.rel.rules.AggregateJoinTransposeRule;
 import org.apache.calcite.rel.rules.AggregateProjectMergeRule;
@@ -165,8 +166,8 @@ public class OLAPTableScan extends TableScan implements OLAPRel, EnumerableRel {
     }
 
     @Override
-    public RelOptCost computeSelfCost(RelOptPlanner planner) {
-        return super.computeSelfCost(planner).multiplyBy(.05);
+    public RelOptCost computeSelfCost(RelOptPlanner planner, RelMetadataQuery mq) {
+        return super.computeSelfCost(planner, mq).multiplyBy(.05);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/kylin/blob/df7ae17a/query/src/main/java/org/apache/kylin/query/relnode/OLAPToEnumerableConverter.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/org/apache/kylin/query/relnode/OLAPToEnumerableConverter.java b/query/src/main/java/org/apache/kylin/query/relnode/OLAPToEnumerableConverter.java
index 21057e2..2ae68d6 100644
--- a/query/src/main/java/org/apache/kylin/query/relnode/OLAPToEnumerableConverter.java
+++ b/query/src/main/java/org/apache/kylin/query/relnode/OLAPToEnumerableConverter.java
@@ -35,6 +35,7 @@ import org.apache.calcite.plan.RelOptUtil;
 import org.apache.calcite.plan.RelTraitSet;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.convert.ConverterImpl;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.sql.SqlExplainLevel;
 import org.apache.kylin.metadata.realization.IRealization;
@@ -56,8 +57,8 @@ public class OLAPToEnumerableConverter extends ConverterImpl implements Enumerab
     }
 
     @Override
-    public RelOptCost computeSelfCost(RelOptPlanner planner) {
-        return super.computeSelfCost(planner).multiplyBy(.05);
+    public RelOptCost computeSelfCost(RelOptPlanner planner, RelMetadataQuery mq) {
+        return super.computeSelfCost(planner, mq).multiplyBy(.05);
     }
 
     @Override


[14/52] [abbrv] kylin git commit: KYLIN-1372 Query using PrepareStatement failed with multi OR clause

Posted by li...@apache.org.
KYLIN-1372 Query using PrepareStatement failed with multi OR clause


Project: http://git-wip-us.apache.org/repos/asf/kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/0c13797e
Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/0c13797e
Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/0c13797e

Branch: refs/heads/1.x-HBase1.1.3
Commit: 0c13797e11984ba0411f0c0e3c42540a822e847a
Parents: d697447
Author: sunyerui <su...@gmail.com>
Authored: Tue Jan 26 21:57:21 2016 +0800
Committer: sunyerui <su...@gmail.com>
Committed: Fri Jan 29 11:56:13 2016 +0800

----------------------------------------------------------------------
 .../metadata/filter/CompareTupleFilter.java     |  4 ++-
 .../kylin/query/relnode/OLAPFilterRel.java      |  3 +++
 .../resources/query/sql_dynamic/query03.dat     |  2 ++
 .../resources/query/sql_dynamic/query03.sql     | 26 ++++++++++++++++++++
 4 files changed, 34 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/0c13797e/metadata/src/main/java/org/apache/kylin/metadata/filter/CompareTupleFilter.java
----------------------------------------------------------------------
diff --git a/metadata/src/main/java/org/apache/kylin/metadata/filter/CompareTupleFilter.java b/metadata/src/main/java/org/apache/kylin/metadata/filter/CompareTupleFilter.java
index 73fd0fb..b9db9c4 100644
--- a/metadata/src/main/java/org/apache/kylin/metadata/filter/CompareTupleFilter.java
+++ b/metadata/src/main/java/org/apache/kylin/metadata/filter/CompareTupleFilter.java
@@ -81,7 +81,9 @@ public class CompareTupleFilter extends TupleFilter {
             }
         } else if (child instanceof ConstantTupleFilter) {
             this.conditionValues.addAll(child.getValues());
-            this.firstCondValue = this.conditionValues.iterator().next();
+            if (!this.conditionValues.isEmpty()) {
+                this.firstCondValue = this.conditionValues.iterator().next();
+            }
         } else if (child instanceof DynamicTupleFilter) {
             DynamicTupleFilter dynamicFilter = (DynamicTupleFilter) child;
             this.dynamicVariables.put(dynamicFilter.getVariableName(), null);

http://git-wip-us.apache.org/repos/asf/kylin/blob/0c13797e/query/src/main/java/org/apache/kylin/query/relnode/OLAPFilterRel.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/org/apache/kylin/query/relnode/OLAPFilterRel.java b/query/src/main/java/org/apache/kylin/query/relnode/OLAPFilterRel.java
index 667acf8..a2ab413 100644
--- a/query/src/main/java/org/apache/kylin/query/relnode/OLAPFilterRel.java
+++ b/query/src/main/java/org/apache/kylin/query/relnode/OLAPFilterRel.java
@@ -137,6 +137,7 @@ public class OLAPFilterRel extends Filter implements OLAPRel {
             List<? extends TupleFilter> children = filter.getChildren();
             TblColRef inColumn = null;
             List<String> inValues = new LinkedList<String>();
+            Map<String, String> dynamicVariable = new HashMap<>();
             for (TupleFilter child : children) {
                 if (child.getOperator() == FilterOperatorEnum.EQ) {
                     CompareTupleFilter compFilter = (CompareTupleFilter) child;
@@ -149,6 +150,7 @@ public class OLAPFilterRel extends Filter implements OLAPRel {
                         return null;
                     }
                     inValues.addAll(compFilter.getValues());
+                    dynamicVariable.putAll(compFilter.getVariables());
                 } else {
                     return null;
                 }
@@ -159,6 +161,7 @@ public class OLAPFilterRel extends Filter implements OLAPRel {
             CompareTupleFilter inFilter = new CompareTupleFilter(FilterOperatorEnum.IN);
             inFilter.addChild(new ColumnTupleFilter(inColumn));
             inFilter.addChild(new ConstantTupleFilter(inValues));
+            inFilter.getVariables().putAll(dynamicVariable);
             return inFilter;
         }
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/0c13797e/query/src/test/resources/query/sql_dynamic/query03.dat
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_dynamic/query03.dat b/query/src/test/resources/query/sql_dynamic/query03.dat
new file mode 100644
index 0000000..fc78596
--- /dev/null
+++ b/query/src/test/resources/query/sql_dynamic/query03.dat
@@ -0,0 +1,2 @@
+FP-GTC
+Others
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kylin/blob/0c13797e/query/src/test/resources/query/sql_dynamic/query03.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_dynamic/query03.sql b/query/src/test/resources/query/sql_dynamic/query03.sql
new file mode 100644
index 0000000..31d5d0b
--- /dev/null
+++ b/query/src/test/resources/query/sql_dynamic/query03.sql
@@ -0,0 +1,26 @@
+--
+-- 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.
+--
+
+select test_cal_dt.week_beg_dt,sum(test_kylin_fact.price) as GMV
+ , count(1) as TRANS_CNT, count(distinct leaf_categ_id) as LEAF_CATEG_CNT
+ from test_kylin_fact
+ inner JOIN edw.test_cal_dt as test_cal_dt
+ ON test_kylin_fact.cal_dt = test_cal_dt.cal_dt
+ where test_kylin_fact.lstg_format_name in (?, ?)
+ and test_cal_dt.week_beg_dt between DATE '2013-05-01' and DATE '2013-08-01'
+ group by test_cal_dt.week_beg_dt
\ No newline at end of file


[47/52] [abbrv] kylin git commit: minor, UI update tree control label style

Posted by li...@apache.org.
minor, UI update tree control label style


Project: http://git-wip-us.apache.org/repos/asf/kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/1caa626e
Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/1caa626e
Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/1caa626e

Branch: refs/heads/1.x-HBase1.1.3
Commit: 1caa626e88a82cd1451cd9abda615749a58ed787
Parents: 5666d7f
Author: janzhongi <ji...@ebay.com>
Authored: Thu Feb 18 11:25:31 2016 +0800
Committer: janzhongi <ji...@ebay.com>
Committed: Thu Feb 18 11:25:31 2016 +0800

----------------------------------------------------------------------
 webapp/app/less/app.less | 4 +---
 1 file changed, 1 insertion(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/1caa626e/webapp/app/less/app.less
----------------------------------------------------------------------
diff --git a/webapp/app/less/app.less b/webapp/app/less/app.less
index e80b757..bf2ba77 100644
--- a/webapp/app/less/app.less
+++ b/webapp/app/less/app.less
@@ -416,9 +416,7 @@ treecontrol > ul {
 treecontrol > ul > li {
   padding-left: 0;
 }
-treecontrol li .tree-label{
-  cursor: default;
-}
+
 treecontrol li treeitem .tree-label{
   cursor: pointer;
 }


[28/52] [abbrv] kylin git commit: KYLIN-1416 remove website from coding branch

Posted by li...@apache.org.
http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/assets/js/jquery-1.9.1.min.js
----------------------------------------------------------------------
diff --git a/website/assets/js/jquery-1.9.1.min.js b/website/assets/js/jquery-1.9.1.min.js
deleted file mode 100644
index 006e953..0000000
--- a/website/assets/js/jquery-1.9.1.min.js
+++ /dev/null
@@ -1,5 +0,0 @@
-/*! jQuery v1.9.1 | (c) 2005, 2012 jQuery Foundation, Inc. | jquery.org/license
-//@ sourceMappingURL=jquery.min.map
-*/(function(e,t){var n,r,i=typeof t,o=e.document,a=e.location,s=e.jQuery,u=e.$,l={},c=[],p="1.9.1",f=c.concat,d=c.push,h=c.slice,g=c.indexOf,m=l.toString,y=l.hasOwnProperty,v=p.trim,b=function(e,t){return new b.fn.init(e,t,r)},x=/[+-]?(?:\d*\.|)\d+(?:[eE][+-]?\d+|)/.source,w=/\S+/g,T=/^[\s\uFEFF\xA0]+|[\s\uFEFF\xA0]+$/g,N=/^(?:(<[\w\W]+>)[^>]*|#([\w-]*))$/,C=/^<(\w+)\s*\/?>(?:<\/\1>|)$/,k=/^[\],:{}\s]*$/,E=/(?:^|:|,)(?:\s*\[)+/g,S=/\\(?:["\\\/bfnrt]|u[\da-fA-F]{4})/g,A=/"[^"\\\r\n]*"|true|false|null|-?(?:\d+\.|)\d+(?:[eE][+-]?\d+|)/g,j=/^-ms-/,D=/-([\da-z])/gi,L=function(e,t){return t.toUpperCase()},H=function(e){(o.addEventListener||"load"===e.type||"complete"===o.readyState)&&(q(),b.ready())},q=function(){o.addEventListener?(o.removeEventListener("DOMContentLoaded",H,!1),e.removeEventListener("load",H,!1)):(o.detachEvent("onreadystatechange",H),e.detachEvent("onload",H))};b.fn=b.prototype={jquery:p,constructor:b,init:function(e,n,r){var i,a;if(!e)return this;if("string"==typeof e)
 {if(i="<"===e.charAt(0)&&">"===e.charAt(e.length-1)&&e.length>=3?[null,e,null]:N.exec(e),!i||!i[1]&&n)return!n||n.jquery?(n||r).find(e):this.constructor(n).find(e);if(i[1]){if(n=n instanceof b?n[0]:n,b.merge(this,b.parseHTML(i[1],n&&n.nodeType?n.ownerDocument||n:o,!0)),C.test(i[1])&&b.isPlainObject(n))for(i in n)b.isFunction(this[i])?this[i](n[i]):this.attr(i,n[i]);return this}if(a=o.getElementById(i[2]),a&&a.parentNode){if(a.id!==i[2])return r.find(e);this.length=1,this[0]=a}return this.context=o,this.selector=e,this}return e.nodeType?(this.context=this[0]=e,this.length=1,this):b.isFunction(e)?r.ready(e):(e.selector!==t&&(this.selector=e.selector,this.context=e.context),b.makeArray(e,this))},selector:"",length:0,size:function(){return this.length},toArray:function(){return h.call(this)},get:function(e){return null==e?this.toArray():0>e?this[this.length+e]:this[e]},pushStack:function(e){var t=b.merge(this.constructor(),e);return t.prevObject=this,t.context=this.context,t},each:funct
 ion(e,t){return b.each(this,e,t)},ready:function(e){return b.ready.promise().done(e),this},slice:function(){return this.pushStack(h.apply(this,arguments))},first:function(){return this.eq(0)},last:function(){return this.eq(-1)},eq:function(e){var t=this.length,n=+e+(0>e?t:0);return this.pushStack(n>=0&&t>n?[this[n]]:[])},map:function(e){return this.pushStack(b.map(this,function(t,n){return e.call(t,n,t)}))},end:function(){return this.prevObject||this.constructor(null)},push:d,sort:[].sort,splice:[].splice},b.fn.init.prototype=b.fn,b.extend=b.fn.extend=function(){var e,n,r,i,o,a,s=arguments[0]||{},u=1,l=arguments.length,c=!1;for("boolean"==typeof s&&(c=s,s=arguments[1]||{},u=2),"object"==typeof s||b.isFunction(s)||(s={}),l===u&&(s=this,--u);l>u;u++)if(null!=(o=arguments[u]))for(i in o)e=s[i],r=o[i],s!==r&&(c&&r&&(b.isPlainObject(r)||(n=b.isArray(r)))?(n?(n=!1,a=e&&b.isArray(e)?e:[]):a=e&&b.isPlainObject(e)?e:{},s[i]=b.extend(c,a,r)):r!==t&&(s[i]=r));return s},b.extend({noConflict:fun
 ction(t){return e.$===b&&(e.$=u),t&&e.jQuery===b&&(e.jQuery=s),b},isReady:!1,readyWait:1,holdReady:function(e){e?b.readyWait++:b.ready(!0)},ready:function(e){if(e===!0?!--b.readyWait:!b.isReady){if(!o.body)return setTimeout(b.ready);b.isReady=!0,e!==!0&&--b.readyWait>0||(n.resolveWith(o,[b]),b.fn.trigger&&b(o).trigger("ready").off("ready"))}},isFunction:function(e){return"function"===b.type(e)},isArray:Array.isArray||function(e){return"array"===b.type(e)},isWindow:function(e){return null!=e&&e==e.window},isNumeric:function(e){return!isNaN(parseFloat(e))&&isFinite(e)},type:function(e){return null==e?e+"":"object"==typeof e||"function"==typeof e?l[m.call(e)]||"object":typeof e},isPlainObject:function(e){if(!e||"object"!==b.type(e)||e.nodeType||b.isWindow(e))return!1;try{if(e.constructor&&!y.call(e,"constructor")&&!y.call(e.constructor.prototype,"isPrototypeOf"))return!1}catch(n){return!1}var r;for(r in e);return r===t||y.call(e,r)},isEmptyObject:function(e){var t;for(t in e)return!1;r
 eturn!0},error:function(e){throw Error(e)},parseHTML:function(e,t,n){if(!e||"string"!=typeof e)return null;"boolean"==typeof t&&(n=t,t=!1),t=t||o;var r=C.exec(e),i=!n&&[];return r?[t.createElement(r[1])]:(r=b.buildFragment([e],t,i),i&&b(i).remove(),b.merge([],r.childNodes))},parseJSON:function(n){return e.JSON&&e.JSON.parse?e.JSON.parse(n):null===n?n:"string"==typeof n&&(n=b.trim(n),n&&k.test(n.replace(S,"@").replace(A,"]").replace(E,"")))?Function("return "+n)():(b.error("Invalid JSON: "+n),t)},parseXML:function(n){var r,i;if(!n||"string"!=typeof n)return null;try{e.DOMParser?(i=new DOMParser,r=i.parseFromString(n,"text/xml")):(r=new ActiveXObject("Microsoft.XMLDOM"),r.async="false",r.loadXML(n))}catch(o){r=t}return r&&r.documentElement&&!r.getElementsByTagName("parsererror").length||b.error("Invalid XML: "+n),r},noop:function(){},globalEval:function(t){t&&b.trim(t)&&(e.execScript||function(t){e.eval.call(e,t)})(t)},camelCase:function(e){return e.replace(j,"ms-").replace(D,L)},node
 Name:function(e,t){return e.nodeName&&e.nodeName.toLowerCase()===t.toLowerCase()},each:function(e,t,n){var r,i=0,o=e.length,a=M(e);if(n){if(a){for(;o>i;i++)if(r=t.apply(e[i],n),r===!1)break}else for(i in e)if(r=t.apply(e[i],n),r===!1)break}else if(a){for(;o>i;i++)if(r=t.call(e[i],i,e[i]),r===!1)break}else for(i in e)if(r=t.call(e[i],i,e[i]),r===!1)break;return e},trim:v&&!v.call("\ufeff\u00a0")?function(e){return null==e?"":v.call(e)}:function(e){return null==e?"":(e+"").replace(T,"")},makeArray:function(e,t){var n=t||[];return null!=e&&(M(Object(e))?b.merge(n,"string"==typeof e?[e]:e):d.call(n,e)),n},inArray:function(e,t,n){var r;if(t){if(g)return g.call(t,e,n);for(r=t.length,n=n?0>n?Math.max(0,r+n):n:0;r>n;n++)if(n in t&&t[n]===e)return n}return-1},merge:function(e,n){var r=n.length,i=e.length,o=0;if("number"==typeof r)for(;r>o;o++)e[i++]=n[o];else while(n[o]!==t)e[i++]=n[o++];return e.length=i,e},grep:function(e,t,n){var r,i=[],o=0,a=e.length;for(n=!!n;a>o;o++)r=!!t(e[o],o),n!==r
 &&i.push(e[o]);return i},map:function(e,t,n){var r,i=0,o=e.length,a=M(e),s=[];if(a)for(;o>i;i++)r=t(e[i],i,n),null!=r&&(s[s.length]=r);else for(i in e)r=t(e[i],i,n),null!=r&&(s[s.length]=r);return f.apply([],s)},guid:1,proxy:function(e,n){var r,i,o;return"string"==typeof n&&(o=e[n],n=e,e=o),b.isFunction(e)?(r=h.call(arguments,2),i=function(){return e.apply(n||this,r.concat(h.call(arguments)))},i.guid=e.guid=e.guid||b.guid++,i):t},access:function(e,n,r,i,o,a,s){var u=0,l=e.length,c=null==r;if("object"===b.type(r)){o=!0;for(u in r)b.access(e,n,u,r[u],!0,a,s)}else if(i!==t&&(o=!0,b.isFunction(i)||(s=!0),c&&(s?(n.call(e,i),n=null):(c=n,n=function(e,t,n){return c.call(b(e),n)})),n))for(;l>u;u++)n(e[u],r,s?i:i.call(e[u],u,n(e[u],r)));return o?e:c?n.call(e):l?n(e[0],r):a},now:function(){return(new Date).getTime()}}),b.ready.promise=function(t){if(!n)if(n=b.Deferred(),"complete"===o.readyState)setTimeout(b.ready);else if(o.addEventListener)o.addEventListener("DOMContentLoaded",H,!1),e.addEv
 entListener("load",H,!1);else{o.attachEvent("onreadystatechange",H),e.attachEvent("onload",H);var r=!1;try{r=null==e.frameElement&&o.documentElement}catch(i){}r&&r.doScroll&&function a(){if(!b.isReady){try{r.doScroll("left")}catch(e){return setTimeout(a,50)}q(),b.ready()}}()}return n.promise(t)},b.each("Boolean Number String Function Array Date RegExp Object Error".split(" "),function(e,t){l["[object "+t+"]"]=t.toLowerCase()});function M(e){var t=e.length,n=b.type(e);return b.isWindow(e)?!1:1===e.nodeType&&t?!0:"array"===n||"function"!==n&&(0===t||"number"==typeof t&&t>0&&t-1 in e)}r=b(o);var _={};function F(e){var t=_[e]={};return b.each(e.match(w)||[],function(e,n){t[n]=!0}),t}b.Callbacks=function(e){e="string"==typeof e?_[e]||F(e):b.extend({},e);var n,r,i,o,a,s,u=[],l=!e.once&&[],c=function(t){for(r=e.memory&&t,i=!0,a=s||0,s=0,o=u.length,n=!0;u&&o>a;a++)if(u[a].apply(t[0],t[1])===!1&&e.stopOnFalse){r=!1;break}n=!1,u&&(l?l.length&&c(l.shift()):r?u=[]:p.disable())},p={add:function(
 ){if(u){var t=u.length;(function i(t){b.each(t,function(t,n){var r=b.type(n);"function"===r?e.unique&&p.has(n)||u.push(n):n&&n.length&&"string"!==r&&i(n)})})(arguments),n?o=u.length:r&&(s=t,c(r))}return this},remove:function(){return u&&b.each(arguments,function(e,t){var r;while((r=b.inArray(t,u,r))>-1)u.splice(r,1),n&&(o>=r&&o--,a>=r&&a--)}),this},has:function(e){return e?b.inArray(e,u)>-1:!(!u||!u.length)},empty:function(){return u=[],this},disable:function(){return u=l=r=t,this},disabled:function(){return!u},lock:function(){return l=t,r||p.disable(),this},locked:function(){return!l},fireWith:function(e,t){return t=t||[],t=[e,t.slice?t.slice():t],!u||i&&!l||(n?l.push(t):c(t)),this},fire:function(){return p.fireWith(this,arguments),this},fired:function(){return!!i}};return p},b.extend({Deferred:function(e){var t=[["resolve","done",b.Callbacks("once memory"),"resolved"],["reject","fail",b.Callbacks("once memory"),"rejected"],["notify","progress",b.Callbacks("memory")]],n="pending",r
 ={state:function(){return n},always:function(){return i.done(arguments).fail(arguments),this},then:function(){var e=arguments;return b.Deferred(function(n){b.each(t,function(t,o){var a=o[0],s=b.isFunction(e[t])&&e[t];i[o[1]](function(){var e=s&&s.apply(this,arguments);e&&b.isFunction(e.promise)?e.promise().done(n.resolve).fail(n.reject).progress(n.notify):n[a+"With"](this===r?n.promise():this,s?[e]:arguments)})}),e=null}).promise()},promise:function(e){return null!=e?b.extend(e,r):r}},i={};return r.pipe=r.then,b.each(t,function(e,o){var a=o[2],s=o[3];r[o[1]]=a.add,s&&a.add(function(){n=s},t[1^e][2].disable,t[2][2].lock),i[o[0]]=function(){return i[o[0]+"With"](this===i?r:this,arguments),this},i[o[0]+"With"]=a.fireWith}),r.promise(i),e&&e.call(i,i),i},when:function(e){var t=0,n=h.call(arguments),r=n.length,i=1!==r||e&&b.isFunction(e.promise)?r:0,o=1===i?e:b.Deferred(),a=function(e,t,n){return function(r){t[e]=this,n[e]=arguments.length>1?h.call(arguments):r,n===s?o.notifyWith(t,n):--
 i||o.resolveWith(t,n)}},s,u,l;if(r>1)for(s=Array(r),u=Array(r),l=Array(r);r>t;t++)n[t]&&b.isFunction(n[t].promise)?n[t].promise().done(a(t,l,n)).fail(o.reject).progress(a(t,u,s)):--i;return i||o.resolveWith(l,n),o.promise()}}),b.support=function(){var t,n,r,a,s,u,l,c,p,f,d=o.createElement("div");if(d.setAttribute("className","t"),d.innerHTML="  <link/><table></table><a href='/a'>a</a><input type='checkbox'/>",n=d.getElementsByTagName("*"),r=d.getElementsByTagName("a")[0],!n||!r||!n.length)return{};s=o.createElement("select"),l=s.appendChild(o.createElement("option")),a=d.getElementsByTagName("input")[0],r.style.cssText="top:1px;float:left;opacity:.5",t={getSetAttribute:"t"!==d.className,leadingWhitespace:3===d.firstChild.nodeType,tbody:!d.getElementsByTagName("tbody").length,htmlSerialize:!!d.getElementsByTagName("link").length,style:/top/.test(r.getAttribute("style")),hrefNormalized:"/a"===r.getAttribute("href"),opacity:/^0.5/.test(r.style.opacity),cssFloat:!!r.style.cssFloat,check
 On:!!a.value,optSelected:l.selected,enctype:!!o.createElement("form").enctype,html5Clone:"<:nav></:nav>"!==o.createElement("nav").cloneNode(!0).outerHTML,boxModel:"CSS1Compat"===o.compatMode,deleteExpando:!0,noCloneEvent:!0,inlineBlockNeedsLayout:!1,shrinkWrapBlocks:!1,reliableMarginRight:!0,boxSizingReliable:!0,pixelPosition:!1},a.checked=!0,t.noCloneChecked=a.cloneNode(!0).checked,s.disabled=!0,t.optDisabled=!l.disabled;try{delete d.test}catch(h){t.deleteExpando=!1}a=o.createElement("input"),a.setAttribute("value",""),t.input=""===a.getAttribute("value"),a.value="t",a.setAttribute("type","radio"),t.radioValue="t"===a.value,a.setAttribute("checked","t"),a.setAttribute("name","t"),u=o.createDocumentFragment(),u.appendChild(a),t.appendChecked=a.checked,t.checkClone=u.cloneNode(!0).cloneNode(!0).lastChild.checked,d.attachEvent&&(d.attachEvent("onclick",function(){t.noCloneEvent=!1}),d.cloneNode(!0).click());for(f in{submit:!0,change:!0,focusin:!0})d.setAttribute(c="on"+f,"t"),t[f+"Bub
 bles"]=c in e||d.attributes[c].expando===!1;return d.style.backgroundClip="content-box",d.cloneNode(!0).style.backgroundClip="",t.clearCloneStyle="content-box"===d.style.backgroundClip,b(function(){var n,r,a,s="padding:0;margin:0;border:0;display:block;box-sizing:content-box;-moz-box-sizing:content-box;-webkit-box-sizing:content-box;",u=o.getElementsByTagName("body")[0];u&&(n=o.createElement("div"),n.style.cssText="border:0;width:0;height:0;position:absolute;top:0;left:-9999px;margin-top:1px",u.appendChild(n).appendChild(d),d.innerHTML="<table><tr><td></td><td>t</td></tr></table>",a=d.getElementsByTagName("td"),a[0].style.cssText="padding:0;margin:0;border:0;display:none",p=0===a[0].offsetHeight,a[0].style.display="",a[1].style.display="none",t.reliableHiddenOffsets=p&&0===a[0].offsetHeight,d.innerHTML="",d.style.cssText="box-sizing:border-box;-moz-box-sizing:border-box;-webkit-box-sizing:border-box;padding:1px;border:1px;display:block;width:4px;margin-top:1%;position:absolute;top:1
 %;",t.boxSizing=4===d.offsetWidth,t.doesNotIncludeMarginInBodyOffset=1!==u.offsetTop,e.getComputedStyle&&(t.pixelPosition="1%"!==(e.getComputedStyle(d,null)||{}).top,t.boxSizingReliable="4px"===(e.getComputedStyle(d,null)||{width:"4px"}).width,r=d.appendChild(o.createElement("div")),r.style.cssText=d.style.cssText=s,r.style.marginRight=r.style.width="0",d.style.width="1px",t.reliableMarginRight=!parseFloat((e.getComputedStyle(r,null)||{}).marginRight)),typeof d.style.zoom!==i&&(d.innerHTML="",d.style.cssText=s+"width:1px;padding:1px;display:inline;zoom:1",t.inlineBlockNeedsLayout=3===d.offsetWidth,d.style.display="block",d.innerHTML="<div></div>",d.firstChild.style.width="5px",t.shrinkWrapBlocks=3!==d.offsetWidth,t.inlineBlockNeedsLayout&&(u.style.zoom=1)),u.removeChild(n),n=d=a=r=null)}),n=s=u=l=r=a=null,t}();var O=/(?:\{[\s\S]*\}|\[[\s\S]*\])$/,B=/([A-Z])/g;function P(e,n,r,i){if(b.acceptData(e)){var o,a,s=b.expando,u="string"==typeof n,l=e.nodeType,p=l?b.cache:e,f=l?e[s]:e[s]&&s;
 if(f&&p[f]&&(i||p[f].data)||!u||r!==t)return f||(l?e[s]=f=c.pop()||b.guid++:f=s),p[f]||(p[f]={},l||(p[f].toJSON=b.noop)),("object"==typeof n||"function"==typeof n)&&(i?p[f]=b.extend(p[f],n):p[f].data=b.extend(p[f].data,n)),o=p[f],i||(o.data||(o.data={}),o=o.data),r!==t&&(o[b.camelCase(n)]=r),u?(a=o[n],null==a&&(a=o[b.camelCase(n)])):a=o,a}}function R(e,t,n){if(b.acceptData(e)){var r,i,o,a=e.nodeType,s=a?b.cache:e,u=a?e[b.expando]:b.expando;if(s[u]){if(t&&(o=n?s[u]:s[u].data)){b.isArray(t)?t=t.concat(b.map(t,b.camelCase)):t in o?t=[t]:(t=b.camelCase(t),t=t in o?[t]:t.split(" "));for(r=0,i=t.length;i>r;r++)delete o[t[r]];if(!(n?$:b.isEmptyObject)(o))return}(n||(delete s[u].data,$(s[u])))&&(a?b.cleanData([e],!0):b.support.deleteExpando||s!=s.window?delete s[u]:s[u]=null)}}}b.extend({cache:{},expando:"jQuery"+(p+Math.random()).replace(/\D/g,""),noData:{embed:!0,object:"clsid:D27CDB6E-AE6D-11cf-96B8-444553540000",applet:!0},hasData:function(e){return e=e.nodeType?b.cache[e[b.expando]]:e[
 b.expando],!!e&&!$(e)},data:function(e,t,n){return P(e,t,n)},removeData:function(e,t){return R(e,t)},_data:function(e,t,n){return P(e,t,n,!0)},_removeData:function(e,t){return R(e,t,!0)},acceptData:function(e){if(e.nodeType&&1!==e.nodeType&&9!==e.nodeType)return!1;var t=e.nodeName&&b.noData[e.nodeName.toLowerCase()];return!t||t!==!0&&e.getAttribute("classid")===t}}),b.fn.extend({data:function(e,n){var r,i,o=this[0],a=0,s=null;if(e===t){if(this.length&&(s=b.data(o),1===o.nodeType&&!b._data(o,"parsedAttrs"))){for(r=o.attributes;r.length>a;a++)i=r[a].name,i.indexOf("data-")||(i=b.camelCase(i.slice(5)),W(o,i,s[i]));b._data(o,"parsedAttrs",!0)}return s}return"object"==typeof e?this.each(function(){b.data(this,e)}):b.access(this,function(n){return n===t?o?W(o,e,b.data(o,e)):null:(this.each(function(){b.data(this,e,n)}),t)},null,n,arguments.length>1,null,!0)},removeData:function(e){return this.each(function(){b.removeData(this,e)})}});function W(e,n,r){if(r===t&&1===e.nodeType){var i="data
 -"+n.replace(B,"-$1").toLowerCase();if(r=e.getAttribute(i),"string"==typeof r){try{r="true"===r?!0:"false"===r?!1:"null"===r?null:+r+""===r?+r:O.test(r)?b.parseJSON(r):r}catch(o){}b.data(e,n,r)}else r=t}return r}function $(e){var t;for(t in e)if(("data"!==t||!b.isEmptyObject(e[t]))&&"toJSON"!==t)return!1;return!0}b.extend({queue:function(e,n,r){var i;return e?(n=(n||"fx")+"queue",i=b._data(e,n),r&&(!i||b.isArray(r)?i=b._data(e,n,b.makeArray(r)):i.push(r)),i||[]):t},dequeue:function(e,t){t=t||"fx";var n=b.queue(e,t),r=n.length,i=n.shift(),o=b._queueHooks(e,t),a=function(){b.dequeue(e,t)};"inprogress"===i&&(i=n.shift(),r--),o.cur=i,i&&("fx"===t&&n.unshift("inprogress"),delete o.stop,i.call(e,a,o)),!r&&o&&o.empty.fire()},_queueHooks:function(e,t){var n=t+"queueHooks";return b._data(e,n)||b._data(e,n,{empty:b.Callbacks("once memory").add(function(){b._removeData(e,t+"queue"),b._removeData(e,n)})})}}),b.fn.extend({queue:function(e,n){var r=2;return"string"!=typeof e&&(n=e,e="fx",r--),r>a
 rguments.length?b.queue(this[0],e):n===t?this:this.each(function(){var t=b.queue(this,e,n);b._queueHooks(this,e),"fx"===e&&"inprogress"!==t[0]&&b.dequeue(this,e)})},dequeue:function(e){return this.each(function(){b.dequeue(this,e)})},delay:function(e,t){return e=b.fx?b.fx.speeds[e]||e:e,t=t||"fx",this.queue(t,function(t,n){var r=setTimeout(t,e);n.stop=function(){clearTimeout(r)}})},clearQueue:function(e){return this.queue(e||"fx",[])},promise:function(e,n){var r,i=1,o=b.Deferred(),a=this,s=this.length,u=function(){--i||o.resolveWith(a,[a])};"string"!=typeof e&&(n=e,e=t),e=e||"fx";while(s--)r=b._data(a[s],e+"queueHooks"),r&&r.empty&&(i++,r.empty.add(u));return u(),o.promise(n)}});var I,z,X=/[\t\r\n]/g,U=/\r/g,V=/^(?:input|select|textarea|button|object)$/i,Y=/^(?:a|area)$/i,J=/^(?:checked|selected|autofocus|autoplay|async|controls|defer|disabled|hidden|loop|multiple|open|readonly|required|scoped)$/i,G=/^(?:checked|selected)$/i,Q=b.support.getSetAttribute,K=b.support.input;b.fn.extend(
 {attr:function(e,t){return b.access(this,b.attr,e,t,arguments.length>1)},removeAttr:function(e){return this.each(function(){b.removeAttr(this,e)})},prop:function(e,t){return b.access(this,b.prop,e,t,arguments.length>1)},removeProp:function(e){return e=b.propFix[e]||e,this.each(function(){try{this[e]=t,delete this[e]}catch(n){}})},addClass:function(e){var t,n,r,i,o,a=0,s=this.length,u="string"==typeof e&&e;if(b.isFunction(e))return this.each(function(t){b(this).addClass(e.call(this,t,this.className))});if(u)for(t=(e||"").match(w)||[];s>a;a++)if(n=this[a],r=1===n.nodeType&&(n.className?(" "+n.className+" ").replace(X," "):" ")){o=0;while(i=t[o++])0>r.indexOf(" "+i+" ")&&(r+=i+" ");n.className=b.trim(r)}return this},removeClass:function(e){var t,n,r,i,o,a=0,s=this.length,u=0===arguments.length||"string"==typeof e&&e;if(b.isFunction(e))return this.each(function(t){b(this).removeClass(e.call(this,t,this.className))});if(u)for(t=(e||"").match(w)||[];s>a;a++)if(n=this[a],r=1===n.nodeType&&
 (n.className?(" "+n.className+" ").replace(X," "):"")){o=0;while(i=t[o++])while(r.indexOf(" "+i+" ")>=0)r=r.replace(" "+i+" "," ");n.className=e?b.trim(r):""}return this},toggleClass:function(e,t){var n=typeof e,r="boolean"==typeof t;return b.isFunction(e)?this.each(function(n){b(this).toggleClass(e.call(this,n,this.className,t),t)}):this.each(function(){if("string"===n){var o,a=0,s=b(this),u=t,l=e.match(w)||[];while(o=l[a++])u=r?u:!s.hasClass(o),s[u?"addClass":"removeClass"](o)}else(n===i||"boolean"===n)&&(this.className&&b._data(this,"__className__",this.className),this.className=this.className||e===!1?"":b._data(this,"__className__")||"")})},hasClass:function(e){var t=" "+e+" ",n=0,r=this.length;for(;r>n;n++)if(1===this[n].nodeType&&(" "+this[n].className+" ").replace(X," ").indexOf(t)>=0)return!0;return!1},val:function(e){var n,r,i,o=this[0];{if(arguments.length)return i=b.isFunction(e),this.each(function(n){var o,a=b(this);1===this.nodeType&&(o=i?e.call(this,n,a.val()):e,null==
 o?o="":"number"==typeof o?o+="":b.isArray(o)&&(o=b.map(o,function(e){return null==e?"":e+""})),r=b.valHooks[this.type]||b.valHooks[this.nodeName.toLowerCase()],r&&"set"in r&&r.set(this,o,"value")!==t||(this.value=o))});if(o)return r=b.valHooks[o.type]||b.valHooks[o.nodeName.toLowerCase()],r&&"get"in r&&(n=r.get(o,"value"))!==t?n:(n=o.value,"string"==typeof n?n.replace(U,""):null==n?"":n)}}}),b.extend({valHooks:{option:{get:function(e){var t=e.attributes.value;return!t||t.specified?e.value:e.text}},select:{get:function(e){var t,n,r=e.options,i=e.selectedIndex,o="select-one"===e.type||0>i,a=o?null:[],s=o?i+1:r.length,u=0>i?s:o?i:0;for(;s>u;u++)if(n=r[u],!(!n.selected&&u!==i||(b.support.optDisabled?n.disabled:null!==n.getAttribute("disabled"))||n.parentNode.disabled&&b.nodeName(n.parentNode,"optgroup"))){if(t=b(n).val(),o)return t;a.push(t)}return a},set:function(e,t){var n=b.makeArray(t);return b(e).find("option").each(function(){this.selected=b.inArray(b(this).val(),n)>=0}),n.length|
 |(e.selectedIndex=-1),n}}},attr:function(e,n,r){var o,a,s,u=e.nodeType;if(e&&3!==u&&8!==u&&2!==u)return typeof e.getAttribute===i?b.prop(e,n,r):(a=1!==u||!b.isXMLDoc(e),a&&(n=n.toLowerCase(),o=b.attrHooks[n]||(J.test(n)?z:I)),r===t?o&&a&&"get"in o&&null!==(s=o.get(e,n))?s:(typeof e.getAttribute!==i&&(s=e.getAttribute(n)),null==s?t:s):null!==r?o&&a&&"set"in o&&(s=o.set(e,r,n))!==t?s:(e.setAttribute(n,r+""),r):(b.removeAttr(e,n),t))},removeAttr:function(e,t){var n,r,i=0,o=t&&t.match(w);if(o&&1===e.nodeType)while(n=o[i++])r=b.propFix[n]||n,J.test(n)?!Q&&G.test(n)?e[b.camelCase("default-"+n)]=e[r]=!1:e[r]=!1:b.attr(e,n,""),e.removeAttribute(Q?n:r)},attrHooks:{type:{set:function(e,t){if(!b.support.radioValue&&"radio"===t&&b.nodeName(e,"input")){var n=e.value;return e.setAttribute("type",t),n&&(e.value=n),t}}}},propFix:{tabindex:"tabIndex",readonly:"readOnly","for":"htmlFor","class":"className",maxlength:"maxLength",cellspacing:"cellSpacing",cellpadding:"cellPadding",rowspan:"rowSpan",col
 span:"colSpan",usemap:"useMap",frameborder:"frameBorder",contenteditable:"contentEditable"},prop:function(e,n,r){var i,o,a,s=e.nodeType;if(e&&3!==s&&8!==s&&2!==s)return a=1!==s||!b.isXMLDoc(e),a&&(n=b.propFix[n]||n,o=b.propHooks[n]),r!==t?o&&"set"in o&&(i=o.set(e,r,n))!==t?i:e[n]=r:o&&"get"in o&&null!==(i=o.get(e,n))?i:e[n]},propHooks:{tabIndex:{get:function(e){var n=e.getAttributeNode("tabindex");return n&&n.specified?parseInt(n.value,10):V.test(e.nodeName)||Y.test(e.nodeName)&&e.href?0:t}}}}),z={get:function(e,n){var r=b.prop(e,n),i="boolean"==typeof r&&e.getAttribute(n),o="boolean"==typeof r?K&&Q?null!=i:G.test(n)?e[b.camelCase("default-"+n)]:!!i:e.getAttributeNode(n);return o&&o.value!==!1?n.toLowerCase():t},set:function(e,t,n){return t===!1?b.removeAttr(e,n):K&&Q||!G.test(n)?e.setAttribute(!Q&&b.propFix[n]||n,n):e[b.camelCase("default-"+n)]=e[n]=!0,n}},K&&Q||(b.attrHooks.value={get:function(e,n){var r=e.getAttributeNode(n);return b.nodeName(e,"input")?e.defaultValue:r&&r.specif
 ied?r.value:t},set:function(e,n,r){return b.nodeName(e,"input")?(e.defaultValue=n,t):I&&I.set(e,n,r)}}),Q||(I=b.valHooks.button={get:function(e,n){var r=e.getAttributeNode(n);return r&&("id"===n||"name"===n||"coords"===n?""!==r.value:r.specified)?r.value:t},set:function(e,n,r){var i=e.getAttributeNode(r);return i||e.setAttributeNode(i=e.ownerDocument.createAttribute(r)),i.value=n+="","value"===r||n===e.getAttribute(r)?n:t}},b.attrHooks.contenteditable={get:I.get,set:function(e,t,n){I.set(e,""===t?!1:t,n)}},b.each(["width","height"],function(e,n){b.attrHooks[n]=b.extend(b.attrHooks[n],{set:function(e,r){return""===r?(e.setAttribute(n,"auto"),r):t}})})),b.support.hrefNormalized||(b.each(["href","src","width","height"],function(e,n){b.attrHooks[n]=b.extend(b.attrHooks[n],{get:function(e){var r=e.getAttribute(n,2);return null==r?t:r}})}),b.each(["href","src"],function(e,t){b.propHooks[t]={get:function(e){return e.getAttribute(t,4)}}})),b.support.style||(b.attrHooks.style={get:function(e
 ){return e.style.cssText||t},set:function(e,t){return e.style.cssText=t+""}}),b.support.optSelected||(b.propHooks.selected=b.extend(b.propHooks.selected,{get:function(e){var t=e.parentNode;return t&&(t.selectedIndex,t.parentNode&&t.parentNode.selectedIndex),null}})),b.support.enctype||(b.propFix.enctype="encoding"),b.support.checkOn||b.each(["radio","checkbox"],function(){b.valHooks[this]={get:function(e){return null===e.getAttribute("value")?"on":e.value}}}),b.each(["radio","checkbox"],function(){b.valHooks[this]=b.extend(b.valHooks[this],{set:function(e,n){return b.isArray(n)?e.checked=b.inArray(b(e).val(),n)>=0:t}})});var Z=/^(?:input|select|textarea)$/i,et=/^key/,tt=/^(?:mouse|contextmenu)|click/,nt=/^(?:focusinfocus|focusoutblur)$/,rt=/^([^.]*)(?:\.(.+)|)$/;function it(){return!0}function ot(){return!1}b.event={global:{},add:function(e,n,r,o,a){var s,u,l,c,p,f,d,h,g,m,y,v=b._data(e);if(v){r.handler&&(c=r,r=c.handler,a=c.selector),r.guid||(r.guid=b.guid++),(u=v.events)||(u=v.eve
 nts={}),(f=v.handle)||(f=v.handle=function(e){return typeof b===i||e&&b.event.triggered===e.type?t:b.event.dispatch.apply(f.elem,arguments)},f.elem=e),n=(n||"").match(w)||[""],l=n.length;while(l--)s=rt.exec(n[l])||[],g=y=s[1],m=(s[2]||"").split(".").sort(),p=b.event.special[g]||{},g=(a?p.delegateType:p.bindType)||g,p=b.event.special[g]||{},d=b.extend({type:g,origType:y,data:o,handler:r,guid:r.guid,selector:a,needsContext:a&&b.expr.match.needsContext.test(a),namespace:m.join(".")},c),(h=u[g])||(h=u[g]=[],h.delegateCount=0,p.setup&&p.setup.call(e,o,m,f)!==!1||(e.addEventListener?e.addEventListener(g,f,!1):e.attachEvent&&e.attachEvent("on"+g,f))),p.add&&(p.add.call(e,d),d.handler.guid||(d.handler.guid=r.guid)),a?h.splice(h.delegateCount++,0,d):h.push(d),b.event.global[g]=!0;e=null}},remove:function(e,t,n,r,i){var o,a,s,u,l,c,p,f,d,h,g,m=b.hasData(e)&&b._data(e);if(m&&(c=m.events)){t=(t||"").match(w)||[""],l=t.length;while(l--)if(s=rt.exec(t[l])||[],d=g=s[1],h=(s[2]||"").split(".").sort
 (),d){p=b.event.special[d]||{},d=(r?p.delegateType:p.bindType)||d,f=c[d]||[],s=s[2]&&RegExp("(^|\\.)"+h.join("\\.(?:.*\\.|)")+"(\\.|$)"),u=o=f.length;while(o--)a=f[o],!i&&g!==a.origType||n&&n.guid!==a.guid||s&&!s.test(a.namespace)||r&&r!==a.selector&&("**"!==r||!a.selector)||(f.splice(o,1),a.selector&&f.delegateCount--,p.remove&&p.remove.call(e,a));u&&!f.length&&(p.teardown&&p.teardown.call(e,h,m.handle)!==!1||b.removeEvent(e,d,m.handle),delete c[d])}else for(d in c)b.event.remove(e,d+t[l],n,r,!0);b.isEmptyObject(c)&&(delete m.handle,b._removeData(e,"events"))}},trigger:function(n,r,i,a){var s,u,l,c,p,f,d,h=[i||o],g=y.call(n,"type")?n.type:n,m=y.call(n,"namespace")?n.namespace.split("."):[];if(l=f=i=i||o,3!==i.nodeType&&8!==i.nodeType&&!nt.test(g+b.event.triggered)&&(g.indexOf(".")>=0&&(m=g.split("."),g=m.shift(),m.sort()),u=0>g.indexOf(":")&&"on"+g,n=n[b.expando]?n:new b.Event(g,"object"==typeof n&&n),n.isTrigger=!0,n.namespace=m.join("."),n.namespace_re=n.namespace?RegExp("(^|\\.)
 "+m.join("\\.(?:.*\\.|)")+"(\\.|$)"):null,n.result=t,n.target||(n.target=i),r=null==r?[n]:b.makeArray(r,[n]),p=b.event.special[g]||{},a||!p.trigger||p.trigger.apply(i,r)!==!1)){if(!a&&!p.noBubble&&!b.isWindow(i)){for(c=p.delegateType||g,nt.test(c+g)||(l=l.parentNode);l;l=l.parentNode)h.push(l),f=l;f===(i.ownerDocument||o)&&h.push(f.defaultView||f.parentWindow||e)}d=0;while((l=h[d++])&&!n.isPropagationStopped())n.type=d>1?c:p.bindType||g,s=(b._data(l,"events")||{})[n.type]&&b._data(l,"handle"),s&&s.apply(l,r),s=u&&l[u],s&&b.acceptData(l)&&s.apply&&s.apply(l,r)===!1&&n.preventDefault();if(n.type=g,!(a||n.isDefaultPrevented()||p._default&&p._default.apply(i.ownerDocument,r)!==!1||"click"===g&&b.nodeName(i,"a")||!b.acceptData(i)||!u||!i[g]||b.isWindow(i))){f=i[u],f&&(i[u]=null),b.event.triggered=g;try{i[g]()}catch(v){}b.event.triggered=t,f&&(i[u]=f)}return n.result}},dispatch:function(e){e=b.event.fix(e);var n,r,i,o,a,s=[],u=h.call(arguments),l=(b._data(this,"events")||{})[e.type]||[],c
 =b.event.special[e.type]||{};if(u[0]=e,e.delegateTarget=this,!c.preDispatch||c.preDispatch.call(this,e)!==!1){s=b.event.handlers.call(this,e,l),n=0;while((o=s[n++])&&!e.isPropagationStopped()){e.currentTarget=o.elem,a=0;while((i=o.handlers[a++])&&!e.isImmediatePropagationStopped())(!e.namespace_re||e.namespace_re.test(i.namespace))&&(e.handleObj=i,e.data=i.data,r=((b.event.special[i.origType]||{}).handle||i.handler).apply(o.elem,u),r!==t&&(e.result=r)===!1&&(e.preventDefault(),e.stopPropagation()))}return c.postDispatch&&c.postDispatch.call(this,e),e.result}},handlers:function(e,n){var r,i,o,a,s=[],u=n.delegateCount,l=e.target;if(u&&l.nodeType&&(!e.button||"click"!==e.type))for(;l!=this;l=l.parentNode||this)if(1===l.nodeType&&(l.disabled!==!0||"click"!==e.type)){for(o=[],a=0;u>a;a++)i=n[a],r=i.selector+" ",o[r]===t&&(o[r]=i.needsContext?b(r,this).index(l)>=0:b.find(r,this,null,[l]).length),o[r]&&o.push(i);o.length&&s.push({elem:l,handlers:o})}return n.length>u&&s.push({elem:this,han
 dlers:n.slice(u)}),s},fix:function(e){if(e[b.expando])return e;var t,n,r,i=e.type,a=e,s=this.fixHooks[i];s||(this.fixHooks[i]=s=tt.test(i)?this.mouseHooks:et.test(i)?this.keyHooks:{}),r=s.props?this.props.concat(s.props):this.props,e=new b.Event(a),t=r.length;while(t--)n=r[t],e[n]=a[n];return e.target||(e.target=a.srcElement||o),3===e.target.nodeType&&(e.target=e.target.parentNode),e.metaKey=!!e.metaKey,s.filter?s.filter(e,a):e},props:"altKey bubbles cancelable ctrlKey currentTarget eventPhase metaKey relatedTarget shiftKey target timeStamp view which".split(" "),fixHooks:{},keyHooks:{props:"char charCode key keyCode".split(" "),filter:function(e,t){return null==e.which&&(e.which=null!=t.charCode?t.charCode:t.keyCode),e}},mouseHooks:{props:"button buttons clientX clientY fromElement offsetX offsetY pageX pageY screenX screenY toElement".split(" "),filter:function(e,n){var r,i,a,s=n.button,u=n.fromElement;return null==e.pageX&&null!=n.clientX&&(i=e.target.ownerDocument||o,a=i.documen
 tElement,r=i.body,e.pageX=n.clientX+(a&&a.scrollLeft||r&&r.scrollLeft||0)-(a&&a.clientLeft||r&&r.clientLeft||0),e.pageY=n.clientY+(a&&a.scrollTop||r&&r.scrollTop||0)-(a&&a.clientTop||r&&r.clientTop||0)),!e.relatedTarget&&u&&(e.relatedTarget=u===e.target?n.toElement:u),e.which||s===t||(e.which=1&s?1:2&s?3:4&s?2:0),e}},special:{load:{noBubble:!0},click:{trigger:function(){return b.nodeName(this,"input")&&"checkbox"===this.type&&this.click?(this.click(),!1):t}},focus:{trigger:function(){if(this!==o.activeElement&&this.focus)try{return this.focus(),!1}catch(e){}},delegateType:"focusin"},blur:{trigger:function(){return this===o.activeElement&&this.blur?(this.blur(),!1):t},delegateType:"focusout"},beforeunload:{postDispatch:function(e){e.result!==t&&(e.originalEvent.returnValue=e.result)}}},simulate:function(e,t,n,r){var i=b.extend(new b.Event,n,{type:e,isSimulated:!0,originalEvent:{}});r?b.event.trigger(i,null,t):b.event.dispatch.call(t,i),i.isDefaultPrevented()&&n.preventDefault()}},b.r
 emoveEvent=o.removeEventListener?function(e,t,n){e.removeEventListener&&e.removeEventListener(t,n,!1)}:function(e,t,n){var r="on"+t;e.detachEvent&&(typeof e[r]===i&&(e[r]=null),e.detachEvent(r,n))},b.Event=function(e,n){return this instanceof b.Event?(e&&e.type?(this.originalEvent=e,this.type=e.type,this.isDefaultPrevented=e.defaultPrevented||e.returnValue===!1||e.getPreventDefault&&e.getPreventDefault()?it:ot):this.type=e,n&&b.extend(this,n),this.timeStamp=e&&e.timeStamp||b.now(),this[b.expando]=!0,t):new b.Event(e,n)},b.Event.prototype={isDefaultPrevented:ot,isPropagationStopped:ot,isImmediatePropagationStopped:ot,preventDefault:function(){var e=this.originalEvent;this.isDefaultPrevented=it,e&&(e.preventDefault?e.preventDefault():e.returnValue=!1)},stopPropagation:function(){var e=this.originalEvent;this.isPropagationStopped=it,e&&(e.stopPropagation&&e.stopPropagation(),e.cancelBubble=!0)},stopImmediatePropagation:function(){this.isImmediatePropagationStopped=it,this.stopPropagati
 on()}},b.each({mouseenter:"mouseover",mouseleave:"mouseout"},function(e,t){b.event.special[e]={delegateType:t,bindType:t,handle:function(e){var n,r=this,i=e.relatedTarget,o=e.handleObj;
-return(!i||i!==r&&!b.contains(r,i))&&(e.type=o.origType,n=o.handler.apply(this,arguments),e.type=t),n}}}),b.support.submitBubbles||(b.event.special.submit={setup:function(){return b.nodeName(this,"form")?!1:(b.event.add(this,"click._submit keypress._submit",function(e){var n=e.target,r=b.nodeName(n,"input")||b.nodeName(n,"button")?n.form:t;r&&!b._data(r,"submitBubbles")&&(b.event.add(r,"submit._submit",function(e){e._submit_bubble=!0}),b._data(r,"submitBubbles",!0))}),t)},postDispatch:function(e){e._submit_bubble&&(delete e._submit_bubble,this.parentNode&&!e.isTrigger&&b.event.simulate("submit",this.parentNode,e,!0))},teardown:function(){return b.nodeName(this,"form")?!1:(b.event.remove(this,"._submit"),t)}}),b.support.changeBubbles||(b.event.special.change={setup:function(){return Z.test(this.nodeName)?(("checkbox"===this.type||"radio"===this.type)&&(b.event.add(this,"propertychange._change",function(e){"checked"===e.originalEvent.propertyName&&(this._just_changed=!0)}),b.event.add
 (this,"click._change",function(e){this._just_changed&&!e.isTrigger&&(this._just_changed=!1),b.event.simulate("change",this,e,!0)})),!1):(b.event.add(this,"beforeactivate._change",function(e){var t=e.target;Z.test(t.nodeName)&&!b._data(t,"changeBubbles")&&(b.event.add(t,"change._change",function(e){!this.parentNode||e.isSimulated||e.isTrigger||b.event.simulate("change",this.parentNode,e,!0)}),b._data(t,"changeBubbles",!0))}),t)},handle:function(e){var n=e.target;return this!==n||e.isSimulated||e.isTrigger||"radio"!==n.type&&"checkbox"!==n.type?e.handleObj.handler.apply(this,arguments):t},teardown:function(){return b.event.remove(this,"._change"),!Z.test(this.nodeName)}}),b.support.focusinBubbles||b.each({focus:"focusin",blur:"focusout"},function(e,t){var n=0,r=function(e){b.event.simulate(t,e.target,b.event.fix(e),!0)};b.event.special[t]={setup:function(){0===n++&&o.addEventListener(e,r,!0)},teardown:function(){0===--n&&o.removeEventListener(e,r,!0)}}}),b.fn.extend({on:function(e,n,r
 ,i,o){var a,s;if("object"==typeof e){"string"!=typeof n&&(r=r||n,n=t);for(a in e)this.on(a,n,r,e[a],o);return this}if(null==r&&null==i?(i=n,r=n=t):null==i&&("string"==typeof n?(i=r,r=t):(i=r,r=n,n=t)),i===!1)i=ot;else if(!i)return this;return 1===o&&(s=i,i=function(e){return b().off(e),s.apply(this,arguments)},i.guid=s.guid||(s.guid=b.guid++)),this.each(function(){b.event.add(this,e,i,r,n)})},one:function(e,t,n,r){return this.on(e,t,n,r,1)},off:function(e,n,r){var i,o;if(e&&e.preventDefault&&e.handleObj)return i=e.handleObj,b(e.delegateTarget).off(i.namespace?i.origType+"."+i.namespace:i.origType,i.selector,i.handler),this;if("object"==typeof e){for(o in e)this.off(o,n,e[o]);return this}return(n===!1||"function"==typeof n)&&(r=n,n=t),r===!1&&(r=ot),this.each(function(){b.event.remove(this,e,r,n)})},bind:function(e,t,n){return this.on(e,null,t,n)},unbind:function(e,t){return this.off(e,null,t)},delegate:function(e,t,n,r){return this.on(t,e,n,r)},undelegate:function(e,t,n){return 1===
 arguments.length?this.off(e,"**"):this.off(t,e||"**",n)},trigger:function(e,t){return this.each(function(){b.event.trigger(e,t,this)})},triggerHandler:function(e,n){var r=this[0];return r?b.event.trigger(e,n,r,!0):t}}),function(e,t){var n,r,i,o,a,s,u,l,c,p,f,d,h,g,m,y,v,x="sizzle"+-new Date,w=e.document,T={},N=0,C=0,k=it(),E=it(),S=it(),A=typeof t,j=1<<31,D=[],L=D.pop,H=D.push,q=D.slice,M=D.indexOf||function(e){var t=0,n=this.length;for(;n>t;t++)if(this[t]===e)return t;return-1},_="[\\x20\\t\\r\\n\\f]",F="(?:\\\\.|[\\w-]|[^\\x00-\\xa0])+",O=F.replace("w","w#"),B="([*^$|!~]?=)",P="\\["+_+"*("+F+")"+_+"*(?:"+B+_+"*(?:(['\"])((?:\\\\.|[^\\\\])*?)\\3|("+O+")|)|)"+_+"*\\]",R=":("+F+")(?:\\(((['\"])((?:\\\\.|[^\\\\])*?)\\3|((?:\\\\.|[^\\\\()[\\]]|"+P.replace(3,8)+")*)|.*)\\)|)",W=RegExp("^"+_+"+|((?:^|[^\\\\])(?:\\\\.)*)"+_+"+$","g"),$=RegExp("^"+_+"*,"+_+"*"),I=RegExp("^"+_+"*([\\x20\\t\\r\\n\\f>+~])"+_+"*"),z=RegExp(R),X=RegExp("^"+O+"$"),U={ID:RegExp("^#("+F+")"),CLASS:RegExp("^\\.("+F
 +")"),NAME:RegExp("^\\[name=['\"]?("+F+")['\"]?\\]"),TAG:RegExp("^("+F.replace("w","w*")+")"),ATTR:RegExp("^"+P),PSEUDO:RegExp("^"+R),CHILD:RegExp("^:(only|first|last|nth|nth-last)-(child|of-type)(?:\\("+_+"*(even|odd|(([+-]|)(\\d*)n|)"+_+"*(?:([+-]|)"+_+"*(\\d+)|))"+_+"*\\)|)","i"),needsContext:RegExp("^"+_+"*[>+~]|:(even|odd|eq|gt|lt|nth|first|last)(?:\\("+_+"*((?:-\\d)?\\d*)"+_+"*\\)|)(?=[^-]|$)","i")},V=/[\x20\t\r\n\f]*[+~]/,Y=/^[^{]+\{\s*\[native code/,J=/^(?:#([\w-]+)|(\w+)|\.([\w-]+))$/,G=/^(?:input|select|textarea|button)$/i,Q=/^h\d$/i,K=/'|\\/g,Z=/\=[\x20\t\r\n\f]*([^'"\]]*)[\x20\t\r\n\f]*\]/g,et=/\\([\da-fA-F]{1,6}[\x20\t\r\n\f]?|.)/g,tt=function(e,t){var n="0x"+t-65536;return n!==n?t:0>n?String.fromCharCode(n+65536):String.fromCharCode(55296|n>>10,56320|1023&n)};try{q.call(w.documentElement.childNodes,0)[0].nodeType}catch(nt){q=function(e){var t,n=[];while(t=this[e++])n.push(t);return n}}function rt(e){return Y.test(e+"")}function it(){var e,t=[];return e=function(n,r){re
 turn t.push(n+=" ")>i.cacheLength&&delete e[t.shift()],e[n]=r}}function ot(e){return e[x]=!0,e}function at(e){var t=p.createElement("div");try{return e(t)}catch(n){return!1}finally{t=null}}function st(e,t,n,r){var i,o,a,s,u,l,f,g,m,v;if((t?t.ownerDocument||t:w)!==p&&c(t),t=t||p,n=n||[],!e||"string"!=typeof e)return n;if(1!==(s=t.nodeType)&&9!==s)return[];if(!d&&!r){if(i=J.exec(e))if(a=i[1]){if(9===s){if(o=t.getElementById(a),!o||!o.parentNode)return n;if(o.id===a)return n.push(o),n}else if(t.ownerDocument&&(o=t.ownerDocument.getElementById(a))&&y(t,o)&&o.id===a)return n.push(o),n}else{if(i[2])return H.apply(n,q.call(t.getElementsByTagName(e),0)),n;if((a=i[3])&&T.getByClassName&&t.getElementsByClassName)return H.apply(n,q.call(t.getElementsByClassName(a),0)),n}if(T.qsa&&!h.test(e)){if(f=!0,g=x,m=t,v=9===s&&e,1===s&&"object"!==t.nodeName.toLowerCase()){l=ft(e),(f=t.getAttribute("id"))?g=f.replace(K,"\\$&"):t.setAttribute("id",g),g="[id='"+g+"'] ",u=l.length;while(u--)l[u]=g+dt(l[u]);m
 =V.test(e)&&t.parentNode||t,v=l.join(",")}if(v)try{return H.apply(n,q.call(m.querySelectorAll(v),0)),n}catch(b){}finally{f||t.removeAttribute("id")}}}return wt(e.replace(W,"$1"),t,n,r)}a=st.isXML=function(e){var t=e&&(e.ownerDocument||e).documentElement;return t?"HTML"!==t.nodeName:!1},c=st.setDocument=function(e){var n=e?e.ownerDocument||e:w;return n!==p&&9===n.nodeType&&n.documentElement?(p=n,f=n.documentElement,d=a(n),T.tagNameNoComments=at(function(e){return e.appendChild(n.createComment("")),!e.getElementsByTagName("*").length}),T.attributes=at(function(e){e.innerHTML="<select></select>";var t=typeof e.lastChild.getAttribute("multiple");return"boolean"!==t&&"string"!==t}),T.getByClassName=at(function(e){return e.innerHTML="<div class='hidden e'></div><div class='hidden'></div>",e.getElementsByClassName&&e.getElementsByClassName("e").length?(e.lastChild.className="e",2===e.getElementsByClassName("e").length):!1}),T.getByName=at(function(e){e.id=x+0,e.innerHTML="<a name='"+x+"'><
 /a><div name='"+x+"'></div>",f.insertBefore(e,f.firstChild);var t=n.getElementsByName&&n.getElementsByName(x).length===2+n.getElementsByName(x+0).length;return T.getIdNotName=!n.getElementById(x),f.removeChild(e),t}),i.attrHandle=at(function(e){return e.innerHTML="<a href='#'></a>",e.firstChild&&typeof e.firstChild.getAttribute!==A&&"#"===e.firstChild.getAttribute("href")})?{}:{href:function(e){return e.getAttribute("href",2)},type:function(e){return e.getAttribute("type")}},T.getIdNotName?(i.find.ID=function(e,t){if(typeof t.getElementById!==A&&!d){var n=t.getElementById(e);return n&&n.parentNode?[n]:[]}},i.filter.ID=function(e){var t=e.replace(et,tt);return function(e){return e.getAttribute("id")===t}}):(i.find.ID=function(e,n){if(typeof n.getElementById!==A&&!d){var r=n.getElementById(e);return r?r.id===e||typeof r.getAttributeNode!==A&&r.getAttributeNode("id").value===e?[r]:t:[]}},i.filter.ID=function(e){var t=e.replace(et,tt);return function(e){var n=typeof e.getAttributeNode!=
 =A&&e.getAttributeNode("id");return n&&n.value===t}}),i.find.TAG=T.tagNameNoComments?function(e,n){return typeof n.getElementsByTagName!==A?n.getElementsByTagName(e):t}:function(e,t){var n,r=[],i=0,o=t.getElementsByTagName(e);if("*"===e){while(n=o[i++])1===n.nodeType&&r.push(n);return r}return o},i.find.NAME=T.getByName&&function(e,n){return typeof n.getElementsByName!==A?n.getElementsByName(name):t},i.find.CLASS=T.getByClassName&&function(e,n){return typeof n.getElementsByClassName===A||d?t:n.getElementsByClassName(e)},g=[],h=[":focus"],(T.qsa=rt(n.querySelectorAll))&&(at(function(e){e.innerHTML="<select><option selected=''></option></select>",e.querySelectorAll("[selected]").length||h.push("\\["+_+"*(?:checked|disabled|ismap|multiple|readonly|selected|value)"),e.querySelectorAll(":checked").length||h.push(":checked")}),at(function(e){e.innerHTML="<input type='hidden' i=''/>",e.querySelectorAll("[i^='']").length&&h.push("[*^$]="+_+"*(?:\"\"|'')"),e.querySelectorAll(":enabled").leng
 th||h.push(":enabled",":disabled"),e.querySelectorAll("*,:x"),h.push(",.*:")})),(T.matchesSelector=rt(m=f.matchesSelector||f.mozMatchesSelector||f.webkitMatchesSelector||f.oMatchesSelector||f.msMatchesSelector))&&at(function(e){T.disconnectedMatch=m.call(e,"div"),m.call(e,"[s!='']:x"),g.push("!=",R)}),h=RegExp(h.join("|")),g=RegExp(g.join("|")),y=rt(f.contains)||f.compareDocumentPosition?function(e,t){var n=9===e.nodeType?e.documentElement:e,r=t&&t.parentNode;return e===r||!(!r||1!==r.nodeType||!(n.contains?n.contains(r):e.compareDocumentPosition&&16&e.compareDocumentPosition(r)))}:function(e,t){if(t)while(t=t.parentNode)if(t===e)return!0;return!1},v=f.compareDocumentPosition?function(e,t){var r;return e===t?(u=!0,0):(r=t.compareDocumentPosition&&e.compareDocumentPosition&&e.compareDocumentPosition(t))?1&r||e.parentNode&&11===e.parentNode.nodeType?e===n||y(w,e)?-1:t===n||y(w,t)?1:0:4&r?-1:1:e.compareDocumentPosition?-1:1}:function(e,t){var r,i=0,o=e.parentNode,a=t.parentNode,s=[e],l
 =[t];if(e===t)return u=!0,0;if(!o||!a)return e===n?-1:t===n?1:o?-1:a?1:0;if(o===a)return ut(e,t);r=e;while(r=r.parentNode)s.unshift(r);r=t;while(r=r.parentNode)l.unshift(r);while(s[i]===l[i])i++;return i?ut(s[i],l[i]):s[i]===w?-1:l[i]===w?1:0},u=!1,[0,0].sort(v),T.detectDuplicates=u,p):p},st.matches=function(e,t){return st(e,null,null,t)},st.matchesSelector=function(e,t){if((e.ownerDocument||e)!==p&&c(e),t=t.replace(Z,"='$1']"),!(!T.matchesSelector||d||g&&g.test(t)||h.test(t)))try{var n=m.call(e,t);if(n||T.disconnectedMatch||e.document&&11!==e.document.nodeType)return n}catch(r){}return st(t,p,null,[e]).length>0},st.contains=function(e,t){return(e.ownerDocument||e)!==p&&c(e),y(e,t)},st.attr=function(e,t){var n;return(e.ownerDocument||e)!==p&&c(e),d||(t=t.toLowerCase()),(n=i.attrHandle[t])?n(e):d||T.attributes?e.getAttribute(t):((n=e.getAttributeNode(t))||e.getAttribute(t))&&e[t]===!0?t:n&&n.specified?n.value:null},st.error=function(e){throw Error("Syntax error, unrecognized expressi
 on: "+e)},st.uniqueSort=function(e){var t,n=[],r=1,i=0;if(u=!T.detectDuplicates,e.sort(v),u){for(;t=e[r];r++)t===e[r-1]&&(i=n.push(r));while(i--)e.splice(n[i],1)}return e};function ut(e,t){var n=t&&e,r=n&&(~t.sourceIndex||j)-(~e.sourceIndex||j);if(r)return r;if(n)while(n=n.nextSibling)if(n===t)return-1;return e?1:-1}function lt(e){return function(t){var n=t.nodeName.toLowerCase();return"input"===n&&t.type===e}}function ct(e){return function(t){var n=t.nodeName.toLowerCase();return("input"===n||"button"===n)&&t.type===e}}function pt(e){return ot(function(t){return t=+t,ot(function(n,r){var i,o=e([],n.length,t),a=o.length;while(a--)n[i=o[a]]&&(n[i]=!(r[i]=n[i]))})})}o=st.getText=function(e){var t,n="",r=0,i=e.nodeType;if(i){if(1===i||9===i||11===i){if("string"==typeof e.textContent)return e.textContent;for(e=e.firstChild;e;e=e.nextSibling)n+=o(e)}else if(3===i||4===i)return e.nodeValue}else for(;t=e[r];r++)n+=o(t);return n},i=st.selectors={cacheLength:50,createPseudo:ot,match:U,find:{
 },relative:{">":{dir:"parentNode",first:!0}," ":{dir:"parentNode"},"+":{dir:"previousSibling",first:!0},"~":{dir:"previousSibling"}},preFilter:{ATTR:function(e){return e[1]=e[1].replace(et,tt),e[3]=(e[4]||e[5]||"").replace(et,tt),"~="===e[2]&&(e[3]=" "+e[3]+" "),e.slice(0,4)},CHILD:function(e){return e[1]=e[1].toLowerCase(),"nth"===e[1].slice(0,3)?(e[3]||st.error(e[0]),e[4]=+(e[4]?e[5]+(e[6]||1):2*("even"===e[3]||"odd"===e[3])),e[5]=+(e[7]+e[8]||"odd"===e[3])):e[3]&&st.error(e[0]),e},PSEUDO:function(e){var t,n=!e[5]&&e[2];return U.CHILD.test(e[0])?null:(e[4]?e[2]=e[4]:n&&z.test(n)&&(t=ft(n,!0))&&(t=n.indexOf(")",n.length-t)-n.length)&&(e[0]=e[0].slice(0,t),e[2]=n.slice(0,t)),e.slice(0,3))}},filter:{TAG:function(e){return"*"===e?function(){return!0}:(e=e.replace(et,tt).toLowerCase(),function(t){return t.nodeName&&t.nodeName.toLowerCase()===e})},CLASS:function(e){var t=k[e+" "];return t||(t=RegExp("(^|"+_+")"+e+"("+_+"|$)"))&&k(e,function(e){return t.test(e.className||typeof e.getAttr
 ibute!==A&&e.getAttribute("class")||"")})},ATTR:function(e,t,n){return function(r){var i=st.attr(r,e);return null==i?"!="===t:t?(i+="","="===t?i===n:"!="===t?i!==n:"^="===t?n&&0===i.indexOf(n):"*="===t?n&&i.indexOf(n)>-1:"$="===t?n&&i.slice(-n.length)===n:"~="===t?(" "+i+" ").indexOf(n)>-1:"|="===t?i===n||i.slice(0,n.length+1)===n+"-":!1):!0}},CHILD:function(e,t,n,r,i){var o="nth"!==e.slice(0,3),a="last"!==e.slice(-4),s="of-type"===t;return 1===r&&0===i?function(e){return!!e.parentNode}:function(t,n,u){var l,c,p,f,d,h,g=o!==a?"nextSibling":"previousSibling",m=t.parentNode,y=s&&t.nodeName.toLowerCase(),v=!u&&!s;if(m){if(o){while(g){p=t;while(p=p[g])if(s?p.nodeName.toLowerCase()===y:1===p.nodeType)return!1;h=g="only"===e&&!h&&"nextSibling"}return!0}if(h=[a?m.firstChild:m.lastChild],a&&v){c=m[x]||(m[x]={}),l=c[e]||[],d=l[0]===N&&l[1],f=l[0]===N&&l[2],p=d&&m.childNodes[d];while(p=++d&&p&&p[g]||(f=d=0)||h.pop())if(1===p.nodeType&&++f&&p===t){c[e]=[N,d,f];break}}else if(v&&(l=(t[x]||(t[x]
 ={}))[e])&&l[0]===N)f=l[1];else while(p=++d&&p&&p[g]||(f=d=0)||h.pop())if((s?p.nodeName.toLowerCase()===y:1===p.nodeType)&&++f&&(v&&((p[x]||(p[x]={}))[e]=[N,f]),p===t))break;return f-=i,f===r||0===f%r&&f/r>=0}}},PSEUDO:function(e,t){var n,r=i.pseudos[e]||i.setFilters[e.toLowerCase()]||st.error("unsupported pseudo: "+e);return r[x]?r(t):r.length>1?(n=[e,e,"",t],i.setFilters.hasOwnProperty(e.toLowerCase())?ot(function(e,n){var i,o=r(e,t),a=o.length;while(a--)i=M.call(e,o[a]),e[i]=!(n[i]=o[a])}):function(e){return r(e,0,n)}):r}},pseudos:{not:ot(function(e){var t=[],n=[],r=s(e.replace(W,"$1"));return r[x]?ot(function(e,t,n,i){var o,a=r(e,null,i,[]),s=e.length;while(s--)(o=a[s])&&(e[s]=!(t[s]=o))}):function(e,i,o){return t[0]=e,r(t,null,o,n),!n.pop()}}),has:ot(function(e){return function(t){return st(e,t).length>0}}),contains:ot(function(e){return function(t){return(t.textContent||t.innerText||o(t)).indexOf(e)>-1}}),lang:ot(function(e){return X.test(e||"")||st.error("unsupported lang: "+
 e),e=e.replace(et,tt).toLowerCase(),function(t){var n;do if(n=d?t.getAttribute("xml:lang")||t.getAttribute("lang"):t.lang)return n=n.toLowerCase(),n===e||0===n.indexOf(e+"-");while((t=t.parentNode)&&1===t.nodeType);return!1}}),target:function(t){var n=e.location&&e.location.hash;return n&&n.slice(1)===t.id},root:function(e){return e===f},focus:function(e){return e===p.activeElement&&(!p.hasFocus||p.hasFocus())&&!!(e.type||e.href||~e.tabIndex)},enabled:function(e){return e.disabled===!1},disabled:function(e){return e.disabled===!0},checked:function(e){var t=e.nodeName.toLowerCase();return"input"===t&&!!e.checked||"option"===t&&!!e.selected},selected:function(e){return e.parentNode&&e.parentNode.selectedIndex,e.selected===!0},empty:function(e){for(e=e.firstChild;e;e=e.nextSibling)if(e.nodeName>"@"||3===e.nodeType||4===e.nodeType)return!1;return!0},parent:function(e){return!i.pseudos.empty(e)},header:function(e){return Q.test(e.nodeName)},input:function(e){return G.test(e.nodeName)},bu
 tton:function(e){var t=e.nodeName.toLowerCase();return"input"===t&&"button"===e.type||"button"===t},text:function(e){var t;return"input"===e.nodeName.toLowerCase()&&"text"===e.type&&(null==(t=e.getAttribute("type"))||t.toLowerCase()===e.type)},first:pt(function(){return[0]}),last:pt(function(e,t){return[t-1]}),eq:pt(function(e,t,n){return[0>n?n+t:n]}),even:pt(function(e,t){var n=0;for(;t>n;n+=2)e.push(n);return e}),odd:pt(function(e,t){var n=1;for(;t>n;n+=2)e.push(n);return e}),lt:pt(function(e,t,n){var r=0>n?n+t:n;for(;--r>=0;)e.push(r);return e}),gt:pt(function(e,t,n){var r=0>n?n+t:n;for(;t>++r;)e.push(r);return e})}};for(n in{radio:!0,checkbox:!0,file:!0,password:!0,image:!0})i.pseudos[n]=lt(n);for(n in{submit:!0,reset:!0})i.pseudos[n]=ct(n);function ft(e,t){var n,r,o,a,s,u,l,c=E[e+" "];if(c)return t?0:c.slice(0);s=e,u=[],l=i.preFilter;while(s){(!n||(r=$.exec(s)))&&(r&&(s=s.slice(r[0].length)||s),u.push(o=[])),n=!1,(r=I.exec(s))&&(n=r.shift(),o.push({value:n,type:r[0].replace(W,"
  ")}),s=s.slice(n.length));for(a in i.filter)!(r=U[a].exec(s))||l[a]&&!(r=l[a](r))||(n=r.shift(),o.push({value:n,type:a,matches:r}),s=s.slice(n.length));if(!n)break}return t?s.length:s?st.error(e):E(e,u).slice(0)}function dt(e){var t=0,n=e.length,r="";for(;n>t;t++)r+=e[t].value;return r}function ht(e,t,n){var i=t.dir,o=n&&"parentNode"===i,a=C++;return t.first?function(t,n,r){while(t=t[i])if(1===t.nodeType||o)return e(t,n,r)}:function(t,n,s){var u,l,c,p=N+" "+a;if(s){while(t=t[i])if((1===t.nodeType||o)&&e(t,n,s))return!0}else while(t=t[i])if(1===t.nodeType||o)if(c=t[x]||(t[x]={}),(l=c[i])&&l[0]===p){if((u=l[1])===!0||u===r)return u===!0}else if(l=c[i]=[p],l[1]=e(t,n,s)||r,l[1]===!0)return!0}}function gt(e){return e.length>1?function(t,n,r){var i=e.length;while(i--)if(!e[i](t,n,r))return!1;return!0}:e[0]}function mt(e,t,n,r,i){var o,a=[],s=0,u=e.length,l=null!=t;for(;u>s;s++)(o=e[s])&&(!n||n(o,r,i))&&(a.push(o),l&&t.push(s));return a}function yt(e,t,n,r,i,o){return r&&!r[x]&&(r=yt(r))
 ,i&&!i[x]&&(i=yt(i,o)),ot(function(o,a,s,u){var l,c,p,f=[],d=[],h=a.length,g=o||xt(t||"*",s.nodeType?[s]:s,[]),m=!e||!o&&t?g:mt(g,f,e,s,u),y=n?i||(o?e:h||r)?[]:a:m;if(n&&n(m,y,s,u),r){l=mt(y,d),r(l,[],s,u),c=l.length;while(c--)(p=l[c])&&(y[d[c]]=!(m[d[c]]=p))}if(o){if(i||e){if(i){l=[],c=y.length;while(c--)(p=y[c])&&l.push(m[c]=p);i(null,y=[],l,u)}c=y.length;while(c--)(p=y[c])&&(l=i?M.call(o,p):f[c])>-1&&(o[l]=!(a[l]=p))}}else y=mt(y===a?y.splice(h,y.length):y),i?i(null,a,y,u):H.apply(a,y)})}function vt(e){var t,n,r,o=e.length,a=i.relative[e[0].type],s=a||i.relative[" "],u=a?1:0,c=ht(function(e){return e===t},s,!0),p=ht(function(e){return M.call(t,e)>-1},s,!0),f=[function(e,n,r){return!a&&(r||n!==l)||((t=n).nodeType?c(e,n,r):p(e,n,r))}];for(;o>u;u++)if(n=i.relative[e[u].type])f=[ht(gt(f),n)];else{if(n=i.filter[e[u].type].apply(null,e[u].matches),n[x]){for(r=++u;o>r;r++)if(i.relative[e[r].type])break;return yt(u>1&&gt(f),u>1&&dt(e.slice(0,u-1)).replace(W,"$1"),n,r>u&&vt(e.slice(u,r)),
 o>r&&vt(e=e.slice(r)),o>r&&dt(e))}f.push(n)}return gt(f)}function bt(e,t){var n=0,o=t.length>0,a=e.length>0,s=function(s,u,c,f,d){var h,g,m,y=[],v=0,b="0",x=s&&[],w=null!=d,T=l,C=s||a&&i.find.TAG("*",d&&u.parentNode||u),k=N+=null==T?1:Math.random()||.1;for(w&&(l=u!==p&&u,r=n);null!=(h=C[b]);b++){if(a&&h){g=0;while(m=e[g++])if(m(h,u,c)){f.push(h);break}w&&(N=k,r=++n)}o&&((h=!m&&h)&&v--,s&&x.push(h))}if(v+=b,o&&b!==v){g=0;while(m=t[g++])m(x,y,u,c);if(s){if(v>0)while(b--)x[b]||y[b]||(y[b]=L.call(f));y=mt(y)}H.apply(f,y),w&&!s&&y.length>0&&v+t.length>1&&st.uniqueSort(f)}return w&&(N=k,l=T),x};return o?ot(s):s}s=st.compile=function(e,t){var n,r=[],i=[],o=S[e+" "];if(!o){t||(t=ft(e)),n=t.length;while(n--)o=vt(t[n]),o[x]?r.push(o):i.push(o);o=S(e,bt(i,r))}return o};function xt(e,t,n){var r=0,i=t.length;for(;i>r;r++)st(e,t[r],n);return n}function wt(e,t,n,r){var o,a,u,l,c,p=ft(e);if(!r&&1===p.length){if(a=p[0]=p[0].slice(0),a.length>2&&"ID"===(u=a[0]).type&&9===t.nodeType&&!d&&i.relative[a[
 1].type]){if(t=i.find.ID(u.matches[0].replace(et,tt),t)[0],!t)return n;e=e.slice(a.shift().value.length)}o=U.needsContext.test(e)?0:a.length;while(o--){if(u=a[o],i.relative[l=u.type])break;if((c=i.find[l])&&(r=c(u.matches[0].replace(et,tt),V.test(a[0].type)&&t.parentNode||t))){if(a.splice(o,1),e=r.length&&dt(a),!e)return H.apply(n,q.call(r,0)),n;break}}}return s(e,p)(r,t,d,n,V.test(e)),n}i.pseudos.nth=i.pseudos.eq;function Tt(){}i.filters=Tt.prototype=i.pseudos,i.setFilters=new Tt,c(),st.attr=b.attr,b.find=st,b.expr=st.selectors,b.expr[":"]=b.expr.pseudos,b.unique=st.uniqueSort,b.text=st.getText,b.isXMLDoc=st.isXML,b.contains=st.contains}(e);var at=/Until$/,st=/^(?:parents|prev(?:Until|All))/,ut=/^.[^:#\[\.,]*$/,lt=b.expr.match.needsContext,ct={children:!0,contents:!0,next:!0,prev:!0};b.fn.extend({find:function(e){var t,n,r,i=this.length;if("string"!=typeof e)return r=this,this.pushStack(b(e).filter(function(){for(t=0;i>t;t++)if(b.contains(r[t],this))return!0}));for(n=[],t=0;i>t;t++
 )b.find(e,this[t],n);return n=this.pushStack(i>1?b.unique(n):n),n.selector=(this.selector?this.selector+" ":"")+e,n},has:function(e){var t,n=b(e,this),r=n.length;return this.filter(function(){for(t=0;r>t;t++)if(b.contains(this,n[t]))return!0})},not:function(e){return this.pushStack(ft(this,e,!1))},filter:function(e){return this.pushStack(ft(this,e,!0))},is:function(e){return!!e&&("string"==typeof e?lt.test(e)?b(e,this.context).index(this[0])>=0:b.filter(e,this).length>0:this.filter(e).length>0)},closest:function(e,t){var n,r=0,i=this.length,o=[],a=lt.test(e)||"string"!=typeof e?b(e,t||this.context):0;for(;i>r;r++){n=this[r];while(n&&n.ownerDocument&&n!==t&&11!==n.nodeType){if(a?a.index(n)>-1:b.find.matchesSelector(n,e)){o.push(n);break}n=n.parentNode}}return this.pushStack(o.length>1?b.unique(o):o)},index:function(e){return e?"string"==typeof e?b.inArray(this[0],b(e)):b.inArray(e.jquery?e[0]:e,this):this[0]&&this[0].parentNode?this.first().prevAll().length:-1},add:function(e,t){var 
 n="string"==typeof e?b(e,t):b.makeArray(e&&e.nodeType?[e]:e),r=b.merge(this.get(),n);return this.pushStack(b.unique(r))},addBack:function(e){return this.add(null==e?this.prevObject:this.prevObject.filter(e))}}),b.fn.andSelf=b.fn.addBack;function pt(e,t){do e=e[t];while(e&&1!==e.nodeType);return e}b.each({parent:function(e){var t=e.parentNode;return t&&11!==t.nodeType?t:null},parents:function(e){return b.dir(e,"parentNode")},parentsUntil:function(e,t,n){return b.dir(e,"parentNode",n)},next:function(e){return pt(e,"nextSibling")},prev:function(e){return pt(e,"previousSibling")},nextAll:function(e){return b.dir(e,"nextSibling")},prevAll:function(e){return b.dir(e,"previousSibling")},nextUntil:function(e,t,n){return b.dir(e,"nextSibling",n)},prevUntil:function(e,t,n){return b.dir(e,"previousSibling",n)},siblings:function(e){return b.sibling((e.parentNode||{}).firstChild,e)},children:function(e){return b.sibling(e.firstChild)},contents:function(e){return b.nodeName(e,"iframe")?e.contentD
 ocument||e.contentWindow.document:b.merge([],e.childNodes)}},function(e,t){b.fn[e]=function(n,r){var i=b.map(this,t,n);return at.test(e)||(r=n),r&&"string"==typeof r&&(i=b.filter(r,i)),i=this.length>1&&!ct[e]?b.unique(i):i,this.length>1&&st.test(e)&&(i=i.reverse()),this.pushStack(i)}}),b.extend({filter:function(e,t,n){return n&&(e=":not("+e+")"),1===t.length?b.find.matchesSelector(t[0],e)?[t[0]]:[]:b.find.matches(e,t)},dir:function(e,n,r){var i=[],o=e[n];while(o&&9!==o.nodeType&&(r===t||1!==o.nodeType||!b(o).is(r)))1===o.nodeType&&i.push(o),o=o[n];return i},sibling:function(e,t){var n=[];for(;e;e=e.nextSibling)1===e.nodeType&&e!==t&&n.push(e);return n}});function ft(e,t,n){if(t=t||0,b.isFunction(t))return b.grep(e,function(e,r){var i=!!t.call(e,r,e);return i===n});if(t.nodeType)return b.grep(e,function(e){return e===t===n});if("string"==typeof t){var r=b.grep(e,function(e){return 1===e.nodeType});if(ut.test(t))return b.filter(t,r,!n);t=b.filter(t,r)}return b.grep(e,function(e){retur
 n b.inArray(e,t)>=0===n})}function dt(e){var t=ht.split("|"),n=e.createDocumentFragment();if(n.createElement)while(t.length)n.createElement(t.pop());return n}var ht="abbr|article|aside|audio|bdi|canvas|data|datalist|details|figcaption|figure|footer|header|hgroup|mark|meter|nav|output|progress|section|summary|time|video",gt=/ jQuery\d+="(?:null|\d+)"/g,mt=RegExp("<(?:"+ht+")[\\s/>]","i"),yt=/^\s+/,vt=/<(?!area|br|col|embed|hr|img|input|link|meta|param)(([\w:]+)[^>]*)\/>/gi,bt=/<([\w:]+)/,xt=/<tbody/i,wt=/<|&#?\w+;/,Tt=/<(?:script|style|link)/i,Nt=/^(?:checkbox|radio)$/i,Ct=/checked\s*(?:[^=]|=\s*.checked.)/i,kt=/^$|\/(?:java|ecma)script/i,Et=/^true\/(.*)/,St=/^\s*<!(?:\[CDATA\[|--)|(?:\]\]|--)>\s*$/g,At={option:[1,"<select multiple='multiple'>","</select>"],legend:[1,"<fieldset>","</fieldset>"],area:[1,"<map>","</map>"],param:[1,"<object>","</object>"],thead:[1,"<table>","</table>"],tr:[2,"<table><tbody>","</tbody></table>"],col:[2,"<table><tbody></tbody><colgroup>","</colgroup></tab
 le>"],td:[3,"<table><tbody><tr>","</tr></tbody></table>"],_default:b.support.htmlSerialize?[0,"",""]:[1,"X<div>","</div>"]},jt=dt(o),Dt=jt.appendChild(o.createElement("div"));At.optgroup=At.option,At.tbody=At.tfoot=At.colgroup=At.caption=At.thead,At.th=At.td,b.fn.extend({text:function(e){return b.access(this,function(e){return e===t?b.text(this):this.empty().append((this[0]&&this[0].ownerDocument||o).createTextNode(e))},null,e,arguments.length)},wrapAll:function(e){if(b.isFunction(e))return this.each(function(t){b(this).wrapAll(e.call(this,t))});if(this[0]){var t=b(e,this[0].ownerDocument).eq(0).clone(!0);this[0].parentNode&&t.insertBefore(this[0]),t.map(function(){var e=this;while(e.firstChild&&1===e.firstChild.nodeType)e=e.firstChild;return e}).append(this)}return this},wrapInner:function(e){return b.isFunction(e)?this.each(function(t){b(this).wrapInner(e.call(this,t))}):this.each(function(){var t=b(this),n=t.contents();n.length?n.wrapAll(e):t.append(e)})},wrap:function(e){var t=b
 .isFunction(e);return this.each(function(n){b(this).wrapAll(t?e.call(this,n):e)})},unwrap:function(){return this.parent().each(function(){b.nodeName(this,"body")||b(this).replaceWith(this.childNodes)}).end()},append:function(){return this.domManip(arguments,!0,function(e){(1===this.nodeType||11===this.nodeType||9===this.nodeType)&&this.appendChild(e)})},prepend:function(){return this.domManip(arguments,!0,function(e){(1===this.nodeType||11===this.nodeType||9===this.nodeType)&&this.insertBefore(e,this.firstChild)})},before:function(){return this.domManip(arguments,!1,function(e){this.parentNode&&this.parentNode.insertBefore(e,this)})},after:function(){return this.domManip(arguments,!1,function(e){this.parentNode&&this.parentNode.insertBefore(e,this.nextSibling)})},remove:function(e,t){var n,r=0;for(;null!=(n=this[r]);r++)(!e||b.filter(e,[n]).length>0)&&(t||1!==n.nodeType||b.cleanData(Ot(n)),n.parentNode&&(t&&b.contains(n.ownerDocument,n)&&Mt(Ot(n,"script")),n.parentNode.removeChild(n
 )));return this},empty:function(){var e,t=0;for(;null!=(e=this[t]);t++){1===e.nodeType&&b.cleanData(Ot(e,!1));while(e.firstChild)e.removeChild(e.firstChild);e.options&&b.nodeName(e,"select")&&(e.options.length=0)}return this},clone:function(e,t){return e=null==e?!1:e,t=null==t?e:t,this.map(function(){return b.clone(this,e,t)})},html:function(e){return b.access(this,function(e){var n=this[0]||{},r=0,i=this.length;if(e===t)return 1===n.nodeType?n.innerHTML.replace(gt,""):t;if(!("string"!=typeof e||Tt.test(e)||!b.support.htmlSerialize&&mt.test(e)||!b.support.leadingWhitespace&&yt.test(e)||At[(bt.exec(e)||["",""])[1].toLowerCase()])){e=e.replace(vt,"<$1></$2>");try{for(;i>r;r++)n=this[r]||{},1===n.nodeType&&(b.cleanData(Ot(n,!1)),n.innerHTML=e);n=0}catch(o){}}n&&this.empty().append(e)},null,e,arguments.length)},replaceWith:function(e){var t=b.isFunction(e);return t||"string"==typeof e||(e=b(e).not(this).detach()),this.domManip([e],!0,function(e){var t=this.nextSibling,n=this.parentNode;
 n&&(b(this).remove(),n.insertBefore(e,t))})},detach:function(e){return this.remove(e,!0)},domManip:function(e,n,r){e=f.apply([],e);var i,o,a,s,u,l,c=0,p=this.length,d=this,h=p-1,g=e[0],m=b.isFunction(g);if(m||!(1>=p||"string"!=typeof g||b.support.checkClone)&&Ct.test(g))return this.each(function(i){var o=d.eq(i);m&&(e[0]=g.call(this,i,n?o.html():t)),o.domManip(e,n,r)});if(p&&(l=b.buildFragment(e,this[0].ownerDocument,!1,this),i=l.firstChild,1===l.childNodes.length&&(l=i),i)){for(n=n&&b.nodeName(i,"tr"),s=b.map(Ot(l,"script"),Ht),a=s.length;p>c;c++)o=l,c!==h&&(o=b.clone(o,!0,!0),a&&b.merge(s,Ot(o,"script"))),r.call(n&&b.nodeName(this[c],"table")?Lt(this[c],"tbody"):this[c],o,c);if(a)for(u=s[s.length-1].ownerDocument,b.map(s,qt),c=0;a>c;c++)o=s[c],kt.test(o.type||"")&&!b._data(o,"globalEval")&&b.contains(u,o)&&(o.src?b.ajax({url:o.src,type:"GET",dataType:"script",async:!1,global:!1,"throws":!0}):b.globalEval((o.text||o.textContent||o.innerHTML||"").replace(St,"")));l=i=null}return thi
 s}});function Lt(e,t){return e.getElementsByTagName(t)[0]||e.appendChild(e.ownerDocument.createElement(t))}function Ht(e){var t=e.getAttributeNode("type");return e.type=(t&&t.specified)+"/"+e.type,e}function qt(e){var t=Et.exec(e.type);return t?e.type=t[1]:e.removeAttribute("type"),e}function Mt(e,t){var n,r=0;for(;null!=(n=e[r]);r++)b._data(n,"globalEval",!t||b._data(t[r],"globalEval"))}function _t(e,t){if(1===t.nodeType&&b.hasData(e)){var n,r,i,o=b._data(e),a=b._data(t,o),s=o.events;if(s){delete a.handle,a.events={};for(n in s)for(r=0,i=s[n].length;i>r;r++)b.event.add(t,n,s[n][r])}a.data&&(a.data=b.extend({},a.data))}}function Ft(e,t){var n,r,i;if(1===t.nodeType){if(n=t.nodeName.toLowerCase(),!b.support.noCloneEvent&&t[b.expando]){i=b._data(t);for(r in i.events)b.removeEvent(t,r,i.handle);t.removeAttribute(b.expando)}"script"===n&&t.text!==e.text?(Ht(t).text=e.text,qt(t)):"object"===n?(t.parentNode&&(t.outerHTML=e.outerHTML),b.support.html5Clone&&e.innerHTML&&!b.trim(t.innerHTML)&
 &(t.innerHTML=e.innerHTML)):"input"===n&&Nt.test(e.type)?(t.defaultChecked=t.checked=e.checked,t.value!==e.value&&(t.value=e.value)):"option"===n?t.defaultSelected=t.selected=e.defaultSelected:("input"===n||"textarea"===n)&&(t.defaultValue=e.defaultValue)}}b.each({appendTo:"append",prependTo:"prepend",insertBefore:"before",insertAfter:"after",replaceAll:"replaceWith"},function(e,t){b.fn[e]=function(e){var n,r=0,i=[],o=b(e),a=o.length-1;for(;a>=r;r++)n=r===a?this:this.clone(!0),b(o[r])[t](n),d.apply(i,n.get());return this.pushStack(i)}});function Ot(e,n){var r,o,a=0,s=typeof e.getElementsByTagName!==i?e.getElementsByTagName(n||"*"):typeof e.querySelectorAll!==i?e.querySelectorAll(n||"*"):t;if(!s)for(s=[],r=e.childNodes||e;null!=(o=r[a]);a++)!n||b.nodeName(o,n)?s.push(o):b.merge(s,Ot(o,n));return n===t||n&&b.nodeName(e,n)?b.merge([e],s):s}function Bt(e){Nt.test(e.type)&&(e.defaultChecked=e.checked)}b.extend({clone:function(e,t,n){var r,i,o,a,s,u=b.contains(e.ownerDocument,e);if(b.supp
 ort.html5Clone||b.isXMLDoc(e)||!mt.test("<"+e.nodeName+">")?o=e.cloneNode(!0):(Dt.innerHTML=e.outerHTML,Dt.removeChild(o=Dt.firstChild)),!(b.support.noCloneEvent&&b.support.noCloneChecked||1!==e.nodeType&&11!==e.nodeType||b.isXMLDoc(e)))for(r=Ot(o),s=Ot(e),a=0;null!=(i=s[a]);++a)r[a]&&Ft(i,r[a]);if(t)if(n)for(s=s||Ot(e),r=r||Ot(o),a=0;null!=(i=s[a]);a++)_t(i,r[a]);else _t(e,o);return r=Ot(o,"script"),r.length>0&&Mt(r,!u&&Ot(e,"script")),r=s=i=null,o},buildFragment:function(e,t,n,r){var i,o,a,s,u,l,c,p=e.length,f=dt(t),d=[],h=0;for(;p>h;h++)if(o=e[h],o||0===o)if("object"===b.type(o))b.merge(d,o.nodeType?[o]:o);else if(wt.test(o)){s=s||f.appendChild(t.createElement("div")),u=(bt.exec(o)||["",""])[1].toLowerCase(),c=At[u]||At._default,s.innerHTML=c[1]+o.replace(vt,"<$1></$2>")+c[2],i=c[0];while(i--)s=s.lastChild;if(!b.support.leadingWhitespace&&yt.test(o)&&d.push(t.createTextNode(yt.exec(o)[0])),!b.support.tbody){o="table"!==u||xt.test(o)?"<table>"!==c[1]||xt.test(o)?0:s:s.firstChild,i
 =o&&o.childNodes.length;while(i--)b.nodeName(l=o.childNodes[i],"tbody")&&!l.childNodes.length&&o.removeChild(l)
-}b.merge(d,s.childNodes),s.textContent="";while(s.firstChild)s.removeChild(s.firstChild);s=f.lastChild}else d.push(t.createTextNode(o));s&&f.removeChild(s),b.support.appendChecked||b.grep(Ot(d,"input"),Bt),h=0;while(o=d[h++])if((!r||-1===b.inArray(o,r))&&(a=b.contains(o.ownerDocument,o),s=Ot(f.appendChild(o),"script"),a&&Mt(s),n)){i=0;while(o=s[i++])kt.test(o.type||"")&&n.push(o)}return s=null,f},cleanData:function(e,t){var n,r,o,a,s=0,u=b.expando,l=b.cache,p=b.support.deleteExpando,f=b.event.special;for(;null!=(n=e[s]);s++)if((t||b.acceptData(n))&&(o=n[u],a=o&&l[o])){if(a.events)for(r in a.events)f[r]?b.event.remove(n,r):b.removeEvent(n,r,a.handle);l[o]&&(delete l[o],p?delete n[u]:typeof n.removeAttribute!==i?n.removeAttribute(u):n[u]=null,c.push(o))}}});var Pt,Rt,Wt,$t=/alpha\([^)]*\)/i,It=/opacity\s*=\s*([^)]*)/,zt=/^(top|right|bottom|left)$/,Xt=/^(none|table(?!-c[ea]).+)/,Ut=/^margin/,Vt=RegExp("^("+x+")(.*)$","i"),Yt=RegExp("^("+x+")(?!px)[a-z%]+$","i"),Jt=RegExp("^([+-])=("+x+
 ")","i"),Gt={BODY:"block"},Qt={position:"absolute",visibility:"hidden",display:"block"},Kt={letterSpacing:0,fontWeight:400},Zt=["Top","Right","Bottom","Left"],en=["Webkit","O","Moz","ms"];function tn(e,t){if(t in e)return t;var n=t.charAt(0).toUpperCase()+t.slice(1),r=t,i=en.length;while(i--)if(t=en[i]+n,t in e)return t;return r}function nn(e,t){return e=t||e,"none"===b.css(e,"display")||!b.contains(e.ownerDocument,e)}function rn(e,t){var n,r,i,o=[],a=0,s=e.length;for(;s>a;a++)r=e[a],r.style&&(o[a]=b._data(r,"olddisplay"),n=r.style.display,t?(o[a]||"none"!==n||(r.style.display=""),""===r.style.display&&nn(r)&&(o[a]=b._data(r,"olddisplay",un(r.nodeName)))):o[a]||(i=nn(r),(n&&"none"!==n||!i)&&b._data(r,"olddisplay",i?n:b.css(r,"display"))));for(a=0;s>a;a++)r=e[a],r.style&&(t&&"none"!==r.style.display&&""!==r.style.display||(r.style.display=t?o[a]||"":"none"));return e}b.fn.extend({css:function(e,n){return b.access(this,function(e,n,r){var i,o,a={},s=0;if(b.isArray(n)){for(o=Rt(e),i=n.
 length;i>s;s++)a[n[s]]=b.css(e,n[s],!1,o);return a}return r!==t?b.style(e,n,r):b.css(e,n)},e,n,arguments.length>1)},show:function(){return rn(this,!0)},hide:function(){return rn(this)},toggle:function(e){var t="boolean"==typeof e;return this.each(function(){(t?e:nn(this))?b(this).show():b(this).hide()})}}),b.extend({cssHooks:{opacity:{get:function(e,t){if(t){var n=Wt(e,"opacity");return""===n?"1":n}}}},cssNumber:{columnCount:!0,fillOpacity:!0,fontWeight:!0,lineHeight:!0,opacity:!0,orphans:!0,widows:!0,zIndex:!0,zoom:!0},cssProps:{"float":b.support.cssFloat?"cssFloat":"styleFloat"},style:function(e,n,r,i){if(e&&3!==e.nodeType&&8!==e.nodeType&&e.style){var o,a,s,u=b.camelCase(n),l=e.style;if(n=b.cssProps[u]||(b.cssProps[u]=tn(l,u)),s=b.cssHooks[n]||b.cssHooks[u],r===t)return s&&"get"in s&&(o=s.get(e,!1,i))!==t?o:l[n];if(a=typeof r,"string"===a&&(o=Jt.exec(r))&&(r=(o[1]+1)*o[2]+parseFloat(b.css(e,n)),a="number"),!(null==r||"number"===a&&isNaN(r)||("number"!==a||b.cssNumber[u]||(r+="px"
 ),b.support.clearCloneStyle||""!==r||0!==n.indexOf("background")||(l[n]="inherit"),s&&"set"in s&&(r=s.set(e,r,i))===t)))try{l[n]=r}catch(c){}}},css:function(e,n,r,i){var o,a,s,u=b.camelCase(n);return n=b.cssProps[u]||(b.cssProps[u]=tn(e.style,u)),s=b.cssHooks[n]||b.cssHooks[u],s&&"get"in s&&(a=s.get(e,!0,r)),a===t&&(a=Wt(e,n,i)),"normal"===a&&n in Kt&&(a=Kt[n]),""===r||r?(o=parseFloat(a),r===!0||b.isNumeric(o)?o||0:a):a},swap:function(e,t,n,r){var i,o,a={};for(o in t)a[o]=e.style[o],e.style[o]=t[o];i=n.apply(e,r||[]);for(o in t)e.style[o]=a[o];return i}}),e.getComputedStyle?(Rt=function(t){return e.getComputedStyle(t,null)},Wt=function(e,n,r){var i,o,a,s=r||Rt(e),u=s?s.getPropertyValue(n)||s[n]:t,l=e.style;return s&&(""!==u||b.contains(e.ownerDocument,e)||(u=b.style(e,n)),Yt.test(u)&&Ut.test(n)&&(i=l.width,o=l.minWidth,a=l.maxWidth,l.minWidth=l.maxWidth=l.width=u,u=s.width,l.width=i,l.minWidth=o,l.maxWidth=a)),u}):o.documentElement.currentStyle&&(Rt=function(e){return e.currentStyle
 },Wt=function(e,n,r){var i,o,a,s=r||Rt(e),u=s?s[n]:t,l=e.style;return null==u&&l&&l[n]&&(u=l[n]),Yt.test(u)&&!zt.test(n)&&(i=l.left,o=e.runtimeStyle,a=o&&o.left,a&&(o.left=e.currentStyle.left),l.left="fontSize"===n?"1em":u,u=l.pixelLeft+"px",l.left=i,a&&(o.left=a)),""===u?"auto":u});function on(e,t,n){var r=Vt.exec(t);return r?Math.max(0,r[1]-(n||0))+(r[2]||"px"):t}function an(e,t,n,r,i){var o=n===(r?"border":"content")?4:"width"===t?1:0,a=0;for(;4>o;o+=2)"margin"===n&&(a+=b.css(e,n+Zt[o],!0,i)),r?("content"===n&&(a-=b.css(e,"padding"+Zt[o],!0,i)),"margin"!==n&&(a-=b.css(e,"border"+Zt[o]+"Width",!0,i))):(a+=b.css(e,"padding"+Zt[o],!0,i),"padding"!==n&&(a+=b.css(e,"border"+Zt[o]+"Width",!0,i)));return a}function sn(e,t,n){var r=!0,i="width"===t?e.offsetWidth:e.offsetHeight,o=Rt(e),a=b.support.boxSizing&&"border-box"===b.css(e,"boxSizing",!1,o);if(0>=i||null==i){if(i=Wt(e,t,o),(0>i||null==i)&&(i=e.style[t]),Yt.test(i))return i;r=a&&(b.support.boxSizingReliable||i===e.style[t]),i=parse
 Float(i)||0}return i+an(e,t,n||(a?"border":"content"),r,o)+"px"}function un(e){var t=o,n=Gt[e];return n||(n=ln(e,t),"none"!==n&&n||(Pt=(Pt||b("<iframe frameborder='0' width='0' height='0'/>").css("cssText","display:block !important")).appendTo(t.documentElement),t=(Pt[0].contentWindow||Pt[0].contentDocument).document,t.write("<!doctype html><html><body>"),t.close(),n=ln(e,t),Pt.detach()),Gt[e]=n),n}function ln(e,t){var n=b(t.createElement(e)).appendTo(t.body),r=b.css(n[0],"display");return n.remove(),r}b.each(["height","width"],function(e,n){b.cssHooks[n]={get:function(e,r,i){return r?0===e.offsetWidth&&Xt.test(b.css(e,"display"))?b.swap(e,Qt,function(){return sn(e,n,i)}):sn(e,n,i):t},set:function(e,t,r){var i=r&&Rt(e);return on(e,t,r?an(e,n,r,b.support.boxSizing&&"border-box"===b.css(e,"boxSizing",!1,i),i):0)}}}),b.support.opacity||(b.cssHooks.opacity={get:function(e,t){return It.test((t&&e.currentStyle?e.currentStyle.filter:e.style.filter)||"")?.01*parseFloat(RegExp.$1)+"":t?"1":"
 "},set:function(e,t){var n=e.style,r=e.currentStyle,i=b.isNumeric(t)?"alpha(opacity="+100*t+")":"",o=r&&r.filter||n.filter||"";n.zoom=1,(t>=1||""===t)&&""===b.trim(o.replace($t,""))&&n.removeAttribute&&(n.removeAttribute("filter"),""===t||r&&!r.filter)||(n.filter=$t.test(o)?o.replace($t,i):o+" "+i)}}),b(function(){b.support.reliableMarginRight||(b.cssHooks.marginRight={get:function(e,n){return n?b.swap(e,{display:"inline-block"},Wt,[e,"marginRight"]):t}}),!b.support.pixelPosition&&b.fn.position&&b.each(["top","left"],function(e,n){b.cssHooks[n]={get:function(e,r){return r?(r=Wt(e,n),Yt.test(r)?b(e).position()[n]+"px":r):t}}})}),b.expr&&b.expr.filters&&(b.expr.filters.hidden=function(e){return 0>=e.offsetWidth&&0>=e.offsetHeight||!b.support.reliableHiddenOffsets&&"none"===(e.style&&e.style.display||b.css(e,"display"))},b.expr.filters.visible=function(e){return!b.expr.filters.hidden(e)}),b.each({margin:"",padding:"",border:"Width"},function(e,t){b.cssHooks[e+t]={expand:function(n){var
  r=0,i={},o="string"==typeof n?n.split(" "):[n];for(;4>r;r++)i[e+Zt[r]+t]=o[r]||o[r-2]||o[0];return i}},Ut.test(e)||(b.cssHooks[e+t].set=on)});var cn=/%20/g,pn=/\[\]$/,fn=/\r?\n/g,dn=/^(?:submit|button|image|reset|file)$/i,hn=/^(?:input|select|textarea|keygen)/i;b.fn.extend({serialize:function(){return b.param(this.serializeArray())},serializeArray:function(){return this.map(function(){var e=b.prop(this,"elements");return e?b.makeArray(e):this}).filter(function(){var e=this.type;return this.name&&!b(this).is(":disabled")&&hn.test(this.nodeName)&&!dn.test(e)&&(this.checked||!Nt.test(e))}).map(function(e,t){var n=b(this).val();return null==n?null:b.isArray(n)?b.map(n,function(e){return{name:t.name,value:e.replace(fn,"\r\n")}}):{name:t.name,value:n.replace(fn,"\r\n")}}).get()}}),b.param=function(e,n){var r,i=[],o=function(e,t){t=b.isFunction(t)?t():null==t?"":t,i[i.length]=encodeURIComponent(e)+"="+encodeURIComponent(t)};if(n===t&&(n=b.ajaxSettings&&b.ajaxSettings.traditional),b.isArra
 y(e)||e.jquery&&!b.isPlainObject(e))b.each(e,function(){o(this.name,this.value)});else for(r in e)gn(r,e[r],n,o);return i.join("&").replace(cn,"+")};function gn(e,t,n,r){var i;if(b.isArray(t))b.each(t,function(t,i){n||pn.test(e)?r(e,i):gn(e+"["+("object"==typeof i?t:"")+"]",i,n,r)});else if(n||"object"!==b.type(t))r(e,t);else for(i in t)gn(e+"["+i+"]",t[i],n,r)}b.each("blur focus focusin focusout load resize scroll unload click dblclick mousedown mouseup mousemove mouseover mouseout mouseenter mouseleave change select submit keydown keypress keyup error contextmenu".split(" "),function(e,t){b.fn[t]=function(e,n){return arguments.length>0?this.on(t,null,e,n):this.trigger(t)}}),b.fn.hover=function(e,t){return this.mouseenter(e).mouseleave(t||e)};var mn,yn,vn=b.now(),bn=/\?/,xn=/#.*$/,wn=/([?&])_=[^&]*/,Tn=/^(.*?):[ \t]*([^\r\n]*)\r?$/gm,Nn=/^(?:about|app|app-storage|.+-extension|file|res|widget):$/,Cn=/^(?:GET|HEAD)$/,kn=/^\/\//,En=/^([\w.+-]+:)(?:\/\/([^\/?#:]*)(?::(\d+)|)|)/,Sn=b.fn
 .load,An={},jn={},Dn="*/".concat("*");try{yn=a.href}catch(Ln){yn=o.createElement("a"),yn.href="",yn=yn.href}mn=En.exec(yn.toLowerCase())||[];function Hn(e){return function(t,n){"string"!=typeof t&&(n=t,t="*");var r,i=0,o=t.toLowerCase().match(w)||[];if(b.isFunction(n))while(r=o[i++])"+"===r[0]?(r=r.slice(1)||"*",(e[r]=e[r]||[]).unshift(n)):(e[r]=e[r]||[]).push(n)}}function qn(e,n,r,i){var o={},a=e===jn;function s(u){var l;return o[u]=!0,b.each(e[u]||[],function(e,u){var c=u(n,r,i);return"string"!=typeof c||a||o[c]?a?!(l=c):t:(n.dataTypes.unshift(c),s(c),!1)}),l}return s(n.dataTypes[0])||!o["*"]&&s("*")}function Mn(e,n){var r,i,o=b.ajaxSettings.flatOptions||{};for(i in n)n[i]!==t&&((o[i]?e:r||(r={}))[i]=n[i]);return r&&b.extend(!0,e,r),e}b.fn.load=function(e,n,r){if("string"!=typeof e&&Sn)return Sn.apply(this,arguments);var i,o,a,s=this,u=e.indexOf(" ");return u>=0&&(i=e.slice(u,e.length),e=e.slice(0,u)),b.isFunction(n)?(r=n,n=t):n&&"object"==typeof n&&(a="POST"),s.length>0&&b.ajax({
 url:e,type:a,dataType:"html",data:n}).done(function(e){o=arguments,s.html(i?b("<div>").append(b.parseHTML(e)).find(i):e)}).complete(r&&function(e,t){s.each(r,o||[e.responseText,t,e])}),this},b.each(["ajaxStart","ajaxStop","ajaxComplete","ajaxError","ajaxSuccess","ajaxSend"],function(e,t){b.fn[t]=function(e){return this.on(t,e)}}),b.each(["get","post"],function(e,n){b[n]=function(e,r,i,o){return b.isFunction(r)&&(o=o||i,i=r,r=t),b.ajax({url:e,type:n,dataType:o,data:r,success:i})}}),b.extend({active:0,lastModified:{},etag:{},ajaxSettings:{url:yn,type:"GET",isLocal:Nn.test(mn[1]),global:!0,processData:!0,async:!0,contentType:"application/x-www-form-urlencoded; charset=UTF-8",accepts:{"*":Dn,text:"text/plain",html:"text/html",xml:"application/xml, text/xml",json:"application/json, text/javascript"},contents:{xml:/xml/,html:/html/,json:/json/},responseFields:{xml:"responseXML",text:"responseText"},converters:{"* text":e.String,"text html":!0,"text json":b.parseJSON,"text xml":b.parseXML}
 ,flatOptions:{url:!0,context:!0}},ajaxSetup:function(e,t){return t?Mn(Mn(e,b.ajaxSettings),t):Mn(b.ajaxSettings,e)},ajaxPrefilter:Hn(An),ajaxTransport:Hn(jn),ajax:function(e,n){"object"==typeof e&&(n=e,e=t),n=n||{};var r,i,o,a,s,u,l,c,p=b.ajaxSetup({},n),f=p.context||p,d=p.context&&(f.nodeType||f.jquery)?b(f):b.event,h=b.Deferred(),g=b.Callbacks("once memory"),m=p.statusCode||{},y={},v={},x=0,T="canceled",N={readyState:0,getResponseHeader:function(e){var t;if(2===x){if(!c){c={};while(t=Tn.exec(a))c[t[1].toLowerCase()]=t[2]}t=c[e.toLowerCase()]}return null==t?null:t},getAllResponseHeaders:function(){return 2===x?a:null},setRequestHeader:function(e,t){var n=e.toLowerCase();return x||(e=v[n]=v[n]||e,y[e]=t),this},overrideMimeType:function(e){return x||(p.mimeType=e),this},statusCode:function(e){var t;if(e)if(2>x)for(t in e)m[t]=[m[t],e[t]];else N.always(e[N.status]);return this},abort:function(e){var t=e||T;return l&&l.abort(t),k(0,t),this}};if(h.promise(N).complete=g.add,N.success=N.d
 one,N.error=N.fail,p.url=((e||p.url||yn)+"").replace(xn,"").replace(kn,mn[1]+"//"),p.type=n.method||n.type||p.method||p.type,p.dataTypes=b.trim(p.dataType||"*").toLowerCase().match(w)||[""],null==p.crossDomain&&(r=En.exec(p.url.toLowerCase()),p.crossDomain=!(!r||r[1]===mn[1]&&r[2]===mn[2]&&(r[3]||("http:"===r[1]?80:443))==(mn[3]||("http:"===mn[1]?80:443)))),p.data&&p.processData&&"string"!=typeof p.data&&(p.data=b.param(p.data,p.traditional)),qn(An,p,n,N),2===x)return N;u=p.global,u&&0===b.active++&&b.event.trigger("ajaxStart"),p.type=p.type.toUpperCase(),p.hasContent=!Cn.test(p.type),o=p.url,p.hasContent||(p.data&&(o=p.url+=(bn.test(o)?"&":"?")+p.data,delete p.data),p.cache===!1&&(p.url=wn.test(o)?o.replace(wn,"$1_="+vn++):o+(bn.test(o)?"&":"?")+"_="+vn++)),p.ifModified&&(b.lastModified[o]&&N.setRequestHeader("If-Modified-Since",b.lastModified[o]),b.etag[o]&&N.setRequestHeader("If-None-Match",b.etag[o])),(p.data&&p.hasContent&&p.contentType!==!1||n.contentType)&&N.setRequestHeader(
 "Content-Type",p.contentType),N.setRequestHeader("Accept",p.dataTypes[0]&&p.accepts[p.dataTypes[0]]?p.accepts[p.dataTypes[0]]+("*"!==p.dataTypes[0]?", "+Dn+"; q=0.01":""):p.accepts["*"]);for(i in p.headers)N.setRequestHeader(i,p.headers[i]);if(p.beforeSend&&(p.beforeSend.call(f,N,p)===!1||2===x))return N.abort();T="abort";for(i in{success:1,error:1,complete:1})N[i](p[i]);if(l=qn(jn,p,n,N)){N.readyState=1,u&&d.trigger("ajaxSend",[N,p]),p.async&&p.timeout>0&&(s=setTimeout(function(){N.abort("timeout")},p.timeout));try{x=1,l.send(y,k)}catch(C){if(!(2>x))throw C;k(-1,C)}}else k(-1,"No Transport");function k(e,n,r,i){var c,y,v,w,T,C=n;2!==x&&(x=2,s&&clearTimeout(s),l=t,a=i||"",N.readyState=e>0?4:0,r&&(w=_n(p,N,r)),e>=200&&300>e||304===e?(p.ifModified&&(T=N.getResponseHeader("Last-Modified"),T&&(b.lastModified[o]=T),T=N.getResponseHeader("etag"),T&&(b.etag[o]=T)),204===e?(c=!0,C="nocontent"):304===e?(c=!0,C="notmodified"):(c=Fn(p,w),C=c.state,y=c.data,v=c.error,c=!v)):(v=C,(e||!C)&&(C="er
 ror",0>e&&(e=0))),N.status=e,N.statusText=(n||C)+"",c?h.resolveWith(f,[y,C,N]):h.rejectWith(f,[N,C,v]),N.statusCode(m),m=t,u&&d.trigger(c?"ajaxSuccess":"ajaxError",[N,p,c?y:v]),g.fireWith(f,[N,C]),u&&(d.trigger("ajaxComplete",[N,p]),--b.active||b.event.trigger("ajaxStop")))}return N},getScript:function(e,n){return b.get(e,t,n,"script")},getJSON:function(e,t,n){return b.get(e,t,n,"json")}});function _n(e,n,r){var i,o,a,s,u=e.contents,l=e.dataTypes,c=e.responseFields;for(s in c)s in r&&(n[c[s]]=r[s]);while("*"===l[0])l.shift(),o===t&&(o=e.mimeType||n.getResponseHeader("Content-Type"));if(o)for(s in u)if(u[s]&&u[s].test(o)){l.unshift(s);break}if(l[0]in r)a=l[0];else{for(s in r){if(!l[0]||e.converters[s+" "+l[0]]){a=s;break}i||(i=s)}a=a||i}return a?(a!==l[0]&&l.unshift(a),r[a]):t}function Fn(e,t){var n,r,i,o,a={},s=0,u=e.dataTypes.slice(),l=u[0];if(e.dataFilter&&(t=e.dataFilter(t,e.dataType)),u[1])for(i in e.converters)a[i.toLowerCase()]=e.converters[i];for(;r=u[++s];)if("*"!==r){if("*"
 !==l&&l!==r){if(i=a[l+" "+r]||a["* "+r],!i)for(n in a)if(o=n.split(" "),o[1]===r&&(i=a[l+" "+o[0]]||a["* "+o[0]])){i===!0?i=a[n]:a[n]!==!0&&(r=o[0],u.splice(s--,0,r));break}if(i!==!0)if(i&&e["throws"])t=i(t);else try{t=i(t)}catch(c){return{state:"parsererror",error:i?c:"No conversion from "+l+" to "+r}}}l=r}return{state:"success",data:t}}b.ajaxSetup({accepts:{script:"text/javascript, application/javascript, application/ecmascript, application/x-ecmascript"},contents:{script:/(?:java|ecma)script/},converters:{"text script":function(e){return b.globalEval(e),e}}}),b.ajaxPrefilter("script",function(e){e.cache===t&&(e.cache=!1),e.crossDomain&&(e.type="GET",e.global=!1)}),b.ajaxTransport("script",function(e){if(e.crossDomain){var n,r=o.head||b("head")[0]||o.documentElement;return{send:function(t,i){n=o.createElement("script"),n.async=!0,e.scriptCharset&&(n.charset=e.scriptCharset),n.src=e.url,n.onload=n.onreadystatechange=function(e,t){(t||!n.readyState||/loaded|complete/.test(n.readySta
 te))&&(n.onload=n.onreadystatechange=null,n.parentNode&&n.parentNode.removeChild(n),n=null,t||i(200,"success"))},r.insertBefore(n,r.firstChild)},abort:function(){n&&n.onload(t,!0)}}}});var On=[],Bn=/(=)\?(?=&|$)|\?\?/;b.ajaxSetup({jsonp:"callback",jsonpCallback:function(){var e=On.pop()||b.expando+"_"+vn++;return this[e]=!0,e}}),b.ajaxPrefilter("json jsonp",function(n,r,i){var o,a,s,u=n.jsonp!==!1&&(Bn.test(n.url)?"url":"string"==typeof n.data&&!(n.contentType||"").indexOf("application/x-www-form-urlencoded")&&Bn.test(n.data)&&"data");return u||"jsonp"===n.dataTypes[0]?(o=n.jsonpCallback=b.isFunction(n.jsonpCallback)?n.jsonpCallback():n.jsonpCallback,u?n[u]=n[u].replace(Bn,"$1"+o):n.jsonp!==!1&&(n.url+=(bn.test(n.url)?"&":"?")+n.jsonp+"="+o),n.converters["script json"]=function(){return s||b.error(o+" was not called"),s[0]},n.dataTypes[0]="json",a=e[o],e[o]=function(){s=arguments},i.always(function(){e[o]=a,n[o]&&(n.jsonpCallback=r.jsonpCallback,On.push(o)),s&&b.isFunction(a)&&a(s[0
 ]),s=a=t}),"script"):t});var Pn,Rn,Wn=0,$n=e.ActiveXObject&&function(){var e;for(e in Pn)Pn[e](t,!0)};function In(){try{return new e.XMLHttpRequest}catch(t){}}function zn(){try{return new e.ActiveXObject("Microsoft.XMLHTTP")}catch(t){}}b.ajaxSettings.xhr=e.ActiveXObject?function(){return!this.isLocal&&In()||zn()}:In,Rn=b.ajaxSettings.xhr(),b.support.cors=!!Rn&&"withCredentials"in Rn,Rn=b.support.ajax=!!Rn,Rn&&b.ajaxTransport(function(n){if(!n.crossDomain||b.support.cors){var r;return{send:function(i,o){var a,s,u=n.xhr();if(n.username?u.open(n.type,n.url,n.async,n.username,n.password):u.open(n.type,n.url,n.async),n.xhrFields)for(s in n.xhrFields)u[s]=n.xhrFields[s];n.mimeType&&u.overrideMimeType&&u.overrideMimeType(n.mimeType),n.crossDomain||i["X-Requested-With"]||(i["X-Requested-With"]="XMLHttpRequest");try{for(s in i)u.setRequestHeader(s,i[s])}catch(l){}u.send(n.hasContent&&n.data||null),r=function(e,i){var s,l,c,p;try{if(r&&(i||4===u.readyState))if(r=t,a&&(u.onreadystatechange=b.n
 oop,$n&&delete Pn[a]),i)4!==u.readyState&&u.abort();else{p={},s=u.status,l=u.getAllResponseHeaders(),"string"==typeof u.responseText&&(p.text=u.responseText);try{c=u.statusText}catch(f){c=""}s||!n.isLocal||n.crossDomain?1223===s&&(s=204):s=p.text?200:404}}catch(d){i||o(-1,d)}p&&o(s,c,p,l)},n.async?4===u.readyState?setTimeout(r):(a=++Wn,$n&&(Pn||(Pn={},b(e).unload($n)),Pn[a]=r),u.onreadystatechange=r):r()},abort:function(){r&&r(t,!0)}}}});var Xn,Un,Vn=/^(?:toggle|show|hide)$/,Yn=RegExp("^(?:([+-])=|)("+x+")([a-z%]*)$","i"),Jn=/queueHooks$/,Gn=[nr],Qn={"*":[function(e,t){var n,r,i=this.createTween(e,t),o=Yn.exec(t),a=i.cur(),s=+a||0,u=1,l=20;if(o){if(n=+o[2],r=o[3]||(b.cssNumber[e]?"":"px"),"px"!==r&&s){s=b.css(i.elem,e,!0)||n||1;do u=u||".5",s/=u,b.style(i.elem,e,s+r);while(u!==(u=i.cur()/a)&&1!==u&&--l)}i.unit=r,i.start=s,i.end=o[1]?s+(o[1]+1)*n:n}return i}]};function Kn(){return setTimeout(function(){Xn=t}),Xn=b.now()}function Zn(e,t){b.each(t,function(t,n){var r=(Qn[t]||[]).concat
 (Qn["*"]),i=0,o=r.length;for(;o>i;i++)if(r[i].call(e,t,n))return})}function er(e,t,n){var r,i,o=0,a=Gn.length,s=b.Deferred().always(function(){delete u.elem}),u=function(){if(i)return!1;var t=Xn||Kn(),n=Math.max(0,l.startTime+l.duration-t),r=n/l.duration||0,o=1-r,a=0,u=l.tweens.length;for(;u>a;a++)l.tweens[a].run(o);return s.notifyWith(e,[l,o,n]),1>o&&u?n:(s.resolveWith(e,[l]),!1)},l=s.promise({elem:e,props:b.extend({},t),opts:b.extend(!0,{specialEasing:{}},n),originalProperties:t,originalOptions:n,startTime:Xn||Kn(),duration:n.duration,tweens:[],createTween:function(t,n){var r=b.Tween(e,l.opts,t,n,l.opts.specialEasing[t]||l.opts.easing);return l.tweens.push(r),r},stop:function(t){var n=0,r=t?l.tweens.length:0;if(i)return this;for(i=!0;r>n;n++)l.tweens[n].run(1);return t?s.resolveWith(e,[l,t]):s.rejectWith(e,[l,t]),this}}),c=l.props;for(tr(c,l.opts.specialEasing);a>o;o++)if(r=Gn[o].call(l,e,c,l.opts))return r;return Zn(l,c),b.isFunction(l.opts.start)&&l.opts.start.call(e,l),b.fx.tim
 er(b.extend(u,{elem:e,anim:l,queue:l.opts.queue})),l.progress(l.opts.progress).done(l.opts.done,l.opts.complete).fail(l.opts.fail).always(l.opts.always)}function tr(e,t){var n,r,i,o,a;for(i in e)if(r=b.camelCase(i),o=t[r],n=e[i],b.isArray(n)&&(o=n[1],n=e[i]=n[0]),i!==r&&(e[r]=n,delete e[i]),a=b.cssHooks[r],a&&"expand"in a){n=a.expand(n),delete e[r];for(i in n)i in e||(e[i]=n[i],t[i]=o)}else t[r]=o}b.Animation=b.extend(er,{tweener:function(e,t){b.isFunction(e)?(t=e,e=["*"]):e=e.split(" ");var n,r=0,i=e.length;for(;i>r;r++)n=e[r],Qn[n]=Qn[n]||[],Qn[n].unshift(t)},prefilter:function(e,t){t?Gn.unshift(e):Gn.push(e)}});function nr(e,t,n){var r,i,o,a,s,u,l,c,p,f=this,d=e.style,h={},g=[],m=e.nodeType&&nn(e);n.queue||(c=b._queueHooks(e,"fx"),null==c.unqueued&&(c.unqueued=0,p=c.empty.fire,c.empty.fire=function(){c.unqueued||p()}),c.unqueued++,f.always(function(){f.always(function(){c.unqueued--,b.queue(e,"fx").length||c.empty.fire()})})),1===e.nodeType&&("height"in t||"width"in t)&&(n.overfl
 ow=[d.overflow,d.overflowX,d.overflowY],"inline"===b.css(e,"display")&&"none"===b.css(e,"float")&&(b.support.inlineBlockNeedsLayout&&"inline"!==un(e.nodeName)?d.zoom=1:d.display="inline-block")),n.overflow&&(d.overflow="hidden",b.support.shrinkWrapBlocks||f.always(function(){d.overflow=n.overflow[0],d.overflowX=n.overflow[1],d.overflowY=n.overflow[2]}));for(i in t)if(a=t[i],Vn.exec(a)){if(delete t[i],u=u||"toggle"===a,a===(m?"hide":"show"))continue;g.push(i)}if(o=g.length){s=b._data(e,"fxshow")||b._data(e,"fxshow",{}),"hidden"in s&&(m=s.hidden),u&&(s.hidden=!m),m?b(e).show():f.done(function(){b(e).hide()}),f.done(function(){var t;b._removeData(e,"fxshow");for(t in h)b.style(e,t,h[t])});for(i=0;o>i;i++)r=g[i],l=f.createTween(r,m?s[r]:0),h[r]=s[r]||b.style(e,r),r in s||(s[r]=l.start,m&&(l.end=l.start,l.start="width"===r||"height"===r?1:0))}}function rr(e,t,n,r,i){return new rr.prototype.init(e,t,n,r,i)}b.Tween=rr,rr.prototype={constructor:rr,init:function(e,t,n,r,i,o){this.elem=e,this
 .prop=n,this.easing=i||"swing",this.options=t,this.start=this.now=this.cur(),this.end=r,this.unit=o||(b.cssNumber[n]?"":"px")},cur:function(){var e=rr.propHooks[this.prop];return e&&e.get?e.get(this):rr.propHooks._default.get(this)},run:function(e){var t,n=rr.propHooks[this.prop];return this.pos=t=this.options.duration?b.easing[this.easing](e,this.options.duration*e,0,1,this.options.duration):e,this.now=(this.end-this.start)*t+this.start,this.options.step&&this.options.step.call(this.elem,this.now,this),n&&n.set?n.set(this):rr.propHooks._default.set(this),this}},rr.prototype.init.prototype=rr.prototype,rr.propHooks={_default:{get:function(e){var t;return null==e.elem[e.prop]||e.elem.style&&null!=e.elem.style[e.prop]?(t=b.css(e.elem,e.prop,""),t&&"auto"!==t?t:0):e.elem[e.prop]},set:function(e){b.fx.step[e.prop]?b.fx.step[e.prop](e):e.elem.style&&(null!=e.elem.style[b.cssProps[e.prop]]||b.cssHooks[e.prop])?b.style(e.elem,e.prop,e.now+e.unit):e.elem[e.prop]=e.now}}},rr.propHooks.scroll
 Top=rr.propHooks.scrollLeft={set:function(e){e.elem.nodeType&&e.elem.parentNode&&(e.elem[e.prop]=e.now)}},b.each(["toggle","show","hide"],function(e,t){var n=b.fn[t];b.fn[t]=function(e,r,i){return null==e||"boolean"==typeof e?n.apply(this,arguments):this.animate(ir(t,!0),e,r,i)}}),b.fn.extend({fadeTo:function(e,t,n,r){return this.filter(nn).css("opacity",0).show().end().animate({opacity:t},e,n,r)},animate:function(e,t,n,r){var i=b.isEmptyObject(e),o=b.speed(t,n,r),a=function(){var t=er(this,b.extend({},e),o);a.finish=function(){t.stop(!0)},(i||b._data(this,"finish"))&&t.stop(!0)};return a.finish=a,i||o.queue===!1?this.each(a):this.queue(o.queue,a)},stop:function(e,n,r){var i=function(e){var t=e.stop;delete e.stop,t(r)};return"string"!=typeof e&&(r=n,n=e,e=t),n&&e!==!1&&this.queue(e||"fx",[]),this.each(function(){var t=!0,n=null!=e&&e+"queueHooks",o=b.timers,a=b._data(this);if(n)a[n]&&a[n].stop&&i(a[n]);else for(n in a)a[n]&&a[n].stop&&Jn.test(n)&&i(a[n]);for(n=o.length;n--;)o[n].ele
 m!==this||null!=e&&o[n].queue!==e||(o[n].anim.stop(r),t=!1,o.splice(n,1));(t||!r)&&b.dequeue(this,e)})},finish:function(e){return e!==!1&&(e=e||"fx"),this.each(function(){var t,n=b._data(this),r=n[e+"queue"],i=n[e+"queueHooks"],o=b.timers,a=r?r.length:0;for(n.finish=!0,b.queue(this,e,[]),i&&i.cur&&i.cur.finish&&i.cur.finish.call(this),t=o.length;t--;)o[t].elem===this&&o[t].queue===e&&(o[t].anim.stop(!0),o.splice(t,1));for(t=0;a>t;t++)r[t]&&r[t].finish&&r[t].finish.call(this);delete n.finish})}});function ir(e,t){var n,r={height:e},i=0;for(t=t?1:0;4>i;i+=2-t)n=Zt[i],r["margin"+n]=r["padding"+n]=e;return t&&(r.opacity=r.width=e),r}b.each({slideDown:ir("show"),slideUp:ir("hide"),slideToggle:ir("toggle"),fadeIn:{opacity:"show"},fadeOut:{opacity:"hide"},fadeToggle:{opacity:"toggle"}},function(e,t){b.fn[e]=function(e,n,r){return this.animate(t,e,n,r)}}),b.speed=function(e,t,n){var r=e&&"object"==typeof e?b.extend({},e):{complete:n||!n&&t||b.isF

<TRUNCATED>

[24/52] [abbrv] kylin git commit: KYLIN-1285 Initial release note for 2.0-alpha

Posted by li...@apache.org.
KYLIN-1285 Initial release note for 2.0-alpha


Project: http://git-wip-us.apache.org/repos/asf/kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/7d3c4f45
Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/7d3c4f45
Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/7d3c4f45

Branch: refs/heads/1.x-HBase1.1.3
Commit: 7d3c4f456180408279ed1a8eb2a22b09932c2827
Parents: eb5deb3
Author: Yang Li <li...@apache.org>
Authored: Tue Feb 9 21:41:43 2016 +0800
Committer: Yang Li <li...@apache.org>
Committed: Tue Feb 9 21:41:43 2016 +0800

----------------------------------------------------------------------
 website/_docs/release_notes.md | 304 +++++++++++++++++++++++++++++++++++-
 1 file changed, 303 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/7d3c4f45/website/_docs/release_notes.md
----------------------------------------------------------------------
diff --git a/website/_docs/release_notes.md b/website/_docs/release_notes.md
index 04cf209..3adea81 100644
--- a/website/_docs/release_notes.md
+++ b/website/_docs/release_notes.md
@@ -3,7 +3,7 @@ layout: docs
 title:  Apache Kylin™ Release Notes
 categories: gettingstarted
 permalink: /docs/release_notes.html
-version: v1.2
+version: v2.0
 since: v0.7.1
 ---
 
@@ -16,6 +16,308 @@ or send to Apache Kylin mailing list:
 * User relative: [user@kylin.apache.org](mailto:user@kylin.apache.org)
 * Development relative: [dev@kylin.apache.org](mailto:dev@kylin.apache.org)
 
+
+## v2.0-alpha - 2016-02-09
+_Tag:_ [kylin-2.0-alpha](https://github.com/apache/kylin/tree/kylin-2.0-alpha)
+
+__Highlights__
+
+    * [KYLIN-875] - A plugin-able architecture, to allow alternative cube engine / storage engine / data source.
+    * [KYLIN-1245] - A better MR cubing algorithm, about 1.5 times faster than 1.x by comparing hundreds of jobs.
+    * [KYLIN-942] - A better storage engine, makes query roughly 2 times faster (especially for slow queries) than 1.x by comparing tens of thousands sqls.
+    * [KYLIN-738] - Streaming cubing EXPERIMENTAL support, source from kafka, build cube in-mem at minutes interval
+    * [KYLIN-943] - TopN pre-calculation (more UDFs coming)
+    * [KYLIN-1065] - ODBC compatible with Tableau 9.1, MS Excel, MS PowerBI
+    * [KYLIN-1219] - Kylin support SSO with Spring SAML
+
+__Below generated from JIRA system, pending manual revision.__
+
+__New Feature__
+
+    * [KYLIN-196] - Support Job Priority
+    * [KYLIN-528] - Build job flow for Inverted Index building
+    * [KYLIN-596] - Support Excel and Power BI
+    * [KYLIN-599] - Near real-time support
+    * [KYLIN-603] - Add mem store for seconds data latency
+    * [KYLIN-606] - Block level index for Inverted-Index
+    * [KYLIN-607] - More efficient cube building
+    * [KYLIN-609] - Add Hybrid as a federation of Cube and Inverted-index realization
+    * [KYLIN-625] - Create GridTable, a data structure that abstracts vertical and horizontal partition of a table
+    * [KYLIN-728] - IGTStore implementation which use disk when memory runs short
+    * [KYLIN-738] - StreamingOLAP
+    * [KYLIN-749] - support timestamp type in II and cube
+    * [KYLIN-774] - Automatically merge cube segments
+    * [KYLIN-868] - add a metadata backup/restore script in bin folder
+    * [KYLIN-886] - Data Retention for streaming data
+    * [KYLIN-906] - cube retention
+    * [KYLIN-943] - Approximate TopN supported by Cube
+    * [KYLIN-986] - Generalize Streaming scripts and put them into code repository 
+    * [KYLIN-1219] - Kylin support SSO with Spring SAML
+    * [KYLIN-1277] - Upgrade tool to put old-version cube and new-version cube into a hybrid model 
+
+__Improvement__
+
+    * [KYLIN-225] - Support edit "cost" of cube
+    * [KYLIN-589] - Cleanup Intermediate hive table after cube build
+    * [KYLIN-623] - update Kylin UI Style to latest AdminLTE
+    * [KYLIN-633] - Support Timestamp for cube partition
+    * [KYLIN-649] -  move the cache layer from service tier back to storage tier
+    * [KYLIN-655] - Migrate cube storage (query side) to use GridTable API
+    * [KYLIN-663] - Push time condition down to ii endpoint
+    * [KYLIN-668] - Out of memory in mapper when building cube in mem
+    * [KYLIN-671] - Implement fine grained cache for cube and ii
+    * [KYLIN-673] - Performance tuning for In-Mem cubing
+    * [KYLIN-674] - IIEndpoint return metrics as well
+    * [KYLIN-675] - cube&model designer refactor
+    * [KYLIN-678] - optimize RowKeyColumnIO
+    * [KYLIN-697] - Reorganize all test cases to unit test and integration tests
+    * [KYLIN-702] - When Kylin create the flat hive table, it generates large number of small files in HDFS 
+    * [KYLIN-708] - replace BitSet for AggrKey
+    * [KYLIN-712] - some enhancement after code review
+    * [KYLIN-717] - optimize OLAPEnumerator.convertCurrentRow()
+    * [KYLIN-718] - replace aliasMap in storage context with a clear specified return column list
+    * [KYLIN-719] - bundle statistics info in endpoint response
+    * [KYLIN-720] - Optimize endpoint's response structure to suit with no-dictionary data
+    * [KYLIN-721] - streaming cli support third-party streammessage parser
+    * [KYLIN-726] - add remote cli port configuration for KylinConfig
+    * [KYLIN-729] - IIEndpoint eliminate the non-aggregate routine
+    * [KYLIN-734] - Push cache layer to each storage engine
+    * [KYLIN-752] - Improved IN clause performance
+    * [KYLIN-753] - Make the dependency on hbase-common to "provided"
+    * [KYLIN-755] - extract copying libs from prepare.sh so that it can be reused
+    * [KYLIN-760] - Improve the hasing performance in Sampling cuboid size
+    * [KYLIN-772] - Continue cube job when hive query return empty resultset
+    * [KYLIN-773] - performance is slow list jobs
+    * [KYLIN-783] - update hdp version in test cases to 2.2.4
+    * [KYLIN-796] - Add REST API to trigger storage cleanup/GC
+    * [KYLIN-809] - Streaming cubing allow multiple kafka clusters/topics
+    * [KYLIN-816] - Allow gap in cube segments, for streaming case
+    * [KYLIN-822] - list cube overview in one page
+    * [KYLIN-823] - replace fk on fact table on rowkey & aggregation group generate
+    * [KYLIN-838] - improve performance of job query
+    * [KYLIN-844] - add backdoor toggles to control query behavior 
+    * [KYLIN-845] - Enable coprocessor even when there is memory hungry distinct count
+    * [KYLIN-858] - add snappy compression support
+    * [KYLIN-866] - Confirm with user when he selects empty segments to merge
+    * [KYLIN-869] - Enhance mail notification
+    * [KYLIN-870] - Speed up hbase segments info by caching
+    * [KYLIN-871] - growing dictionary for streaming case
+    * [KYLIN-874] - script for fill streaming gap automatically
+    * [KYLIN-875] - Decouple with Hadoop to allow alternative Input / Build Engine / Storage
+    * [KYLIN-879] - add a tool to collect orphan hbases 
+    * [KYLIN-880] -  Kylin should change the default folder from /tmp to user configurable destination
+    * [KYLIN-881] - Upgrade Calcite to 1.3.0
+    * [KYLIN-882] - check access to kylin.hdfs.working.dir
+    * [KYLIN-883] - Using configurable option for Hive intermediate tables created by Kylin job
+    * [KYLIN-893] - Remove the dependency on quartz and metrics
+    * [KYLIN-895] - Add "retention_range" attribute for cube instance, and automatically drop the oldest segment when exceeds retention
+    * [KYLIN-896] - Clean ODBC code, add them into main repository and write docs to help compiling
+    * [KYLIN-901] - Add tool for cleanup Kylin metadata storage
+    * [KYLIN-902] - move streaming related parameters into StreamingConfig
+    * [KYLIN-903] - automate metadata cleanup job
+    * [KYLIN-909] - Adapt GTStore to hbase endpoint
+    * [KYLIN-919] - more friendly UI for 0.8
+    * [KYLIN-922] - Enforce same code style for both intellij and eclipse user
+    * [KYLIN-926] - Make sure Kylin leaves no garbage files in local OS and HDFS/HBASE
+    * [KYLIN-927] - Real time cubes merging skipping gaps
+    * [KYLIN-933] - friendly UI to use data model
+    * [KYLIN-938] - add friendly tip to page when rest request failed
+    * [KYLIN-942] - Cube parallel scan on Hbase
+    * [KYLIN-956] - Allow users to configure hbase compression algorithm in kylin.properties
+    * [KYLIN-957] - Support HBase in a separate cluster
+    * [KYLIN-960] - Split storage module to core-storage and storage-hbase
+    * [KYLIN-973] - add a tool to analyse streaming output logs
+    * [KYLIN-984] - Behavior change in streaming data consuming
+    * [KYLIN-987] - Rename 0.7-staging and 0.8 branch
+    * [KYLIN-1014] - Support kerberos authentication while getting status from RM
+    * [KYLIN-1018] - make TimedJsonStreamParser default parser 
+    * [KYLIN-1019] - Remove v1 cube model classes from code repository
+    * [KYLIN-1021] - upload dependent jars of kylin to HDFS and set tmpjars
+    * [KYLIN-1025] - Save cube change is very slow
+    * [KYLIN-1036] - Code Clean, remove code which never used at front end
+    * [KYLIN-1041] - ADD Streaming UI 
+    * [KYLIN-1048] - CPU and memory killer in Cuboid.findById()
+    * [KYLIN-1058] - Remove "right join" during model creation
+    * [KYLIN-1061] - "kylin.sh start" should check whether kylin has already been running
+    * [KYLIN-1064] - restore disabled queries in KylinQueryTest.testVerifyQuery
+    * [KYLIN-1065] - ODBC driver support tableau 9.1
+    * [KYLIN-1068] - Optimize the memory footprint for TopN counter
+    * [KYLIN-1069] - update tip for 'Partition Column' on UI
+    * [KYLIN-1095] - Update AdminLTE to latest version
+    * [KYLIN-1096] - Deprecate minicluster in 2.x staging
+    * [KYLIN-1099] - Support dictionary of cardinality over 10 millions
+    * [KYLIN-1101] - Allow "YYYYMMDD" as a date partition column
+    * [KYLIN-1105] - Cache in AbstractRowKeyEncoder.createInstance() is useless
+    * [KYLIN-1116] - Use local dictionary for InvertedIndex batch building
+    * [KYLIN-1119] - refine find-hive-dependency.sh to correctly get hcatalog path
+    * [KYLIN-1126] - v2 storage(for parallel scan) backward compatibility with v1 storage
+    * [KYLIN-1135] - Pscan use share thread pool
+    * [KYLIN-1136] - Distinguish fast build mode and complete build mode
+    * [KYLIN-1139] - Hive job not starting due to error "conflicting lock present for default mode EXCLUSIVE "
+    * [KYLIN-1149] - When yarn return an incomplete job tracking URL, Kylin will fail to get job status
+    * [KYLIN-1154] - Load job page is very slow when there are a lot of history job
+    * [KYLIN-1157] - CubeMigrationCLI doesn't copy ACL
+    * [KYLIN-1160] - Set default logger appender of log4j for JDBC
+    * [KYLIN-1161] - Rest API /api/cubes?cubeName=  is doing fuzzy match instead of exact match
+    * [KYLIN-1162] - Enhance HadoopStatusGetter to be compatible with YARN-2605
+    * [KYLIN-1190] - Make memory budget per query configurable
+    * [KYLIN-1234] - Cube ACL does not work
+    * [KYLIN-1235] - allow user to select dimension column as options when edit COUNT_DISTINCT measure
+    * [KYLIN-1237] - Revisit on cube size estimation
+    * [KYLIN-1239] - attribute each htable with team contact and owner name
+    * [KYLIN-1244] - In query window, enable fast copy&paste by double clicking tables/columns' names.
+    * [KYLIN-1245] - Switch between layer cubing and in-mem cubing according to stats
+    * [KYLIN-1246] - get cubes API update - offset,limit not required
+    * [KYLIN-1251] - add toggle event for tree label
+    * [KYLIN-1259] - Change font/background color of job progress
+    * [KYLIN-1265] - Make sure 2.0 query is no slower than 1.0
+    * [KYLIN-1266] - Tune 2.0 release package size
+    * [KYLIN-1267] - Check Kryo performance when spilling aggregation cache
+    * [KYLIN-1268] - Fix 2 kylin logs
+    * [KYLIN-1270] - improve TimedJsonStreamParser to support month_start,quarter_start,year_start
+    * [KYLIN-1281] - Add "partition_date_end", and move "partition_date_start" into cube descriptor
+    * [KYLIN-1283] - Replace GTScanRequest's SerDer form Kryo to manual 
+    * [KYLIN-1287] - UI update for streaming build action
+    * [KYLIN-1297] - Diagnose query performance issues in 2.x versions
+    * [KYLIN-1301] - fix segment pruning failure in 2.x versions
+    * [KYLIN-1308] - query storage v2 enable parallel cube visiting
+    * [KYLIN-1312] - Enhance DeployCoprocessorCLI to support Cube level filter
+    * [KYLIN-1318] - enable gc log for kylin server instance
+    * [KYLIN-1323] - Improve performance of converting data to hfile
+    * [KYLIN-1327] - Tool for batch updating host information of htables
+    * [KYLIN-1334] - allow truncating string for fixed length dimensions
+    * [KYLIN-1341] - Display JSON of Data Model in the dialog
+    * [KYLIN-1350] - hbase Result.binarySearch is found to be problematic in concurrent environments
+    * [KYLIN-1368] - JDBC Driver is not generic to restAPI json result
+
+__Bug__
+
+    * [KYLIN-404] - Can't get cube source record size.
+    * [KYLIN-457] - log4j error and dup lines in kylin.log
+    * [KYLIN-521] - No verification even if join condition is invalid
+    * [KYLIN-632] - "kylin.sh stop" doesn't check whether KYLIN_HOME was set
+    * [KYLIN-635] - IN clause within CASE when is not working
+    * [KYLIN-656] - REST API get cube desc NullPointerException when cube is not exists
+    * [KYLIN-660] - Make configurable of dictionary cardinality cap
+    * [KYLIN-665] - buffer error while in mem cubing
+    * [KYLIN-688] - possible memory leak for segmentIterator
+    * [KYLIN-731] - Parallel stream build will throw OOM
+    * [KYLIN-740] - Slowness with many IN() values
+    * [KYLIN-747] - bad query performance when IN clause contains a value doesn't exist in the dictionary
+    * [KYLIN-748] - II returned result not correct when decimal omits precision and scal
+    * [KYLIN-751] - Max on negative double values is not working
+    * [KYLIN-766] - round BigDecimal according to the DataType scale
+    * [KYLIN-769] - empty segment build fail due to no dictionary 
+    * [KYLIN-771] - query cache is not evicted when metadata changes
+    * [KYLIN-778] - can't build cube after package to binary 
+    * [KYLIN-780] - Upgrade Calcite to 1.0
+    * [KYLIN-797] - Cuboid cache will cache massive invalid cuboid if existed many cubes which already be deleted 
+    * [KYLIN-801] - fix remaining issues on query cache and storage cache
+    * [KYLIN-805] - Drop useless Hive intermediate table and HBase tables in the last step of cube build/merge
+    * [KYLIN-807] - Avoid write conflict between job engine and stream cube builder
+    * [KYLIN-817] - Support Extract() on timestamp column
+    * [KYLIN-824] - Cube Build fails if lookup table doesn't have any files under HDFS location
+    * [KYLIN-828] - kylin still use ldap profile when comment the line "kylin.sandbox=false" in kylin.properties
+    * [KYLIN-834] - optimize StreamingUtil binary search perf
+    * [KYLIN-837] - fix submit build type when refresh cube
+    * [KYLIN-873] - cancel button does not work when [resume][discard] job
+    * [KYLIN-889] - Support more than one HDFS files of lookup table
+    * [KYLIN-897] - Update CubeMigrationCLI to copy data model info
+    * [KYLIN-898] - "CUBOID_CACHE" in Cuboid.java never flushes
+    * [KYLIN-905] - Boolean type not supported
+    * [KYLIN-911] - NEW segments not DELETED when cancel BuildAndMerge Job
+    * [KYLIN-912] - $KYLIN_HOME/tomcat/temp folder takes much disk space after long run
+    * [KYLIN-913] - Cannot find rowkey column XXX in cube CubeDesc
+    * [KYLIN-914] - Scripts shebang should use /bin/bash
+    * [KYLIN-918] - Calcite throws "java.lang.Float cannot be cast to java.lang.Double" error while executing SQL
+    * [KYLIN-929] - can not sort cubes by [Source Records] at cubes list page
+    * [KYLIN-930] - can't see realizations under each project at project list page
+    * [KYLIN-934] - Negative number in SUM result and Kylin results not matching exactly Hive results
+    * [KYLIN-935] - always loading when try to view the log of the sub-step of cube build job
+    * [KYLIN-936] - can not see job step log 
+    * [KYLIN-944] - update doc about how to consume kylin API in javascript
+    * [KYLIN-946] - [UI] refresh page show no results when Project selected as [--Select All--]
+    * [KYLIN-950] - Web UI "Jobs" tab view the job reduplicated
+    * [KYLIN-951] - Drop RowBlock concept from GridTable general API
+    * [KYLIN-952] - User can trigger a Refresh job on an non-existing cube segment via REST API
+    * [KYLIN-967] - Dump running queries on memory shortage
+    * [KYLIN-975] - change kylin.job.hive.database.for.intermediatetable cause job to fail
+    * [KYLIN-978] - GarbageCollectionStep dropped Hive Intermediate Table but didn't drop external hdfs path
+    * [KYLIN-982] - package.sh should grep out "Download*" messages when determining version
+    * [KYLIN-983] - Query sql offset keyword bug
+    * [KYLIN-985] - Don't suppoprt aggregation AVG while executing SQL
+    * [KYLIN-991] - StorageCleanupJob may clean a newly created HTable in streaming cube building
+    * [KYLIN-992] - ConcurrentModificationException when initializing ResourceStore
+    * [KYLIN-1001] - Kylin generates wrong HDFS path in creating intermediate table
+    * [KYLIN-1004] - Dictionary with '' value cause cube merge to fail
+    * [KYLIN-1020] - Although "kylin.query.scan.threshold" is set, it still be restricted to less than 4 million 
+    * [KYLIN-1026] - Error message for git check is not correct in package.sh
+    * [KYLIN-1027] - HBase Token not added after KYLIN-1007
+    * [KYLIN-1033] - Error when joining two sub-queries
+    * [KYLIN-1039] - Filter like (A or false) yields wrong result
+    * [KYLIN-1047] - Upgrade to Calcite 1.4
+    * [KYLIN-1066] - Only 1 reducer is started in the "Build cube" step of MR_Engine_V2
+    * [KYLIN-1067] - Support get MapReduce Job status for ResourceManager HA Env
+    * [KYLIN-1075] - select [MeasureCol] from [FactTbl] is not supported
+    * [KYLIN-1078] - UI - Cannot have comments in the end of New Query textbox
+    * [KYLIN-1093] - Consolidate getCurrentHBaseConfiguration() and newHBaseConfiguration() in HadoopUtil
+    * [KYLIN-1106] - Can not send email caused by Build Base Cuboid Data step failed
+    * [KYLIN-1108] - Return Type Empty When Measure-> Count In Cube Design
+    * [KYLIN-1113] - Support TopN query in v2/CubeStorageQuery.java
+    * [KYLIN-1115] - Clean up ODBC driver code
+    * [KYLIN-1121] - ResourceTool download/upload does not work in binary package
+    * [KYLIN-1127] - Refactor CacheService
+    * [KYLIN-1137] - TopN measure need support dictionary merge
+    * [KYLIN-1138] - Bad CubeDesc signature cause segment be delete when enable a cube
+    * [KYLIN-1140] - Kylin's sample cube "kylin_sales_cube" couldn't be saved.
+    * [KYLIN-1151] - Menu items should be aligned when create new model
+    * [KYLIN-1152] - ResourceStore should read content and timestamp in one go
+    * [KYLIN-1153] - Upgrade is needed for cubedesc metadata from 1.x to 2.0
+    * [KYLIN-1171] - KylinConfig truncate bug
+    * [KYLIN-1179] - Cannot use String as partition column
+    * [KYLIN-1180] - Some NPE in Dictionary
+    * [KYLIN-1181] - Split metadata size exceeded when data got huge in one segment
+    * [KYLIN-1192] - Cannot edit data model desc without name change
+    * [KYLIN-1205] - hbase RpcClient java.io.IOException: Unexpected closed connection
+    * [KYLIN-1211] - Add 'Enable Cache' button in System page
+    * [KYLIN-1216] - Can't parse DateFormat like 'YYYYMMDD' correctly in query
+    * [KYLIN-1218] - java.lang.NullPointerException in MeasureTypeFactory when sync hive table
+    * [KYLIN-1220] - JsonMappingException: Can not deserialize instance of java.lang.String out of START_ARRAY
+    * [KYLIN-1225] - Only 15 cubes listed in the /models page
+    * [KYLIN-1226] - InMemCubeBuilder throw OOM for multiple HLLC measures
+    * [KYLIN-1230] - When CubeMigrationCLI copied ACL from one env to another, it may not work
+    * [KYLIN-1236] - redirect to home page when input invalid url
+    * [KYLIN-1250] - Got NPE when discarding a job
+    * [KYLIN-1260] - Job status labels are not in same style
+    * [KYLIN-1269] - Can not get last error message in email
+    * [KYLIN-1271] - Create streaming table layer will disappear if click on outside
+    * [KYLIN-1274] - Query from JDBC is partial results by default
+    * [KYLIN-1282] - Comparison filter on Date/Time column not work for query
+    * [KYLIN-1289] - Click on subsequent wizard steps doesn't work when editing existing cube or model
+    * [KYLIN-1303] - Error when in-mem cubing on empty data source which has boolean columns
+    * [KYLIN-1306] - Null strings are not applied during fast cubing
+    * [KYLIN-1314] - Display issue for aggression groups 
+    * [KYLIN-1315] - UI: Cannot add normal dimension when creating new cube 
+    * [KYLIN-1316] - Wrong label in Dialog CUBE REFRESH CONFIRM
+    * [KYLIN-1317] - Kill underlying running hadoop job while discard a job
+    * [KYLIN-1328] - "UnsupportedOperationException" is thrown when remove a data model
+    * [KYLIN-1330] - UI create model: Press enter will go back to pre step
+    * [KYLIN-1336] - 404 errors of model page and api 'access/DataModelDesc' in console
+    * [KYLIN-1337] - Sort cube name doesn't work well 
+    * [KYLIN-1346] - IllegalStateException happens in SparkCubing
+    * [KYLIN-1347] - UI: cannot place cursor in front of the last dimension
+    * [KYLIN-1349] - 'undefined' is logged in console when adding lookup table
+    * [KYLIN-1352] - 'Cache already exists' exception in high-concurrency query situation
+    * [KYLIN-1356] - use exec-maven-plugin for IT environment provision
+    * [KYLIN-1357] - Cloned cube has build time information
+    * [KYLIN-1372] - Query using PrepareStatement failed with multi OR clause
+    * [KYLIN-1382] - CubeMigrationCLI reports error when migrate cube
+    * [KYLIN-1396] - minor bug in BigDecimalSerializer - avoidVerbose should be incremented each time when input scale is larger than given scale 
+    * [KYLIN-1400] - kylin.metadata.url with hbase namespace problem
+    * [KYLIN-1402] - StringIndexOutOfBoundsException in Kylin Hive Column Cardinality Job
+    * [KYLIN-1414] - Couldn't drag and drop rowkey, js error is thrown in browser console
+
+
 ## v1.2 - 2015-12-15
 _Tag:_ [kylin-1.2](https://github.com/apache/kylin/tree/kylin-1.2)
 


[49/52] [abbrv] kylin git commit: KYLIN-1330 use select box instead of typehead

Posted by li...@apache.org.
KYLIN-1330 use select box instead of typehead


Project: http://git-wip-us.apache.org/repos/asf/kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/0c120c34
Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/0c120c34
Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/0c120c34

Branch: refs/heads/1.x-HBase1.1.3
Commit: 0c120c34d0fc08df07c3161500aac73aea3eb9f3
Parents: 9d96cfa
Author: janzhongi <ji...@ebay.com>
Authored: Thu Feb 18 14:58:14 2016 +0800
Committer: janzhongi <ji...@ebay.com>
Committed: Thu Feb 18 14:58:14 2016 +0800

----------------------------------------------------------------------
 .../app/partials/cubeDesigner/data_model.html   | 21 +++++++++++++++++---
 1 file changed, 18 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/0c120c34/webapp/app/partials/cubeDesigner/data_model.html
----------------------------------------------------------------------
diff --git a/webapp/app/partials/cubeDesigner/data_model.html b/webapp/app/partials/cubeDesigner/data_model.html
index cf19c23..557eaa9 100644
--- a/webapp/app/partials/cubeDesigner/data_model.html
+++ b/webapp/app/partials/cubeDesigner/data_model.html
@@ -26,8 +26,15 @@
                 <b>Fact Table</b>
             </label>
             <div class="col-xs-12 col-sm-6">
-                <typeahead ng-if="state.mode=='edit'" items="tableModel.selectProjectTables" prompt="Fact Table Name"
-                           title="name" model="metaModel.model.fact_table" required="true"></typeahead>
+              <select chosen ng-model="metaModel.model.fact_table" ng-if="state.mode=='edit'"
+                      ng-options="table.name as table.name for table in tableModel.selectProjectTables"
+                      style="width:100%;"
+                      ng-required="true"
+                      data-placeholder="Fact Table Name"
+                      class="chosen-select">
+                <option value=""> -- Select Fact Table -- </option>
+              </select>
+
                 <span ng-if="state.mode=='view'">{{metaModel.model.fact_table}}</span>
             </div>
         </div>
@@ -113,7 +120,15 @@
                         <div class="row">
                             <label class="control-label col-xs-12 col-sm-3 no-padding-right font-color-default"><b>Lookup Table Name</b></label>
                             <div class="col-xs-12 col-sm-6">
-                                <typeahead items="tableModel.selectProjectTables" prompt="Lookup Table Name" title="name" model="newLookup.table"></typeahead>
+                              <select chosen ng-model="newLookup.table"
+                                      ng-options="table.name as table.name for table in tableModel.selectProjectTables"
+                                      style="width:100%;"
+                                      ng-required="true"
+                                      data-placeholder="Lookup Table Name"
+                                      class="chosen-select">
+                                <option value=""> -- Select Lookup Table -- </option>
+                              </select>
+
                             </div>
                         </div>
                     </div>


[46/52] [abbrv] kylin git commit: KYLIN-1423 fix HBase size precision issue

Posted by li...@apache.org.
KYLIN-1423 fix HBase size precision issue


Project: http://git-wip-us.apache.org/repos/asf/kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/5666d7f2
Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/5666d7f2
Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/5666d7f2

Branch: refs/heads/1.x-HBase1.1.3
Commit: 5666d7f238529529a1091063474ad8e39327013d
Parents: b1fe5ea
Author: janzhongi <ji...@ebay.com>
Authored: Wed Feb 17 16:23:08 2016 +0800
Committer: janzhongi <ji...@ebay.com>
Committed: Wed Feb 17 16:23:31 2016 +0800

----------------------------------------------------------------------
 webapp/app/js/filters/filter.js            | 19 +++++++++++++++----
 webapp/app/partials/cubes/cube_detail.html |  4 ++--
 2 files changed, 17 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/5666d7f2/webapp/app/js/filters/filter.js
----------------------------------------------------------------------
diff --git a/webapp/app/js/filters/filter.js b/webapp/app/js/filters/filter.js
index 2cf8114..5fd664e 100755
--- a/webapp/app/js/filters/filter.js
+++ b/webapp/app/js/filters/filter.js
@@ -83,20 +83,31 @@ KylinApp
   .filter('bytes', function () {
     return function (bytes, precision) {
       if (bytes === 0) {
-        return '0 bytes'
+        return 'N/A';
       }
-      ;
       if (isNaN(parseFloat(bytes)) || !isFinite(bytes)) {
         return '-';
       }
 
       if (typeof precision === 'undefined') {
-        precision = 1;
+        precision = 3;
       }
 
       var units = ['bytes', 'kB', 'MB', 'GB', 'TB', 'PB'],
         number = Math.floor(Math.log(bytes) / Math.log(1024));
-      return (bytes / Math.pow(1024, Math.floor(number))).toFixed(precision) + ' ' + units[number];
+      switch(number){
+        case 0:
+          precision = 0;
+          break;
+        case 1:
+        case 2:
+          precision = 3;
+          break;
+        default:
+          precision = 5;
+      }
+
+      return Math.round((bytes / Math.pow(1024, Math.floor(number)))*Math.pow(10,precision))/Math.pow(10,precision) + ' ' + units[number];
     }
   }).filter('resizePieHeight', function () {
     return function (item) {

http://git-wip-us.apache.org/repos/asf/kylin/blob/5666d7f2/webapp/app/partials/cubes/cube_detail.html
----------------------------------------------------------------------
diff --git a/webapp/app/partials/cubes/cube_detail.html b/webapp/app/partials/cubes/cube_detail.html
index 41f9275..81dbf46 100755
--- a/webapp/app/partials/cubes/cube_detail.html
+++ b/webapp/app/partials/cubes/cube_detail.html
@@ -106,14 +106,14 @@
                 <h5><b>HTable:</b> {{table.tableName}}</h5>
                 <ul>
                     <li>Region Count: <span class="red">{{table.regionCount}}</span></li>
-                    <li>Size: <span class="red">{{table.tableSize | bytes:2}}</span></li>
+                    <li>Size: <span class="red">{{table.tableSize | bytes}}</span></li>
                     <li>Start Time: <span class="red">{{table.dateRangeStart | reverseToGMT0}}</span></li>
                     <li>End Time: <span class="red">{{table.dateRangeEnd | reverseToGMT0}}</span></li>
                 </ul>
             </div>
             <div ng-if="cube.hbase">
                 <div class="hr hr8 hr-double hr-dotted"></div>
-                <h5><b>Total Size:</b> <span class="red">{{cube.totalSize | bytes:2}}</span></h5>
+                <h5><b>Total Size:</b> <span class="red">{{cube.totalSize | bytes}}</span></h5>
                 <h5><b>Total Number:</b> <span class="red">{{cube.hbase.length}}</span></h5>
             </div>
             <div ng-if="cube.hbase.length == 0">


[07/52] [abbrv] kylin git commit: KYLIN-1353 fix unit test

Posted by li...@apache.org.
KYLIN-1353 fix unit test


Project: http://git-wip-us.apache.org/repos/asf/kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/73cfbef8
Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/73cfbef8
Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/73cfbef8

Branch: refs/heads/1.x-HBase1.1.3
Commit: 73cfbef88f6a19812e64a28aacaee7e3a16d4f01
Parents: dfdeb92
Author: shaofengshi <sh...@apache.org>
Authored: Sun Jan 24 16:17:41 2016 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Sun Jan 24 16:18:11 2016 +0800

----------------------------------------------------------------------
 .../java/org/apache/kylin/rest/service/CubeService.java   |  8 +++-----
 .../org/apache/kylin/rest/service/CacheServiceTest.java   | 10 ++++++++--
 2 files changed, 11 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/73cfbef8/server/src/main/java/org/apache/kylin/rest/service/CubeService.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/kylin/rest/service/CubeService.java b/server/src/main/java/org/apache/kylin/rest/service/CubeService.java
index 4440afd..52bbdc4 100644
--- a/server/src/main/java/org/apache/kylin/rest/service/CubeService.java
+++ b/server/src/main/java/org/apache/kylin/rest/service/CubeService.java
@@ -221,12 +221,10 @@ public class CubeService extends BasicService {
             throw new JobException("Cube schema shouldn't be changed with running job.");
         }
 
-        desc.init(getConfig(), getMetadataManager().getAllTablesMap());
-        int cuboidCount = CuboidCLI.simulateCuboidGeneration(desc);
-        logger.info("Updated cube " + cube.getName() + " has " + cuboidCount + " cuboids");
-
         try {
-            
+            desc.init(getConfig(), getMetadataManager().getAllTablesMap());
+            int cuboidCount = CuboidCLI.simulateCuboidGeneration(desc);
+            logger.info("Updated cube " + cube.getName() + " has " + cuboidCount + " cuboids");
             CubeDesc updatedCubeDesc = getCubeDescManager().updateCubeDesc(desc);
             if (!updatedCubeDesc.getError().isEmpty()) {
                 return updatedCubeDesc;

http://git-wip-us.apache.org/repos/asf/kylin/blob/73cfbef8/server/src/test/java/org/apache/kylin/rest/service/CacheServiceTest.java
----------------------------------------------------------------------
diff --git a/server/src/test/java/org/apache/kylin/rest/service/CacheServiceTest.java b/server/src/test/java/org/apache/kylin/rest/service/CacheServiceTest.java
index a14be1d..1a2b211 100644
--- a/server/src/test/java/org/apache/kylin/rest/service/CacheServiceTest.java
+++ b/server/src/test/java/org/apache/kylin/rest/service/CacheServiceTest.java
@@ -426,8 +426,14 @@ public class CacheServiceTest extends LocalFileMetadataTestCase {
         cubeRequest.setCubeName(newCubeName);
         cubeRequest.setModelDescData(JsonUtil.writeValueAsString(modelDescCopy));
         cubeRequest.setCubeDescData(JsonUtil.writeValueAsString(invalidCubeDesc));
-        cubeRequest = cubeController.updateCubeDesc(cubeRequest);
-        assertTrue(cubeRequest.getSuccessful() == false);
+        boolean throwException = false; 
+        try {
+            cubeController.updateCubeDesc(cubeRequest);
+        } catch (Exception e) {
+            throwException = true;
+        }
+        
+        assertTrue(throwException);
         // 2 events, all for model
         assertEquals(2, broadcaster.getCounterAndClear());
         waitForCounterAndClear(2);


[52/52] [abbrv] kylin git commit: KYLIN-920 & KYLIN-782 Upgrade to HBase 1.1 (with help from murkrishn )

Posted by li...@apache.org.
KYLIN-920 & KYLIN-782 Upgrade to HBase 1.1 (with help from murkrishn <mu...@ebay.com>)

Signed-off-by: Li, Yang <ya...@ebay.com>


Project: http://git-wip-us.apache.org/repos/asf/kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/48cec941
Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/48cec941
Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/48cec941

Branch: refs/heads/1.x-HBase1.1.3
Commit: 48cec941bf05f796c24bd6e8d6bd0e997c91d6a5
Parents: 0e5be05
Author: Yang Li <li...@apache.org>
Authored: Sun Aug 16 20:22:13 2015 +0800
Committer: Yang Li <li...@apache.org>
Committed: Sun Feb 21 19:50:18 2016 +0800

----------------------------------------------------------------------
 .../common/persistence/HBaseConnection.java     |  31 +-
 .../common/persistence/HBaseResourceStore.java  |  31 +-
 .../common/util/HBaseRegionSizeCalculator.java  |  41 +-
 .../kylin/common/util/BasicHadoopTest.java      |  11 +-
 .../kylin/job/cube/GarbageCollectionStep.java   |  22 +-
 .../kylin/job/hadoop/cube/CubeHFileJob.java     |  18 +-
 .../job/hadoop/cube/StorageCleanupJob.java      |  26 +-
 .../kylin/job/hadoop/hbase/CreateHTableJob.java |   8 +-
 .../hadoop/invertedindex/IICreateHFileJob.java  |  22 +-
 .../hadoop/invertedindex/IICreateHTableJob.java |  11 +-
 .../apache/kylin/job/tools/CleanHtableCLI.java  |   8 +-
 .../kylin/job/tools/CubeMigrationCLI.java       |  67 +-
 .../kylin/job/tools/DeployCoprocessorCLI.java   | 769 ++++++++++---------
 .../job/tools/GridTableHBaseBenchmark.java      |  37 +-
 .../kylin/job/tools/HtableAlterMetadataCLI.java |   8 +-
 .../apache/kylin/job/tools/RowCounterCLI.java   |  11 +-
 .../org/apache/kylin/job/ExportHBaseData.java   |  18 +-
 .../kylin/job/hadoop/hbase/TestHbaseClient.java |  13 +-
 .../kylin/job/tools/HBaseRowDigestTest.java     |  11 +-
 monitor/pom.xml                                 |   6 +
 .../kylin/monitor/MonitorMetaManager.java       |  49 +-
 pom.xml                                         |  14 +-
 .../apache/kylin/rest/service/AclService.java   |  38 +-
 .../apache/kylin/rest/service/CubeService.java  |  35 +-
 .../apache/kylin/rest/service/QueryService.java |  21 +-
 .../apache/kylin/rest/service/UserService.java  |  27 +-
 .../storage/filter/BitMapFilterEvaluator.java   |   1 -
 .../storage/hbase/CubeSegmentTupleIterator.java |  19 +-
 .../kylin/storage/hbase/CubeStorageEngine.java  |   4 +-
 .../storage/hbase/HBaseClientKVIterator.java    | 187 ++---
 .../hbase/InvertedIndexStorageEngine.java       | 114 +--
 .../kylin/storage/hbase/PingHBaseCLI.java       | 179 ++---
 .../storage/hbase/RegionScannerAdapter.java     |  10 +-
 .../hbase/SerializedHBaseTupleIterator.java     |   4 +-
 .../endpoint/EndpointTupleIterator.java         |  15 +-
 .../hbase/coprocessor/endpoint/IIEndpoint.java  |   2 +-
 .../observer/AggregateRegionObserver.java       |   2 +-
 .../observer/AggregationScanner.java            |  14 +-
 .../observer/ObserverAggregationCache.java      |  10 +-
 .../coprocessor/observer/ObserverEnabler.java   |   4 +-
 .../storage/hbase/InvertedIndexHBaseTest.java   | 227 +++---
 .../observer/AggregateRegionObserverTest.java   |  72 +-
 .../minicluster/HiveMiniClusterTest.java        |   3 +-
 43 files changed, 1110 insertions(+), 1110 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/48cec941/common/src/main/java/org/apache/kylin/common/persistence/HBaseConnection.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/kylin/common/persistence/HBaseConnection.java b/common/src/main/java/org/apache/kylin/common/persistence/HBaseConnection.java
index 5b8fe54..a3d8166 100644
--- a/common/src/main/java/org/apache/kylin/common/persistence/HBaseConnection.java
+++ b/common/src/main/java/org/apache/kylin/common/persistence/HBaseConnection.java
@@ -27,9 +27,10 @@ import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotFoundException;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
-import org.apache.hadoop.hbase.client.HConnection;
-import org.apache.hadoop.hbase.client.HConnectionManager;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.util.HadoopUtil;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -42,13 +43,13 @@ public class HBaseConnection {
 
     private static final Logger logger = LoggerFactory.getLogger(HBaseConnection.class);
 
-    private static final Map<String, HConnection> ConnPool = new ConcurrentHashMap<String, HConnection>();
+    private static final Map<String, Connection> ConnPool = new ConcurrentHashMap<String, Connection>();
 
     static {
         Runtime.getRuntime().addShutdownHook(new Thread() {
             @Override
             public void run() {
-                for (HConnection conn : ConnPool.values()) {
+                for (Connection conn : ConnPool.values()) {
                     try {
                         conn.close();
                     } catch (IOException e) {
@@ -62,16 +63,20 @@ public class HBaseConnection {
     public static void clearCache() {
         ConnPool.clear();
     }
+    
+    public static Connection get() {
+        return get(KylinConfig.getInstanceFromEnv().getStorageUrl());
+    }
 
-    public static HConnection get(String url) {
+    public static Connection get(String url) {
 
-        HConnection connection = ConnPool.get(url);
+        Connection connection = ConnPool.get(url);
         try {
             // I don't use DCL since recreate a connection is not a big issue.
             if (connection == null) {
                 // find configuration
                 Configuration conf = HadoopUtil.getCurrentHBaseConfiguration();
-                connection = HConnectionManager.createConnection(conf);
+                connection = ConnectionFactory.createConnection(conf);
                 ConnPool.put(url, connection);
             }
         } catch (Throwable t) {
@@ -85,13 +90,13 @@ public class HBaseConnection {
         createHTableIfNeeded(HBaseConnection.get(hbaseUrl), tableName, families);
     }
 
-    public static void createHTableIfNeeded(HConnection conn, String tableName, String... families) throws IOException {
-        HBaseAdmin hbase = new HBaseAdmin(conn);
+    public static void createHTableIfNeeded(Connection conn, String tableName, String... families) throws IOException {
+        Admin admin = conn.getAdmin();
 
         try {
             boolean tableExist = false;
             try {
-                hbase.getTableDescriptor(TableName.valueOf(tableName));
+                admin.getTableDescriptor(TableName.valueOf(tableName));
                 tableExist = true;
             } catch (TableNotFoundException e) {
             }
@@ -112,11 +117,11 @@ public class HBaseConnection {
                     desc.addFamily(fd);
                 }
             }
-            hbase.createTable(desc);
+            admin.createTable(desc);
 
             logger.debug("HTable '" + tableName + "' created");
         } finally {
-            hbase.close();
+            admin.close();
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/48cec941/common/src/main/java/org/apache/kylin/common/persistence/HBaseResourceStore.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/kylin/common/persistence/HBaseResourceStore.java b/common/src/main/java/org/apache/kylin/common/persistence/HBaseResourceStore.java
index d1ff27a..0c06847 100644
--- a/common/src/main/java/org/apache/kylin/common/persistence/HBaseResourceStore.java
+++ b/common/src/main/java/org/apache/kylin/common/persistence/HBaseResourceStore.java
@@ -34,13 +34,14 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.Delete;
-import org.apache.hadoop.hbase.client.HConnection;
-import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.filter.KeyOnlyFilter;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.util.Bytes;
@@ -77,7 +78,7 @@ public class HBaseResourceStore extends ResourceStore {
 
     //    final Map<String, String> tableNameMap; // path prefix ==> HBase table name
 
-    private HConnection getConnection() throws IOException {
+    private Connection getConnection() throws IOException {
         return HBaseConnection.get(hbaseUrl);
     }
 
@@ -114,7 +115,7 @@ public class HBaseResourceStore extends ResourceStore {
 
         ArrayList<String> result = new ArrayList<String>();
 
-        HTableInterface table = getConnection().getTable(getAllInOneTableName());
+        Table table = getConnection().getTable(TableName.valueOf(getAllInOneTableName()));
         Scan scan = new Scan(startRow, endRow);
         scan.setFilter(new KeyOnlyFilter());
         try {
@@ -150,7 +151,7 @@ public class HBaseResourceStore extends ResourceStore {
         scan.addColumn(B_FAMILY, B_COLUMN);
         tuneScanParameters(scan);
 
-        HTableInterface table = getConnection().getTable(getAllInOneTableName());
+        Table table = getConnection().getTable(TableName.valueOf(getAllInOneTableName()));
         List<RawResource> result = Lists.newArrayList();
         try {
             ResultScanner scanner = table.getScanner(scan);
@@ -219,13 +220,12 @@ public class HBaseResourceStore extends ResourceStore {
         IOUtils.copy(content, bout);
         bout.close();
 
-        HTableInterface table = getConnection().getTable(getAllInOneTableName());
+        Table table = getConnection().getTable(TableName.valueOf(getAllInOneTableName()));
         try {
             byte[] row = Bytes.toBytes(resPath);
             Put put = buildPut(resPath, ts, row, bout.toByteArray(), table);
 
             table.put(put);
-            table.flushCommits();
         } finally {
             IOUtils.closeQuietly(table);
         }
@@ -233,7 +233,7 @@ public class HBaseResourceStore extends ResourceStore {
 
     @Override
     protected long checkAndPutResourceImpl(String resPath, byte[] content, long oldTS, long newTS) throws IOException, IllegalStateException {
-        HTableInterface table = getConnection().getTable(getAllInOneTableName());
+        Table table = getConnection().getTable(TableName.valueOf(getAllInOneTableName()));
         try {
             byte[] row = Bytes.toBytes(resPath);
             byte[] bOldTS = oldTS == 0 ? null : Bytes.toBytes(oldTS);
@@ -245,8 +245,6 @@ public class HBaseResourceStore extends ResourceStore {
                 throw new IllegalStateException("Overwriting conflict " + resPath + ", expect old TS " + real + ", but it is " + oldTS);
             }
 
-            table.flushCommits();
-
             return newTS;
         } finally {
             IOUtils.closeQuietly(table);
@@ -255,11 +253,10 @@ public class HBaseResourceStore extends ResourceStore {
 
     @Override
     protected void deleteResourceImpl(String resPath) throws IOException {
-        HTableInterface table = getConnection().getTable(getAllInOneTableName());
+        Table table = getConnection().getTable(TableName.valueOf(getAllInOneTableName()));
         try {
             Delete del = new Delete(Bytes.toBytes(resPath));
             table.delete(del);
-            table.flushCommits();
         } finally {
             IOUtils.closeQuietly(table);
         }
@@ -284,7 +281,7 @@ public class HBaseResourceStore extends ResourceStore {
                 scan.addColumn(B_FAMILY, B_COLUMN_TS);
         }
 
-        HTableInterface table = getConnection().getTable(getAllInOneTableName());
+        Table table = getConnection().getTable(TableName.valueOf(getAllInOneTableName()));
         try {
             ResultScanner scanner = table.getScanner(scan);
             Result result = null;
@@ -303,7 +300,7 @@ public class HBaseResourceStore extends ResourceStore {
         return endRow;
     }
 
-    private Path writeLargeCellToHdfs(String resPath, byte[] largeColumn, HTableInterface table) throws IOException {
+    private Path writeLargeCellToHdfs(String resPath, byte[] largeColumn, Table table) throws IOException {
         Path redirectPath = bigCellHDFSPath(resPath);
         Configuration hconf = HadoopUtil.getCurrentHBaseConfiguration();
         FileSystem fileSystem = FileSystem.get(hconf);
@@ -329,7 +326,7 @@ public class HBaseResourceStore extends ResourceStore {
         return redirectPath;
     }
 
-    private Put buildPut(String resPath, long ts, byte[] row, byte[] content, HTableInterface table) throws IOException {
+    private Put buildPut(String resPath, long ts, byte[] row, byte[] content, Table table) throws IOException {
         int kvSizeLimit = this.kylinConfig.getHBaseKeyValueSize();
         if (content.length > kvSizeLimit) {
             writeLargeCellToHdfs(resPath, content, table);
@@ -337,8 +334,8 @@ public class HBaseResourceStore extends ResourceStore {
         }
 
         Put put = new Put(row);
-        put.add(B_FAMILY, B_COLUMN, content);
-        put.add(B_FAMILY, B_COLUMN_TS, Bytes.toBytes(ts));
+        put.addColumn(B_FAMILY, B_COLUMN, content);
+        put.addColumn(B_FAMILY, B_COLUMN_TS, Bytes.toBytes(ts));
 
         return put;
     }

http://git-wip-us.apache.org/repos/asf/kylin/blob/48cec941/common/src/main/java/org/apache/kylin/common/util/HBaseRegionSizeCalculator.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/kylin/common/util/HBaseRegionSizeCalculator.java b/common/src/main/java/org/apache/kylin/common/util/HBaseRegionSizeCalculator.java
index 093ac9e..ccbb6f0 100644
--- a/common/src/main/java/org/apache/kylin/common/util/HBaseRegionSizeCalculator.java
+++ b/common/src/main/java/org/apache/kylin/common/util/HBaseRegionSizeCalculator.java
@@ -23,19 +23,24 @@ import java.io.IOException;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.TreeMap;
 import java.util.TreeSet;
 
+import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.ClusterStatus;
-import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.RegionLoad;
 import org.apache.hadoop.hbase.ServerLoad;
 import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
-import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.RegionLocator;
+import org.apache.hadoop.hbase.client.Table;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -53,30 +58,31 @@ public class HBaseRegionSizeCalculator {
     /**
      * Computes size of each region for table and given column families.
      * */
-    public HBaseRegionSizeCalculator(HTable table) throws IOException {
-        this(table, new HBaseAdmin(table.getConfiguration()));
-    }
-
-    /** Constructor for unit testing */
-    HBaseRegionSizeCalculator(HTable table, HBaseAdmin hBaseAdmin) throws IOException {
-
+    public HBaseRegionSizeCalculator(String tableName , Connection hbaseConnection) throws IOException {
+        Table table = null;
+        Admin admin = null;
+        
         try {
+            table = hbaseConnection.getTable(TableName.valueOf(tableName));
+            admin = hbaseConnection.getAdmin();
+            
             if (!enabled(table.getConfiguration())) {
                 logger.info("Region size calculation disabled.");
                 return;
             }
 
-            logger.info("Calculating region sizes for table \"" + new String(table.getTableName()) + "\".");
+            logger.info("Calculating region sizes for table \"" + table.getName() + "\".");
 
             // Get regions for table.
-            Set<HRegionInfo> tableRegionInfos = table.getRegionLocations().keySet();
+            RegionLocator regionLocator = hbaseConnection.getRegionLocator(table.getName());
+            List<HRegionLocation> regionLocationList = regionLocator.getAllRegionLocations();
             Set<byte[]> tableRegions = new TreeSet<byte[]>(Bytes.BYTES_COMPARATOR);
 
-            for (HRegionInfo regionInfo : tableRegionInfos) {
-                tableRegions.add(regionInfo.getRegionName());
+            for (HRegionLocation hRegionLocation : regionLocationList) {
+                tableRegions.add(hRegionLocation.getRegionInfo().getRegionName());
             }
 
-            ClusterStatus clusterStatus = hBaseAdmin.getClusterStatus();
+            ClusterStatus clusterStatus = admin.getClusterStatus();
             Collection<ServerName> servers = clusterStatus.getServers();
             final long megaByte = 1024L * 1024L;
 
@@ -99,7 +105,8 @@ public class HBaseRegionSizeCalculator {
                 }
             }
         } finally {
-            hBaseAdmin.close();
+            IOUtils.closeQuietly(table);
+            IOUtils.closeQuietly(admin);
         }
 
     }
@@ -124,4 +131,4 @@ public class HBaseRegionSizeCalculator {
     public Map<byte[], Long> getRegionSizeMap() {
         return Collections.unmodifiableMap(sizeMap);
     }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kylin/blob/48cec941/common/src/test/java/org/apache/kylin/common/util/BasicHadoopTest.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/org/apache/kylin/common/util/BasicHadoopTest.java b/common/src/test/java/org/apache/kylin/common/util/BasicHadoopTest.java
index 6d2762c..481fc6c 100644
--- a/common/src/test/java/org/apache/kylin/common/util/BasicHadoopTest.java
+++ b/common/src/test/java/org/apache/kylin/common/util/BasicHadoopTest.java
@@ -21,12 +21,11 @@ package org.apache.kylin.common.util;
 import java.io.File;
 import java.io.IOException;
 
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.kylin.common.persistence.HBaseConnection;
 import org.junit.BeforeClass;
 import org.junit.Ignore;
 import org.junit.Test;
@@ -56,16 +55,14 @@ public class BasicHadoopTest {
         cf.setBlocksize(4 * 1024 * 1024); // set to 4MB
         tableDesc.addFamily(cf);
 
-        Configuration conf = HBaseConfiguration.create();
-        HBaseAdmin admin = new HBaseAdmin(conf);
+        Admin admin = HBaseConnection.get().getAdmin();
         admin.createTable(tableDesc);
         admin.close();
     }
 
     @Test
     public void testRetriveHtableHost() throws IOException {
-        Configuration conf = HBaseConfiguration.create();
-        HBaseAdmin hbaseAdmin = new HBaseAdmin(conf);
+        Admin hbaseAdmin = HBaseConnection.get().getAdmin();
         HTableDescriptor[] tableDescriptors = hbaseAdmin.listTables();
         for (HTableDescriptor table : tableDescriptors) {
             String value = table.getValue("KYLIN_HOST");

http://git-wip-us.apache.org/repos/asf/kylin/blob/48cec941/job/src/main/java/org/apache/kylin/job/cube/GarbageCollectionStep.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/cube/GarbageCollectionStep.java b/job/src/main/java/org/apache/kylin/job/cube/GarbageCollectionStep.java
index f2f1fc0..8c61a3a 100644
--- a/job/src/main/java/org/apache/kylin/job/cube/GarbageCollectionStep.java
+++ b/job/src/main/java/org/apache/kylin/job/cube/GarbageCollectionStep.java
@@ -24,14 +24,13 @@ import java.util.Collections;
 import java.util.List;
 
 import org.apache.commons.lang.StringUtils;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
 import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.util.Bytes;
+import org.apache.kylin.common.persistence.HBaseConnection;
 import org.apache.kylin.common.util.HadoopUtil;
 import org.apache.kylin.job.cmd.ShellCmdOutput;
 import org.apache.kylin.job.exception.ExecuteException;
@@ -99,19 +98,18 @@ public class GarbageCollectionStep extends AbstractExecutable {
         List<String> oldTables = getOldHTables();
         if (oldTables != null && oldTables.size() > 0) {
             String metadataUrlPrefix = KylinConfig.getInstanceFromEnv().getMetadataUrlPrefix();
-            Configuration conf = HBaseConfiguration.create();
-            HBaseAdmin admin = null;
+            Admin admin = null;
             try {
-                admin = new HBaseAdmin(conf);
+                admin = HBaseConnection.get().getAdmin();
                 for (String table : oldTables) {
-                    if (admin.tableExists(table)) {
-                        HTableDescriptor tableDescriptor = admin.getTableDescriptor(Bytes.toBytes(table));
+                    if (admin.tableExists(TableName.valueOf(table))) {
+                        HTableDescriptor tableDescriptor = admin.getTableDescriptor(TableName.valueOf(table));
                         String host = tableDescriptor.getValue(IRealizationConstants.HTableTag);
                         if (metadataUrlPrefix.equalsIgnoreCase(host)) {
-                            if (admin.isTableEnabled(table)) {
-                                admin.disableTable(table);
+                            if (admin.isTableEnabled(TableName.valueOf(table))) {
+                                admin.disableTable(TableName.valueOf(table));
                             }
-                            admin.deleteTable(table);
+                            admin.deleteTable(TableName.valueOf(table));
                             logger.debug("Dropped HBase table " + table);
                             output.append("Dropped HBase table " + table + " \n");
                         } else {

http://git-wip-us.apache.org/repos/asf/kylin/blob/48cec941/job/src/main/java/org/apache/kylin/job/hadoop/cube/CubeHFileJob.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/cube/CubeHFileJob.java b/job/src/main/java/org/apache/kylin/job/hadoop/cube/CubeHFileJob.java
index 3c1e4a5..6f36eff 100644
--- a/job/src/main/java/org/apache/kylin/job/hadoop/cube/CubeHFileJob.java
+++ b/job/src/main/java/org/apache/kylin/job/hadoop/cube/CubeHFileJob.java
@@ -19,11 +19,15 @@
 package org.apache.kylin.job.hadoop.cube;
 
 import org.apache.commons.cli.Options;
+import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.client.HTable;
-import org.apache.hadoop.hbase.mapreduce.HFileOutputFormat;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.RegionLocator;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.mapreduce.HFileOutputFormat2;
 import org.apache.hadoop.hbase.mapreduce.KeyValueSortReducer;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.mapreduce.Job;
@@ -31,6 +35,7 @@ import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
 import org.apache.hadoop.util.ToolRunner;
 import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.persistence.HBaseConnection;
 import org.apache.kylin.cube.CubeInstance;
 import org.apache.kylin.cube.CubeManager;
 import org.apache.kylin.job.constant.BatchConstants;
@@ -47,6 +52,8 @@ public class CubeHFileJob extends AbstractHadoopJob {
 
     public int run(String[] args) throws Exception {
         Options options = new Options();
+        Connection connection = null;
+        Table table = null;
 
         try {
             options.addOption(OPTION_JOB_NAME);
@@ -80,10 +87,12 @@ public class CubeHFileJob extends AbstractHadoopJob {
             attachKylinPropsAndMetadata(cube, job.getConfiguration());
 
             String tableName = getOptionValue(OPTION_HTABLE_NAME).toUpperCase();
-            HTable htable = new HTable(conf, tableName);
+            connection = HBaseConnection.get();
+            table = connection.getTable(TableName.valueOf(tableName));
+            RegionLocator regionLocator = connection.getRegionLocator(TableName.valueOf(tableName));
 
             //Automatic config !
-            HFileOutputFormat.configureIncrementalLoad(job, htable);
+            HFileOutputFormat2.configureIncrementalLoad(job, table, regionLocator);
 
             // set block replication to 3 for hfiles
             conf.set(DFSConfigKeys.DFS_REPLICATION_KEY, "3");
@@ -96,6 +105,7 @@ public class CubeHFileJob extends AbstractHadoopJob {
             printUsage(options);
             throw e;
         } finally {
+            IOUtils.closeQuietly(table);
             if (job != null)
                 cleanupTempConfFile(job.getConfiguration());
         }

http://git-wip-us.apache.org/repos/asf/kylin/blob/48cec941/job/src/main/java/org/apache/kylin/job/hadoop/cube/StorageCleanupJob.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/cube/StorageCleanupJob.java b/job/src/main/java/org/apache/kylin/job/hadoop/cube/StorageCleanupJob.java
index 3b25ee1..184b6cd 100644
--- a/job/src/main/java/org/apache/kylin/job/hadoop/cube/StorageCleanupJob.java
+++ b/job/src/main/java/org/apache/kylin/job/hadoop/cube/StorageCleanupJob.java
@@ -18,6 +18,13 @@
 
 package org.apache.kylin.job.hadoop.cube;
 
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.StringReader;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
 import org.apache.commons.cli.Option;
 import org.apache.commons.cli.OptionBuilder;
 import org.apache.commons.cli.Options;
@@ -28,10 +35,12 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.MasterNotRunningException;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.util.ToolRunner;
 import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.persistence.HBaseConnection;
 import org.apache.kylin.cube.CubeInstance;
 import org.apache.kylin.cube.CubeManager;
 import org.apache.kylin.cube.CubeSegment;
@@ -50,13 +59,6 @@ import org.apache.kylin.metadata.realization.IRealizationConstants;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.BufferedReader;
-import java.io.IOException;
-import java.io.StringReader;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-
 /**
  * @author ysong1
  */
@@ -107,7 +109,7 @@ public class StorageCleanupJob extends AbstractHadoopJob {
         IIManager iiManager = IIManager.getInstance(KylinConfig.getInstanceFromEnv());
 
         // get all kylin hbase tables
-        HBaseAdmin hbaseAdmin = new HBaseAdmin(conf);
+        Admin hbaseAdmin = HBaseConnection.get().getAdmin();
         String tableNamePrefix = IRealizationConstants.SharedHbaseStorageLocationPrefix;
         HTableDescriptor[] tableDescriptors = hbaseAdmin.listTables(tableNamePrefix + ".*");
         List<String> allTablesNeedToBeDropped = new ArrayList<String>();
@@ -141,9 +143,9 @@ public class StorageCleanupJob extends AbstractHadoopJob {
             // drop tables
             for (String htableName : allTablesNeedToBeDropped) {
                 log.info("Deleting HBase table " + htableName);
-                if (hbaseAdmin.tableExists(htableName)) {
-                    hbaseAdmin.disableTable(htableName);
-                    hbaseAdmin.deleteTable(htableName);
+                if (hbaseAdmin.tableExists(TableName.valueOf(htableName))) {
+                    hbaseAdmin.disableTable(TableName.valueOf(htableName));
+                    hbaseAdmin.deleteTable(TableName.valueOf(htableName));
                     log.info("Deleted HBase table " + htableName);
                 } else {
                     log.info("HBase table" + htableName + " does not exist");

http://git-wip-us.apache.org/repos/asf/kylin/blob/48cec941/job/src/main/java/org/apache/kylin/job/hadoop/hbase/CreateHTableJob.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/hbase/CreateHTableJob.java b/job/src/main/java/org/apache/kylin/job/hadoop/hbase/CreateHTableJob.java
index 027c0ca..9f5e062 100644
--- a/job/src/main/java/org/apache/kylin/job/hadoop/hbase/CreateHTableJob.java
+++ b/job/src/main/java/org/apache/kylin/job/hadoop/hbase/CreateHTableJob.java
@@ -25,11 +25,10 @@ import org.apache.commons.cli.Options;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.io.compress.Compression.Algorithm;
 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
 import org.apache.hadoop.hbase.regionserver.ConstantSizeRegionSplitPolicy;
@@ -42,6 +41,7 @@ import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.ToolRunner;
 import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.persistence.HBaseConnection;
 import org.apache.kylin.common.util.HadoopUtil;
 import org.apache.kylin.cube.CubeInstance;
 import org.apache.kylin.cube.CubeManager;
@@ -81,7 +81,7 @@ public class CreateHTableJob extends AbstractHadoopJob {
         tableDesc.setValue(IRealizationConstants.HTableTag, config.getMetadataUrlPrefix());
 
         Configuration conf = HadoopUtil.getCurrentHBaseConfiguration();
-        HBaseAdmin admin = new HBaseAdmin(conf);
+        Admin admin = HBaseConnection.get().getAdmin();
 
         try {
             if (User.isHBaseSecurityEnabled(conf)) {
@@ -139,7 +139,7 @@ public class CreateHTableJob extends AbstractHadoopJob {
 
             byte[][] splitKeys = getSplits(conf, partitionFilePath);
 
-            if (admin.tableExists(tableName)) {
+            if (admin.tableExists(TableName.valueOf(tableName))) {
                 // admin.disableTable(tableName);
                 // admin.deleteTable(tableName);
                 throw new RuntimeException("HBase table " + tableName + " exists!");

http://git-wip-us.apache.org/repos/asf/kylin/blob/48cec941/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IICreateHFileJob.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IICreateHFileJob.java b/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IICreateHFileJob.java
index c032bbc..fa42148 100644
--- a/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IICreateHFileJob.java
+++ b/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IICreateHFileJob.java
@@ -19,17 +19,20 @@
 package org.apache.kylin.job.hadoop.invertedindex;
 
 import org.apache.commons.cli.Options;
+import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.RegionLocator;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.hadoop.hbase.mapreduce.HFileOutputFormat;
+import org.apache.hadoop.hbase.mapreduce.HFileOutputFormat2;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
 import org.apache.hadoop.util.ToolRunner;
-import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.persistence.HBaseConnection;
 import org.apache.kylin.common.util.HadoopUtil;
 import org.apache.kylin.job.hadoop.AbstractHadoopJob;
 import org.slf4j.Logger;
@@ -45,6 +48,8 @@ public class IICreateHFileJob extends AbstractHadoopJob {
 
     public int run(String[] args) throws Exception {
         Options options = new Options();
+        Connection connection = null;
+        Table table = null;
 
         try {
             options.addOption(OPTION_JOB_NAME);
@@ -69,8 +74,11 @@ public class IICreateHFileJob extends AbstractHadoopJob {
             job.setMapOutputValueClass(KeyValue.class);
 
             String tableName = getOptionValue(OPTION_HTABLE_NAME);
-            HTable htable = new HTable(HBaseConfiguration.create(getConf()), tableName);
-            HFileOutputFormat.configureIncrementalLoad(job, htable);
+
+            connection = HBaseConnection.get();
+            table = connection.getTable(TableName.valueOf(tableName));
+            RegionLocator regionLocator = connection.getRegionLocator(TableName.valueOf(tableName));
+            HFileOutputFormat2.configureIncrementalLoad(job, table, regionLocator);
 
             this.deletePath(job.getConfiguration(), output);
 
@@ -78,6 +86,8 @@ public class IICreateHFileJob extends AbstractHadoopJob {
         } catch (Exception e) {
             printUsage(options);
             throw e;
+        } finally {
+            IOUtils.closeQuietly(table);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/48cec941/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IICreateHTableJob.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IICreateHTableJob.java b/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IICreateHTableJob.java
index 32d065a..63777ef 100644
--- a/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IICreateHTableJob.java
+++ b/job/src/main/java/org/apache/kylin/job/hadoop/invertedindex/IICreateHTableJob.java
@@ -24,11 +24,12 @@ import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.util.ToolRunner;
 import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.persistence.HBaseConnection;
 import org.apache.kylin.common.util.BytesUtil;
 import org.apache.kylin.common.util.HadoopUtil;
 import org.apache.kylin.invertedindex.IIInstance;
@@ -78,10 +79,10 @@ public class IICreateHTableJob extends AbstractHadoopJob {
             DeployCoprocessorCLI.deployCoprocessor(tableDesc);
 
             // drop the table first
-            HBaseAdmin admin = new HBaseAdmin(conf);
-            if (admin.tableExists(tableName)) {
-                admin.disableTable(tableName);
-                admin.deleteTable(tableName);
+            Admin admin = HBaseConnection.get().getAdmin();
+            if (admin.tableExists(TableName.valueOf(tableName))) {
+                admin.disableTable(TableName.valueOf(tableName));
+                admin.deleteTable(TableName.valueOf(tableName));
             }
 
             // create table

http://git-wip-us.apache.org/repos/asf/kylin/blob/48cec941/job/src/main/java/org/apache/kylin/job/tools/CleanHtableCLI.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/tools/CleanHtableCLI.java b/job/src/main/java/org/apache/kylin/job/tools/CleanHtableCLI.java
index b6e5af5..7fc1d72 100644
--- a/job/src/main/java/org/apache/kylin/job/tools/CleanHtableCLI.java
+++ b/job/src/main/java/org/apache/kylin/job/tools/CleanHtableCLI.java
@@ -21,11 +21,10 @@ package org.apache.kylin.job.tools;
 import java.io.IOException;
 
 import org.apache.commons.cli.Options;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.util.ToolRunner;
+import org.apache.kylin.common.persistence.HBaseConnection;
 import org.apache.kylin.job.hadoop.AbstractHadoopJob;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -54,8 +53,7 @@ public class CleanHtableCLI extends AbstractHadoopJob {
     }
 
     private void clean() throws IOException {
-        Configuration conf = HBaseConfiguration.create();
-        HBaseAdmin hbaseAdmin = new HBaseAdmin(conf);
+        Admin hbaseAdmin = HBaseConnection.get().getAdmin();
 
         for (HTableDescriptor descriptor : hbaseAdmin.listTables()) {
             String name = descriptor.getNameAsString().toLowerCase();

http://git-wip-us.apache.org/repos/asf/kylin/blob/48cec941/job/src/main/java/org/apache/kylin/job/tools/CubeMigrationCLI.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/tools/CubeMigrationCLI.java b/job/src/main/java/org/apache/kylin/job/tools/CubeMigrationCLI.java
index 44bc2c3..4c25d8b 100644
--- a/job/src/main/java/org/apache/kylin/job/tools/CubeMigrationCLI.java
+++ b/job/src/main/java/org/apache/kylin/job/tools/CubeMigrationCLI.java
@@ -18,15 +18,31 @@
 
 package org.apache.kylin.job.tools;
 
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
 import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.*;
-import org.apache.hadoop.hbase.client.*;
-import org.apache.hadoop.hdfs.web.JsonUtil;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.persistence.*;
+import org.apache.kylin.common.persistence.HBaseConnection;
+import org.apache.kylin.common.persistence.JsonSerializer;
+import org.apache.kylin.common.persistence.RawResource;
+import org.apache.kylin.common.persistence.ResourceStore;
+import org.apache.kylin.common.persistence.Serializer;
 import org.apache.kylin.common.util.Bytes;
 import org.apache.kylin.cube.CubeInstance;
 import org.apache.kylin.cube.CubeManager;
@@ -47,11 +63,6 @@ import org.apache.kylin.metadata.realization.RealizationType;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-
 /**
  * Created by honma on 9/3/14.
  * <p/>
@@ -71,7 +82,7 @@ public class CubeMigrationCLI {
     private static ResourceStore srcStore;
     private static ResourceStore dstStore;
     private static FileSystem hdfsFS;
-    private static HBaseAdmin hbaseAdmin;
+    private static Admin hbaseAdmin;
 
     public static final String ACL_INFO_FAMILY = "i";
     private static final String ACL_TABLE_NAME = "_acl";
@@ -117,8 +128,7 @@ public class CubeMigrationCLI {
 
         checkAndGetHbaseUrl();
 
-        Configuration conf = HBaseConfiguration.create();
-        hbaseAdmin = new HBaseAdmin(conf);
+        hbaseAdmin = HBaseConnection.get().getAdmin();
 
         hdfsFS = FileSystem.get(new Configuration());
 
@@ -141,6 +151,8 @@ public class CubeMigrationCLI {
         } else {
             showOpts();
         }
+
+        IOUtils.closeQuietly(hbaseAdmin);
     }
 
     public static void moveCube(String srcCfgUri, String dstCfgUri, String cubeName, String projectName, String copyAcl, String purgeAndDisable, String overwriteIfExists, String realExecute) throws IOException, InterruptedException {
@@ -304,10 +316,10 @@ public class CubeMigrationCLI {
         case CHANGE_HTABLE_HOST: {
             String tableName = (String) opt.params[0];
             HTableDescriptor desc = hbaseAdmin.getTableDescriptor(TableName.valueOf(tableName));
-            hbaseAdmin.disableTable(tableName);
+            hbaseAdmin.disableTable(TableName.valueOf(tableName));
             desc.setValue(IRealizationConstants.HTableTag, dstConfig.getMetadataUrlPrefix());
-            hbaseAdmin.modifyTable(tableName, desc);
-            hbaseAdmin.enableTable(tableName);
+            hbaseAdmin.modifyTable(TableName.valueOf(tableName), desc);
+            hbaseAdmin.enableTable(TableName.valueOf(tableName));
             logger.info("CHANGE_HTABLE_HOST is completed");
             break;
         }
@@ -418,14 +430,14 @@ public class CubeMigrationCLI {
             Serializer<ProjectInstance> projectSerializer = new JsonSerializer<ProjectInstance>(ProjectInstance.class);
             ProjectInstance project = dstStore.getResource(projectResPath, ProjectInstance.class, projectSerializer);
             String projUUID = project.getUuid();
-            HTableInterface srcAclHtable = null;
-            HTableInterface destAclHtable = null;
+            Table srcAclHtable = null;
+            Table destAclHtable = null;
             try {
-                srcAclHtable = HBaseConnection.get(srcConfig.getMetadataUrl()).getTable(srcConfig.getMetadataUrlPrefix() + "_acl");
-                destAclHtable = HBaseConnection.get(dstConfig.getMetadataUrl()).getTable(dstConfig.getMetadataUrlPrefix() + "_acl");
+                srcAclHtable = HBaseConnection.get(srcConfig.getMetadataUrl()).getTable(TableName.valueOf(srcConfig.getMetadataUrlPrefix() + "_acl"));
+                destAclHtable = HBaseConnection.get(dstConfig.getMetadataUrl()).getTable(TableName.valueOf(dstConfig.getMetadataUrlPrefix() + "_acl"));
 
                 // cube acl
-                Result result  = srcAclHtable.get(new Get(Bytes.toBytes(cubeId)));
+                Result result = srcAclHtable.get(new Get(Bytes.toBytes(cubeId)));
                 if (result.listCells() != null) {
                     for (Cell cell : result.listCells()) {
                         byte[] family = CellUtil.cloneFamily(cell);
@@ -438,11 +450,10 @@ public class CubeMigrationCLI {
                             value = Bytes.toBytes(valueString);
                         }
                         Put put = new Put(Bytes.toBytes(cubeId));
-                        put.add(family, column, value);
+                        put.addColumn(CellUtil.cloneFamily(cell), CellUtil.cloneQualifier(cell), CellUtil.cloneValue(cell));
                         destAclHtable.put(put);
                     }
                 }
-                destAclHtable.flushCommits();
             } finally {
                 IOUtils.closeQuietly(srcAclHtable);
                 IOUtils.closeQuietly(destAclHtable);
@@ -469,10 +480,10 @@ public class CubeMigrationCLI {
         case CHANGE_HTABLE_HOST: {
             String tableName = (String) opt.params[0];
             HTableDescriptor desc = hbaseAdmin.getTableDescriptor(TableName.valueOf(tableName));
-            hbaseAdmin.disableTable(tableName);
+            hbaseAdmin.disableTable(TableName.valueOf(tableName));
             desc.setValue(IRealizationConstants.HTableTag, srcConfig.getMetadataUrlPrefix());
-            hbaseAdmin.modifyTable(tableName, desc);
-            hbaseAdmin.enableTable(tableName);
+            hbaseAdmin.modifyTable(TableName.valueOf(tableName), desc);
+            hbaseAdmin.enableTable(TableName.valueOf(tableName));
             break;
         }
         case COPY_FILE_IN_META: {
@@ -502,13 +513,11 @@ public class CubeMigrationCLI {
         case COPY_ACL: {
             String cubeId = (String) opt.params[0];
             String modelId = (String) opt.params[1];
-            HTableInterface destAclHtable = null;
+            Table destAclHtable = null;
             try {
-                destAclHtable = HBaseConnection.get(dstConfig.getMetadataUrl()).getTable(dstConfig.getMetadataUrlPrefix() + "_acl");
-
+                destAclHtable = HBaseConnection.get(dstConfig.getMetadataUrl()).getTable(TableName.valueOf(dstConfig.getMetadataUrlPrefix() + "_acl"));
                 destAclHtable.delete(new Delete(Bytes.toBytes(cubeId)));
                 destAclHtable.delete(new Delete(Bytes.toBytes(modelId)));
-                destAclHtable.flushCommits();
             } finally {
                 IOUtils.closeQuietly(destAclHtable);
             }

http://git-wip-us.apache.org/repos/asf/kylin/blob/48cec941/job/src/main/java/org/apache/kylin/job/tools/DeployCoprocessorCLI.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/tools/DeployCoprocessorCLI.java b/job/src/main/java/org/apache/kylin/job/tools/DeployCoprocessorCLI.java
index bf655a0..28f52f2 100644
--- a/job/src/main/java/org/apache/kylin/job/tools/DeployCoprocessorCLI.java
+++ b/job/src/main/java/org/apache/kylin/job/tools/DeployCoprocessorCLI.java
@@ -1,384 +1,385 @@
-/*
- * 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.job.tools;
-
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.regex.Matcher;
-
-import org.apache.commons.io.IOUtils;
-import org.apache.commons.lang.StringUtils;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.TableNotFoundException;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
-import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.util.Bytes;
-import org.apache.kylin.common.util.HadoopUtil;
-import org.apache.kylin.cube.CubeInstance;
-import org.apache.kylin.cube.CubeManager;
-import org.apache.kylin.cube.CubeSegment;
-import org.apache.kylin.invertedindex.IIInstance;
-import org.apache.kylin.invertedindex.IIManager;
-import org.apache.kylin.invertedindex.IISegment;
-import org.apache.kylin.metadata.model.SegmentStatusEnum;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.common.collect.Lists;
-
-/**
- * @author yangli9
- */
-public class DeployCoprocessorCLI {
-
-    public static final String CubeObserverClassV2 = "org.apache.kylin.storage.hbase.cube.v1.coprocessor.observer.AggregateRegionObserver";
-    public static final String CubeEndpointClassV2 = "org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.CubeVisitService";
-    public static final String IIEndpointClassV2 = "org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.IIEndpoint";
-    public static final String OBSERVER_CLS_NAME = "org.apache.kylin.storage.hbase.coprocessor.observer.AggregateRegionObserver";
-    public static final String ENDPOINT_CLS_NAMAE = "org.apache.kylin.storage.hbase.coprocessor.endpoint.IIEndpoint";
-    private static final Logger logger = LoggerFactory.getLogger(DeployCoprocessorCLI.class);
-
-    public static void main(String[] args) throws IOException {
-        KylinConfig kylinConfig = KylinConfig.getInstanceFromEnv();
-        Configuration hconf = HadoopUtil.getCurrentHBaseConfiguration();
-        FileSystem fileSystem = FileSystem.get(hconf);
-        HBaseAdmin hbaseAdmin = new HBaseAdmin(hconf);
-
-        String localCoprocessorJar = new File(args[0]).getAbsolutePath();
-        logger.info("Identify coprocessor jar " + localCoprocessorJar);
-
-        List<String> tableNames = getHTableNames(kylinConfig);
-        logger.info("Identify tables " + tableNames);
-
-        if (args.length <= 1) {
-            printUsageAndExit();
-        }
-
-        String filterType = args[1].toLowerCase();
-        if (filterType.equals("-table")) {
-            tableNames = filterByTables(tableNames, Arrays.asList(args).subList(2, args.length));
-        } else if (filterType.equals("-cube")) {
-            tableNames = filterByCubes(tableNames, Arrays.asList(args).subList(2, args.length));
-        } else if (!filterType.equals("all")) {
-            printUsageAndExit();
-        }
-
-        logger.info("Will execute tables " + tableNames);
-
-        Set<String> oldJarPaths = getCoprocessorJarPaths(hbaseAdmin, tableNames);
-        logger.info("Old coprocessor jar: " + oldJarPaths);
-
-        Path hdfsCoprocessorJar = uploadCoprocessorJar(localCoprocessorJar, fileSystem, oldJarPaths);
-        logger.info("New coprocessor jar: " + hdfsCoprocessorJar);
-
-        List<String> processedTables = resetCoprocessorOnHTables(hbaseAdmin, hdfsCoprocessorJar, tableNames);
-
-        // Don't remove old jars, missing coprocessor jar will fail hbase
-        // removeOldJars(oldJarPaths, fileSystem);
-
-        hbaseAdmin.close();
-
-        logger.info("Processed " + processedTables);
-        logger.info("Active coprocessor jar: " + hdfsCoprocessorJar);
-    }
-
-    private static void printUsageAndExit() {
-        logger.warn("Probe run, exiting.");
-        logger.info("Usage: bin/kylin.sh org.apache.kylin.job.tools.DeployCoprocessorCLI JAR_FILE all|-cube CUBE1 CUBE2|-table TABLE1 TABLE2");
-        System.exit(0);
-    }
-
-    private static List<String> filterByCubes(List<String> allTableNames, List<String> cubeNames) {
-        CubeManager cubeManager = CubeManager.getInstance(KylinConfig.getInstanceFromEnv());
-        List<String> result = Lists.newArrayList();
-        for (String c : cubeNames) {
-            c = c.trim();
-            if (c.endsWith(","))
-                c = c.substring(0, c.length() - 1);
-
-            CubeInstance cubeInstance = cubeManager.getCube(c);
-            for (CubeSegment segment : cubeInstance.getSegments()) {
-                String tableName = segment.getStorageLocationIdentifier();
-                if (allTableNames.contains(tableName)) {
-                    result.add(tableName);
-                }
-            }
-        }
-        return result;
-    }
-
-    private static List<String> filterByTables(List<String> allTableNames, List<String> tableNames) {
-        List<String> result = Lists.newArrayList();
-        for (String t : tableNames) {
-            t = t.trim();
-            if (t.endsWith(","))
-                t = t.substring(0, t.length() - 1);
-
-            if (allTableNames.contains(t)) {
-                result.add(t);
-            }
-        }
-        return result;
-    }
-
-    public static void deployCoprocessor(HTableDescriptor tableDesc) {
-        try {
-            initHTableCoprocessor(tableDesc);
-            logger.info("hbase table " + tableDesc.getName() + " deployed with coprocessor.");
-
-        } catch (Exception ex) {
-            logger.error("Error deploying coprocessor on " + tableDesc.getName(), ex);
-            logger.error("Will try creating the table without coprocessor.");
-        }
-    }
-
-    private static void initHTableCoprocessor(HTableDescriptor desc) throws IOException {
-        KylinConfig kylinConfig = KylinConfig.getInstanceFromEnv();
-        Configuration hconf = HadoopUtil.getCurrentHBaseConfiguration();
-        FileSystem fileSystem = FileSystem.get(hconf);
-
-        String localCoprocessorJar = kylinConfig.getCoprocessorLocalJar();
-        Path hdfsCoprocessorJar = DeployCoprocessorCLI.uploadCoprocessorJar(localCoprocessorJar, fileSystem, null);
-
-        DeployCoprocessorCLI.addCoprocessorOnHTable(desc, hdfsCoprocessorJar);
-    }
-
-    public static void addCoprocessorOnHTable(HTableDescriptor desc, Path hdfsCoprocessorJar) throws IOException {
-        logger.info("Add coprocessor on " + desc.getNameAsString());
-        desc.addCoprocessor(ENDPOINT_CLS_NAMAE, hdfsCoprocessorJar, 1000, null);
-        desc.addCoprocessor(OBSERVER_CLS_NAME, hdfsCoprocessorJar, 1001, null);
-    }
-
-    public static void resetCoprocessor(String tableName, HBaseAdmin hbaseAdmin, Path hdfsCoprocessorJar) throws IOException {
-        logger.info("Disable " + tableName);
-        hbaseAdmin.disableTable(tableName);
-
-        logger.info("Unset coprocessor on " + tableName);
-        HTableDescriptor desc = hbaseAdmin.getTableDescriptor(TableName.valueOf(tableName));
-
-        // remove coprocessors of 1.x version
-        while (desc.hasCoprocessor(OBSERVER_CLS_NAME)) {
-            desc.removeCoprocessor(OBSERVER_CLS_NAME);
-        }
-        while (desc.hasCoprocessor(ENDPOINT_CLS_NAMAE)) {
-            desc.removeCoprocessor(ENDPOINT_CLS_NAMAE);
-        }
-        // remove coprocessors of 2.x version
-        while (desc.hasCoprocessor(CubeObserverClassV2)) {
-            desc.removeCoprocessor(CubeObserverClassV2);
-        }
-        while (desc.hasCoprocessor(CubeEndpointClassV2)) {
-            desc.removeCoprocessor(CubeEndpointClassV2);
-        }
-        while (desc.hasCoprocessor(IIEndpointClassV2)) {
-            desc.removeCoprocessor(IIEndpointClassV2);
-        }
-
-        addCoprocessorOnHTable(desc, hdfsCoprocessorJar);
-        hbaseAdmin.modifyTable(tableName, desc);
-
-        logger.info("Enable " + tableName);
-        hbaseAdmin.enableTable(tableName);
-    }
-
-    private static List<String> resetCoprocessorOnHTables(HBaseAdmin hbaseAdmin, Path hdfsCoprocessorJar, List<String> tableNames) throws IOException {
-        List<String> processed = new ArrayList<String>();
-
-        for (String tableName : tableNames) {
-            try {
-                resetCoprocessor(tableName, hbaseAdmin, hdfsCoprocessorJar);
-                processed.add(tableName);
-            } catch (IOException ex) {
-                logger.error("Error processing " + tableName, ex);
-            }
-        }
-        return processed;
-    }
-
-    public static Path getNewestCoprocessorJar(KylinConfig config, FileSystem fileSystem) throws IOException {
-        Path coprocessorDir = getCoprocessorHDFSDir(fileSystem, config);
-        FileStatus newestJar = null;
-        for (FileStatus fileStatus : fileSystem.listStatus(coprocessorDir)) {
-            if (fileStatus.getPath().toString().endsWith(".jar")) {
-                if (newestJar == null) {
-                    newestJar = fileStatus;
-                } else {
-                    if (newestJar.getModificationTime() < fileStatus.getModificationTime())
-                        newestJar = fileStatus;
-                }
-            }
-        }
-        if (newestJar == null)
-            return null;
-
-        Path path = newestJar.getPath().makeQualified(fileSystem.getUri(), null);
-        logger.info("The newest coprocessor is " + path.toString());
-        return path;
-    }
-
-    public static Path uploadCoprocessorJar(String localCoprocessorJar, FileSystem fileSystem, Set<String> oldJarPaths) throws IOException {
-        Path uploadPath = null;
-        File localCoprocessorFile = new File(localCoprocessorJar);
-
-        // check existing jars
-        if (oldJarPaths == null) {
-            oldJarPaths = new HashSet<String>();
-        }
-        Path coprocessorDir = getCoprocessorHDFSDir(fileSystem, KylinConfig.getInstanceFromEnv());
-        for (FileStatus fileStatus : fileSystem.listStatus(coprocessorDir)) {
-            if (fileStatus.getLen() == localCoprocessorJar.length() && fileStatus.getModificationTime() == localCoprocessorFile.lastModified()) {
-                uploadPath = fileStatus.getPath();
-                break;
-            }
-            String filename = fileStatus.getPath().toString();
-            if (filename.endsWith(".jar")) {
-                oldJarPaths.add(filename);
-            }
-        }
-
-        // upload if not existing
-        if (uploadPath == null) {
-            // figure out a unique new jar file name
-            Set<String> oldJarNames = new HashSet<String>();
-            for (String path : oldJarPaths) {
-                oldJarNames.add(new Path(path).getName());
-            }
-            String baseName = getBaseFileName(localCoprocessorJar);
-            String newName = null;
-            int i = 0;
-            while (newName == null) {
-                newName = baseName + "-" + (i++) + ".jar";
-                if (oldJarNames.contains(newName))
-                    newName = null;
-            }
-
-            // upload
-            uploadPath = new Path(coprocessorDir, newName);
-            FileInputStream in = null;
-            FSDataOutputStream out = null;
-            try {
-                in = new FileInputStream(localCoprocessorFile);
-                out = fileSystem.create(uploadPath);
-                IOUtils.copy(in, out);
-            } finally {
-                IOUtils.closeQuietly(in);
-                IOUtils.closeQuietly(out);
-            }
-
-            fileSystem.setTimes(uploadPath, localCoprocessorFile.lastModified(), -1);
-
-        }
-
-        uploadPath = uploadPath.makeQualified(fileSystem.getUri(), null);
-        return uploadPath;
-    }
-
-    private static String getBaseFileName(String localCoprocessorJar) {
-        File localJar = new File(localCoprocessorJar);
-        String baseName = localJar.getName();
-        if (baseName.endsWith(".jar"))
-            baseName = baseName.substring(0, baseName.length() - ".jar".length());
-        return baseName;
-    }
-
-    private static Path getCoprocessorHDFSDir(FileSystem fileSystem, KylinConfig config) throws IOException {
-        String hdfsWorkingDirectory = config.getHdfsWorkingDirectory();
-        Path coprocessorDir = new Path(hdfsWorkingDirectory, "coprocessor");
-        fileSystem.mkdirs(coprocessorDir);
-        return coprocessorDir;
-    }
-
-    private static Set<String> getCoprocessorJarPaths(HBaseAdmin hbaseAdmin, List<String> tableNames) throws IOException {
-        HashSet<String> result = new HashSet<String>();
-
-        for (String tableName : tableNames) {
-            HTableDescriptor tableDescriptor = null;
-            try {
-                tableDescriptor = hbaseAdmin.getTableDescriptor(TableName.valueOf(tableName));
-            } catch (TableNotFoundException e) {
-                logger.warn("Table not found " + tableName, e);
-                continue;
-            }
-
-            Matcher keyMatcher;
-            Matcher valueMatcher;
-            for (Map.Entry<ImmutableBytesWritable, ImmutableBytesWritable> e : tableDescriptor.getValues().entrySet()) {
-                keyMatcher = HConstants.CP_HTD_ATTR_KEY_PATTERN.matcher(Bytes.toString(e.getKey().get()));
-                if (!keyMatcher.matches()) {
-                    continue;
-                }
-                valueMatcher = HConstants.CP_HTD_ATTR_VALUE_PATTERN.matcher(Bytes.toString(e.getValue().get()));
-                if (!valueMatcher.matches()) {
-                    continue;
-                }
-
-                String jarPath = valueMatcher.group(1).trim();
-                String clsName = valueMatcher.group(2).trim();
-
-                if (OBSERVER_CLS_NAME.equals(clsName)) {
-                    result.add(jarPath);
-                }
-            }
-        }
-
-        return result;
-    }
-
-    private static List<String> getHTableNames(KylinConfig config) {
-        CubeManager cubeMgr = CubeManager.getInstance(config);
-
-        ArrayList<String> result = new ArrayList<String>();
-        for (CubeInstance cube : cubeMgr.listAllCubes()) {
-            for (CubeSegment seg : cube.getSegments(SegmentStatusEnum.READY)) {
-                String tableName = seg.getStorageLocationIdentifier();
-                if (StringUtils.isBlank(tableName) == false) {
-                    result.add(tableName);
-                    System.out.println("added new table: " + tableName);
-                }
-            }
-        }
-
-        for (IIInstance ii : IIManager.getInstance(config).listAllIIs()) {
-            for (IISegment seg : ii.getSegments(SegmentStatusEnum.READY)) {
-                String tableName = seg.getStorageLocationIdentifier();
-                if (StringUtils.isBlank(tableName) == false) {
-                    result.add(tableName);
-                    System.out.println("added new table: " + tableName);
-                }
-            }
-        }
-
-        return result;
-    }
-}
+/*
+ * 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.job.tools;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.regex.Matcher;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.TableNotFoundException;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.persistence.HBaseConnection;
+import org.apache.kylin.common.util.Bytes;
+import org.apache.kylin.common.util.HadoopUtil;
+import org.apache.kylin.cube.CubeInstance;
+import org.apache.kylin.cube.CubeManager;
+import org.apache.kylin.cube.CubeSegment;
+import org.apache.kylin.invertedindex.IIInstance;
+import org.apache.kylin.invertedindex.IIManager;
+import org.apache.kylin.invertedindex.IISegment;
+import org.apache.kylin.metadata.model.SegmentStatusEnum;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.collect.Lists;
+
+/**
+ * @author yangli9
+ */
+public class DeployCoprocessorCLI {
+
+    public static final String CubeObserverClassV2 = "org.apache.kylin.storage.hbase.cube.v1.coprocessor.observer.AggregateRegionObserver";
+    public static final String CubeEndpointClassV2 = "org.apache.kylin.storage.hbase.cube.v2.coprocessor.endpoint.CubeVisitService";
+    public static final String IIEndpointClassV2 = "org.apache.kylin.storage.hbase.ii.coprocessor.endpoint.IIEndpoint";
+    public static final String OBSERVER_CLS_NAME = "org.apache.kylin.storage.hbase.coprocessor.observer.AggregateRegionObserver";
+    public static final String ENDPOINT_CLS_NAMAE = "org.apache.kylin.storage.hbase.coprocessor.endpoint.IIEndpoint";
+    private static final Logger logger = LoggerFactory.getLogger(DeployCoprocessorCLI.class);
+
+    public static void main(String[] args) throws IOException {
+        KylinConfig kylinConfig = KylinConfig.getInstanceFromEnv();
+        Configuration hconf = HadoopUtil.getCurrentHBaseConfiguration();
+        FileSystem fileSystem = FileSystem.get(hconf);
+        Admin hbaseAdmin = HBaseConnection.get().getAdmin();
+
+        String localCoprocessorJar = new File(args[0]).getAbsolutePath();
+        logger.info("Identify coprocessor jar " + localCoprocessorJar);
+
+        List<String> tableNames = getHTableNames(kylinConfig);
+        logger.info("Identify tables " + tableNames);
+
+        if (args.length <= 1) {
+            printUsageAndExit();
+        }
+
+        String filterType = args[1].toLowerCase();
+        if (filterType.equals("-table")) {
+            tableNames = filterByTables(tableNames, Arrays.asList(args).subList(2, args.length));
+        } else if (filterType.equals("-cube")) {
+            tableNames = filterByCubes(tableNames, Arrays.asList(args).subList(2, args.length));
+        } else if (!filterType.equals("all")) {
+            printUsageAndExit();
+        }
+
+        logger.info("Will execute tables " + tableNames);
+
+        Set<String> oldJarPaths = getCoprocessorJarPaths(hbaseAdmin, tableNames);
+        logger.info("Old coprocessor jar: " + oldJarPaths);
+
+        Path hdfsCoprocessorJar = uploadCoprocessorJar(localCoprocessorJar, fileSystem, oldJarPaths);
+        logger.info("New coprocessor jar: " + hdfsCoprocessorJar);
+
+        List<String> processedTables = resetCoprocessorOnHTables(hbaseAdmin, hdfsCoprocessorJar, tableNames);
+
+        // Don't remove old jars, missing coprocessor jar will fail hbase
+        // removeOldJars(oldJarPaths, fileSystem);
+
+        hbaseAdmin.close();
+
+        logger.info("Processed " + processedTables);
+        logger.info("Active coprocessor jar: " + hdfsCoprocessorJar);
+    }
+
+    private static void printUsageAndExit() {
+        logger.warn("Probe run, exiting.");
+        logger.info("Usage: bin/kylin.sh org.apache.kylin.job.tools.DeployCoprocessorCLI JAR_FILE all|-cube CUBE1 CUBE2|-table TABLE1 TABLE2");
+        System.exit(0);
+    }
+
+    private static List<String> filterByCubes(List<String> allTableNames, List<String> cubeNames) {
+        CubeManager cubeManager = CubeManager.getInstance(KylinConfig.getInstanceFromEnv());
+        List<String> result = Lists.newArrayList();
+        for (String c : cubeNames) {
+            c = c.trim();
+            if (c.endsWith(","))
+                c = c.substring(0, c.length() - 1);
+
+            CubeInstance cubeInstance = cubeManager.getCube(c);
+            for (CubeSegment segment : cubeInstance.getSegments()) {
+                String tableName = segment.getStorageLocationIdentifier();
+                if (allTableNames.contains(tableName)) {
+                    result.add(tableName);
+                }
+            }
+        }
+        return result;
+    }
+
+    private static List<String> filterByTables(List<String> allTableNames, List<String> tableNames) {
+        List<String> result = Lists.newArrayList();
+        for (String t : tableNames) {
+            t = t.trim();
+            if (t.endsWith(","))
+                t = t.substring(0, t.length() - 1);
+
+            if (allTableNames.contains(t)) {
+                result.add(t);
+            }
+        }
+        return result;
+    }
+
+    public static void deployCoprocessor(HTableDescriptor tableDesc) {
+        try {
+            initHTableCoprocessor(tableDesc);
+            logger.info("hbase table " + tableDesc.getTableName() + " deployed with coprocessor.");
+
+        } catch (Exception ex) {
+            logger.error("Error deploying coprocessor on " + tableDesc.getTableName(), ex);
+            logger.error("Will try creating the table without coprocessor.");
+        }
+    }
+
+    private static void initHTableCoprocessor(HTableDescriptor desc) throws IOException {
+        KylinConfig kylinConfig = KylinConfig.getInstanceFromEnv();
+        Configuration hconf = HadoopUtil.getCurrentHBaseConfiguration();
+        FileSystem fileSystem = FileSystem.get(hconf);
+
+        String localCoprocessorJar = kylinConfig.getCoprocessorLocalJar();
+        Path hdfsCoprocessorJar = DeployCoprocessorCLI.uploadCoprocessorJar(localCoprocessorJar, fileSystem, null);
+
+        DeployCoprocessorCLI.addCoprocessorOnHTable(desc, hdfsCoprocessorJar);
+    }
+
+    public static void addCoprocessorOnHTable(HTableDescriptor desc, Path hdfsCoprocessorJar) throws IOException {
+        logger.info("Add coprocessor on " + desc.getNameAsString());
+        desc.addCoprocessor(ENDPOINT_CLS_NAMAE, hdfsCoprocessorJar, 1000, null);
+        desc.addCoprocessor(OBSERVER_CLS_NAME, hdfsCoprocessorJar, 1001, null);
+    }
+
+    public static void resetCoprocessor(String tableName, Admin hbaseAdmin, Path hdfsCoprocessorJar) throws IOException {
+        logger.info("Disable " + tableName);
+        hbaseAdmin.disableTable(TableName.valueOf(tableName));
+
+        logger.info("Unset coprocessor on " + tableName);
+        HTableDescriptor desc = hbaseAdmin.getTableDescriptor(TableName.valueOf(tableName));
+
+        // remove coprocessors of 1.x version
+        while (desc.hasCoprocessor(OBSERVER_CLS_NAME)) {
+            desc.removeCoprocessor(OBSERVER_CLS_NAME);
+        }
+        while (desc.hasCoprocessor(ENDPOINT_CLS_NAMAE)) {
+            desc.removeCoprocessor(ENDPOINT_CLS_NAMAE);
+        }
+        // remove coprocessors of 2.x version
+        while (desc.hasCoprocessor(CubeObserverClassV2)) {
+            desc.removeCoprocessor(CubeObserverClassV2);
+        }
+        while (desc.hasCoprocessor(CubeEndpointClassV2)) {
+            desc.removeCoprocessor(CubeEndpointClassV2);
+        }
+        while (desc.hasCoprocessor(IIEndpointClassV2)) {
+            desc.removeCoprocessor(IIEndpointClassV2);
+        }
+
+        addCoprocessorOnHTable(desc, hdfsCoprocessorJar);
+        hbaseAdmin.modifyTable(TableName.valueOf(tableName), desc);
+
+        logger.info("Enable " + tableName);
+        hbaseAdmin.enableTable(TableName.valueOf(tableName));
+    }
+
+    private static List<String> resetCoprocessorOnHTables(Admin hbaseAdmin, Path hdfsCoprocessorJar, List<String> tableNames) throws IOException {
+        List<String> processed = new ArrayList<String>();
+
+        for (String tableName : tableNames) {
+            try {
+                resetCoprocessor(tableName, hbaseAdmin, hdfsCoprocessorJar);
+                processed.add(tableName);
+            } catch (IOException ex) {
+                logger.error("Error processing " + tableName, ex);
+            }
+        }
+        return processed;
+    }
+
+    public static Path getNewestCoprocessorJar(KylinConfig config, FileSystem fileSystem) throws IOException {
+        Path coprocessorDir = getCoprocessorHDFSDir(fileSystem, config);
+        FileStatus newestJar = null;
+        for (FileStatus fileStatus : fileSystem.listStatus(coprocessorDir)) {
+            if (fileStatus.getPath().toString().endsWith(".jar")) {
+                if (newestJar == null) {
+                    newestJar = fileStatus;
+                } else {
+                    if (newestJar.getModificationTime() < fileStatus.getModificationTime())
+                        newestJar = fileStatus;
+                }
+            }
+        }
+        if (newestJar == null)
+            return null;
+
+        Path path = newestJar.getPath().makeQualified(fileSystem.getUri(), null);
+        logger.info("The newest coprocessor is " + path.toString());
+        return path;
+    }
+
+    public static Path uploadCoprocessorJar(String localCoprocessorJar, FileSystem fileSystem, Set<String> oldJarPaths) throws IOException {
+        Path uploadPath = null;
+        File localCoprocessorFile = new File(localCoprocessorJar);
+
+        // check existing jars
+        if (oldJarPaths == null) {
+            oldJarPaths = new HashSet<String>();
+        }
+        Path coprocessorDir = getCoprocessorHDFSDir(fileSystem, KylinConfig.getInstanceFromEnv());
+        for (FileStatus fileStatus : fileSystem.listStatus(coprocessorDir)) {
+            if (fileStatus.getLen() == localCoprocessorJar.length() && fileStatus.getModificationTime() == localCoprocessorFile.lastModified()) {
+                uploadPath = fileStatus.getPath();
+                break;
+            }
+            String filename = fileStatus.getPath().toString();
+            if (filename.endsWith(".jar")) {
+                oldJarPaths.add(filename);
+            }
+        }
+
+        // upload if not existing
+        if (uploadPath == null) {
+            // figure out a unique new jar file name
+            Set<String> oldJarNames = new HashSet<String>();
+            for (String path : oldJarPaths) {
+                oldJarNames.add(new Path(path).getName());
+            }
+            String baseName = getBaseFileName(localCoprocessorJar);
+            String newName = null;
+            int i = 0;
+            while (newName == null) {
+                newName = baseName + "-" + (i++) + ".jar";
+                if (oldJarNames.contains(newName))
+                    newName = null;
+            }
+
+            // upload
+            uploadPath = new Path(coprocessorDir, newName);
+            FileInputStream in = null;
+            FSDataOutputStream out = null;
+            try {
+                in = new FileInputStream(localCoprocessorFile);
+                out = fileSystem.create(uploadPath);
+                IOUtils.copy(in, out);
+            } finally {
+                IOUtils.closeQuietly(in);
+                IOUtils.closeQuietly(out);
+            }
+
+            fileSystem.setTimes(uploadPath, localCoprocessorFile.lastModified(), -1);
+
+        }
+
+        uploadPath = uploadPath.makeQualified(fileSystem.getUri(), null);
+        return uploadPath;
+    }
+
+    private static String getBaseFileName(String localCoprocessorJar) {
+        File localJar = new File(localCoprocessorJar);
+        String baseName = localJar.getName();
+        if (baseName.endsWith(".jar"))
+            baseName = baseName.substring(0, baseName.length() - ".jar".length());
+        return baseName;
+    }
+
+    private static Path getCoprocessorHDFSDir(FileSystem fileSystem, KylinConfig config) throws IOException {
+        String hdfsWorkingDirectory = config.getHdfsWorkingDirectory();
+        Path coprocessorDir = new Path(hdfsWorkingDirectory, "coprocessor");
+        fileSystem.mkdirs(coprocessorDir);
+        return coprocessorDir;
+    }
+
+    private static Set<String> getCoprocessorJarPaths(Admin hbaseAdmin, List<String> tableNames) throws IOException {
+        HashSet<String> result = new HashSet<String>();
+
+        for (String tableName : tableNames) {
+            HTableDescriptor tableDescriptor = null;
+            try {
+                tableDescriptor = hbaseAdmin.getTableDescriptor(TableName.valueOf(tableName));
+            } catch (TableNotFoundException e) {
+                logger.warn("Table not found " + tableName, e);
+                continue;
+            }
+
+            Matcher keyMatcher;
+            Matcher valueMatcher;
+            for (Map.Entry<ImmutableBytesWritable, ImmutableBytesWritable> e : tableDescriptor.getValues().entrySet()) {
+                keyMatcher = HConstants.CP_HTD_ATTR_KEY_PATTERN.matcher(Bytes.toString(e.getKey().get()));
+                if (!keyMatcher.matches()) {
+                    continue;
+                }
+                valueMatcher = HConstants.CP_HTD_ATTR_VALUE_PATTERN.matcher(Bytes.toString(e.getValue().get()));
+                if (!valueMatcher.matches()) {
+                    continue;
+                }
+
+                String jarPath = valueMatcher.group(1).trim();
+                String clsName = valueMatcher.group(2).trim();
+
+                if (OBSERVER_CLS_NAME.equals(clsName)) {
+                    result.add(jarPath);
+                }
+            }
+        }
+
+        return result;
+    }
+
+    private static List<String> getHTableNames(KylinConfig config) {
+        CubeManager cubeMgr = CubeManager.getInstance(config);
+
+        ArrayList<String> result = new ArrayList<String>();
+        for (CubeInstance cube : cubeMgr.listAllCubes()) {
+            for (CubeSegment seg : cube.getSegments(SegmentStatusEnum.READY)) {
+                String tableName = seg.getStorageLocationIdentifier();
+                if (StringUtils.isBlank(tableName) == false) {
+                    result.add(tableName);
+                    System.out.println("added new table: " + tableName);
+                }
+            }
+        }
+
+        for (IIInstance ii : IIManager.getInstance(config).listAllIIs()) {
+            for (IISegment seg : ii.getSegments(SegmentStatusEnum.READY)) {
+                String tableName = seg.getStorageLocationIdentifier();
+                if (StringUtils.isBlank(tableName) == false) {
+                    result.add(tableName);
+                    System.out.println("added new table: " + tableName);
+                }
+            }
+        }
+
+        return result;
+    }
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/48cec941/job/src/main/java/org/apache/kylin/job/tools/GridTableHBaseBenchmark.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/tools/GridTableHBaseBenchmark.java b/job/src/main/java/org/apache/kylin/job/tools/GridTableHBaseBenchmark.java
index 70e1df6..5fe5e58 100644
--- a/job/src/main/java/org/apache/kylin/job/tools/GridTableHBaseBenchmark.java
+++ b/job/src/main/java/org/apache/kylin/job/tools/GridTableHBaseBenchmark.java
@@ -28,13 +28,13 @@ import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotFoundException;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
-import org.apache.hadoop.hbase.client.HConnection;
-import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.filter.KeyOnlyFilter;
 import org.apache.kylin.common.persistence.HBaseConnection;
 import org.apache.kylin.common.util.Bytes;
@@ -74,8 +74,7 @@ public class GridTableHBaseBenchmark {
     public static void testGridTable(double hitRatio, double indexRatio) throws IOException {
         System.out.println("Testing grid table scanning, hit ratio " + hitRatio + ", index ratio " + indexRatio);
         String hbaseUrl = "hbase"; // use hbase-site.xml on classpath
-
-        HConnection conn = HBaseConnection.get(hbaseUrl);
+        Connection conn = HBaseConnection.get(hbaseUrl);
         createHTableIfNeeded(conn, TEST_TABLE);
         prepareData(conn);
 
@@ -91,10 +90,10 @@ public class GridTableHBaseBenchmark {
 
     }
 
-    private static void testColumnScan(HConnection conn, List<Pair<Integer, Integer>> colScans) throws IOException {
+    private static void testColumnScan(Connection conn, List<Pair<Integer, Integer>> colScans) throws IOException {
         Stats stats = new Stats("COLUMN_SCAN");
 
-        HTableInterface table = conn.getTable(TEST_TABLE);
+        Table table = conn.getTable(TableName.valueOf(TEST_TABLE));
         try {
             stats.markStart();
 
@@ -122,20 +121,20 @@ public class GridTableHBaseBenchmark {
         }
     }
 
-    private static void testRowScanNoIndexFullScan(HConnection conn, boolean[] hits) throws IOException {
+    private static void testRowScanNoIndexFullScan(Connection conn, boolean[] hits) throws IOException {
         fullScan(conn, hits, new Stats("ROW_SCAN_NO_IDX_FULL"));
     }
 
-    private static void testRowScanNoIndexSkipScan(HConnection conn, boolean[] hits) throws IOException {
+    private static void testRowScanNoIndexSkipScan(Connection conn, boolean[] hits) throws IOException {
         jumpScan(conn, hits, new Stats("ROW_SCAN_NO_IDX_SKIP"));
     }
 
-    private static void testRowScanWithIndex(HConnection conn, boolean[] hits) throws IOException {
+    private static void testRowScanWithIndex(Connection conn, boolean[] hits) throws IOException {
         jumpScan(conn, hits, new Stats("ROW_SCAN_IDX"));
     }
 
-    private static void fullScan(HConnection conn, boolean[] hits, Stats stats) throws IOException {
-        HTableInterface table = conn.getTable(TEST_TABLE);
+    private static void fullScan(Connection conn, boolean[] hits, Stats stats) throws IOException {
+        Table table = conn.getTable(TableName.valueOf(TEST_TABLE));
         try {
             stats.markStart();
 
@@ -156,11 +155,11 @@ public class GridTableHBaseBenchmark {
         }
     }
 
-    private static void jumpScan(HConnection conn, boolean[] hits, Stats stats) throws IOException {
+    private static void jumpScan(Connection conn, boolean[] hits, Stats stats) throws IOException {
 
         final int jumpThreshold = 6; // compensate for Scan() overhead, totally by experience
 
-        HTableInterface table = conn.getTable(TEST_TABLE);
+        Table table = conn.getTable(TableName.valueOf(TEST_TABLE));
         try {
 
             stats.markStart();
@@ -204,8 +203,8 @@ public class GridTableHBaseBenchmark {
         }
     }
 
-    private static void prepareData(HConnection conn) throws IOException {
-        HTableInterface table = conn.getTable(TEST_TABLE);
+    private static void prepareData(Connection conn) throws IOException {
+        Table table = conn.getTable(TableName.valueOf(TEST_TABLE));
 
         try {
             // check how many rows existing
@@ -232,7 +231,7 @@ public class GridTableHBaseBenchmark {
                 byte[] rowkey = Bytes.toBytes(i);
                 Put put = new Put(rowkey);
                 byte[] cell = randomBytes();
-                put.add(CF, QN, cell);
+                put.addColumn(CF, QN, cell);
                 table.put(put);
                 nBytes += cell.length;
                 dot(i, N_ROWS);
@@ -258,8 +257,8 @@ public class GridTableHBaseBenchmark {
         return bytes;
     }
 
-    private static void createHTableIfNeeded(HConnection conn, String tableName) throws IOException {
-        HBaseAdmin hbase = new HBaseAdmin(conn);
+    private static void createHTableIfNeeded(Connection conn, String tableName) throws IOException {
+        Admin hbase = conn.getAdmin();
 
         try {
             boolean tableExist = false;

http://git-wip-us.apache.org/repos/asf/kylin/blob/48cec941/job/src/main/java/org/apache/kylin/job/tools/HtableAlterMetadataCLI.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/tools/HtableAlterMetadataCLI.java b/job/src/main/java/org/apache/kylin/job/tools/HtableAlterMetadataCLI.java
index 53930e3..e283748 100644
--- a/job/src/main/java/org/apache/kylin/job/tools/HtableAlterMetadataCLI.java
+++ b/job/src/main/java/org/apache/kylin/job/tools/HtableAlterMetadataCLI.java
@@ -23,12 +23,11 @@ import java.io.IOException;
 import org.apache.commons.cli.Option;
 import org.apache.commons.cli.OptionBuilder;
 import org.apache.commons.cli.Options;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.util.ToolRunner;
+import org.apache.kylin.common.persistence.HBaseConnection;
 import org.apache.kylin.job.hadoop.AbstractHadoopJob;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -71,8 +70,7 @@ public class HtableAlterMetadataCLI extends AbstractHadoopJob {
     }
 
     private void alter() throws IOException {
-        Configuration conf = HBaseConfiguration.create();
-        HBaseAdmin hbaseAdmin = new HBaseAdmin(conf);
+        Admin hbaseAdmin = HBaseConnection.get().getAdmin();
         HTableDescriptor table = hbaseAdmin.getTableDescriptor(TableName.valueOf(tableName));
 
         hbaseAdmin.disableTable(table.getTableName());

http://git-wip-us.apache.org/repos/asf/kylin/blob/48cec941/job/src/main/java/org/apache/kylin/job/tools/RowCounterCLI.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/tools/RowCounterCLI.java b/job/src/main/java/org/apache/kylin/job/tools/RowCounterCLI.java
index 3329d27..4d44088 100644
--- a/job/src/main/java/org/apache/kylin/job/tools/RowCounterCLI.java
+++ b/job/src/main/java/org/apache/kylin/job/tools/RowCounterCLI.java
@@ -22,11 +22,12 @@ import java.util.Iterator;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.client.HConnection;
-import org.apache.hadoop.hbase.client.HConnectionManager;
-import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.kylin.common.util.Bytes;
 import org.apache.kylin.common.util.BytesUtil;
 import org.slf4j.Logger;
@@ -69,8 +70,8 @@ public class RowCounterCLI {
 
         logger.info("My Scan " + scan.toString());
 
-        HConnection conn = HConnectionManager.createConnection(conf);
-        HTableInterface tableInterface = conn.getTable(htableName);
+        Connection conn = ConnectionFactory.createConnection(conf);
+        Table tableInterface = conn.getTable(TableName.valueOf(htableName));
 
         Iterator<Result> iterator = tableInterface.getScanner(scan).iterator();
         int counter = 0;


[12/52] [abbrv] kylin git commit: Fix more unary operator expected bugs on v1.x

Posted by li...@apache.org.
Fix more unary operator expected bugs on v1.x

Signed-off-by: honma <ho...@ebay.com>


Project: http://git-wip-us.apache.org/repos/asf/kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/d6974474
Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/d6974474
Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/d6974474

Branch: refs/heads/1.x-HBase1.1.3
Commit: d6974474aaa827793981f79df14f2a6ce8612733
Parents: 4e18dfe
Author: Hao Chen <ha...@apache.org>
Authored: Wed Jan 27 18:00:48 2016 +0800
Committer: honma <ho...@ebay.com>
Committed: Wed Jan 27 21:39:14 2016 +0800

----------------------------------------------------------------------
 bin/kylin.sh     | 2 +-
 bin/metastore.sh | 8 ++++----
 2 files changed, 5 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/d6974474/bin/kylin.sh
----------------------------------------------------------------------
diff --git a/bin/kylin.sh b/bin/kylin.sh
index 5bf3eab..6b0a495 100755
--- a/bin/kylin.sh
+++ b/bin/kylin.sh
@@ -115,7 +115,7 @@ then
     fi
     
 # tool command
-elif [[ $1 = org.apache.kylin.* ]]
+elif [[ "$1" = org.apache.kylin.* ]]
 then
     #retrive $hive_dependency and $hbase_dependency
     source ${dir}/find-hive-dependency.sh

http://git-wip-us.apache.org/repos/asf/kylin/blob/d6974474/bin/metastore.sh
----------------------------------------------------------------------
diff --git a/bin/metastore.sh b/bin/metastore.sh
index 39593d4..5d4ef40 100755
--- a/bin/metastore.sh
+++ b/bin/metastore.sh
@@ -28,7 +28,7 @@
 dir=$(dirname ${0})
 source ${dir}/check-env.sh
 
-if [ $1 == "backup" ]
+if [ "$1" == "backup" ]
 then
 
     mkdir -p ${KYLIN_HOME}/meta_backups
@@ -41,19 +41,19 @@ then
     ${KYLIN_HOME}/bin/kylin.sh org.apache.kylin.common.persistence.ResourceTool download ${_file}
     echo "metadata store backed up to ${_file}"
 
-elif [ $1 == "restore" ]
+elif [ "$1" == "restore" ]
 then
 
     _file=$2
     echo "Starting restoring $_file"
     ${KYLIN_HOME}/bin/kylin.sh org.apache.kylin.common.persistence.ResourceTool upload $_file
 
-elif [ $1 == "reset" ]
+elif [ "$1" == "reset" ]
 then
 
     ${KYLIN_HOME}/bin/kylin.sh org.apache.kylin.common.persistence.ResourceTool  reset
     
-elif [ $1 == "clean" ]
+elif [ "$1" == "clean" ]
 then
 
     ${KYLIN_HOME}/bin/kylin.sh org.apache.kylin.job.hadoop.cube.MetadataCleanupJob "${@:2}"


[20/52] [abbrv] kylin git commit: KYLIN-1400 kylin.metadata.url with hbase namespace problem

Posted by li...@apache.org.
KYLIN-1400 kylin.metadata.url with hbase namespace problem


Project: http://git-wip-us.apache.org/repos/asf/kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/474dbfa6
Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/474dbfa6
Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/474dbfa6

Branch: refs/heads/1.x-HBase1.1.3
Commit: 474dbfa67f3b4cba6439b6b5d4ec68a85806f2de
Parents: 62a8cb2
Author: Zhihua Deng <de...@163.com>
Authored: Wed Feb 3 14:40:55 2016 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Fri Feb 5 15:53:00 2016 +0800

----------------------------------------------------------------------
 common/src/main/java/org/apache/kylin/common/KylinConfig.java | 5 ++++-
 1 file changed, 4 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/474dbfa6/common/src/main/java/org/apache/kylin/common/KylinConfig.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/kylin/common/KylinConfig.java b/common/src/main/java/org/apache/kylin/common/KylinConfig.java
index 790e7a5..59cb86b 100644
--- a/common/src/main/java/org/apache/kylin/common/KylinConfig.java
+++ b/common/src/main/java/org/apache/kylin/common/KylinConfig.java
@@ -316,7 +316,10 @@ public class KylinConfig {
         if (!root.endsWith("/")) {
             root += "/";
         }
-        return root + getMetadataUrlPrefix() + "/";
+        return new StringBuffer(root)
+                     .append(StringUtils.replaceChars(getMetadataUrlPrefix(), ':', '-'))
+                     .append("/")
+                     .toString();
     }
 
     public String getHBaseClusterFs() {


[23/52] [abbrv] kylin git commit: KYLIN-1401 Passwords in kylin.properties should be enctrypted

Posted by li...@apache.org.
KYLIN-1401 Passwords in kylin.properties should be enctrypted

Project: http://git-wip-us.apache.org/repos/asf/kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/eb5deb31
Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/eb5deb31
Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/eb5deb31

Branch: refs/heads/1.x-HBase1.1.3
Commit: eb5deb31e88f398258828f15542d68187c90ee8d
Parents: d19eb0d
Author: shaofengshi <sh...@apache.org>
Authored: Fri Feb 5 17:46:07 2016 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Fri Feb 5 17:46:07 2016 +0800

----------------------------------------------------------------------
 .../kylin/rest/security/PasswordPlaceholderConfigurer.java   | 8 +++++++-
 website/_docs/howto/howto_ldap_and_sso.md                    | 4 ++--
 2 files changed, 9 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/eb5deb31/server/src/main/java/org/apache/kylin/rest/security/PasswordPlaceholderConfigurer.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/kylin/rest/security/PasswordPlaceholderConfigurer.java b/server/src/main/java/org/apache/kylin/rest/security/PasswordPlaceholderConfigurer.java
index d7aa119..36b498c 100644
--- a/server/src/main/java/org/apache/kylin/rest/security/PasswordPlaceholderConfigurer.java
+++ b/server/src/main/java/org/apache/kylin/rest/security/PasswordPlaceholderConfigurer.java
@@ -77,7 +77,13 @@ public class PasswordPlaceholderConfigurer extends PropertyPlaceholderConfigurer
     }
 
     public static void main(String[] args) {
+        if (args.length != 1) {
+            System.out.println("Usage: java org.apache.kylin.rest.security.PasswordPlaceholderConfigurer <your_password>");
+            System.exit(1);
+        }
+        
         BCryptPasswordEncoder bCryptPasswordEncoder = new BCryptPasswordEncoder();
-        System.out.println(bCryptPasswordEncoder.encode("MODELER"));
+        System.out.println("The hash of your password is: ");
+        System.out.println(bCryptPasswordEncoder.encode(args[0]));
     }
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/eb5deb31/website/_docs/howto/howto_ldap_and_sso.md
----------------------------------------------------------------------
diff --git a/website/_docs/howto/howto_ldap_and_sso.md b/website/_docs/howto/howto_ldap_and_sso.md
index a810371..1102559 100644
--- a/website/_docs/howto/howto_ldap_and_sso.md
+++ b/website/_docs/howto/howto_ldap_and_sso.md
@@ -13,12 +13,12 @@ Kylin supports LDAP authentication for enterprise or production deployment; This
 
 #### Configure LDAP server info
 
-Firstly, provide LDAP URL, and username/password if the LDAP server is secured;
+Firstly, provide LDAP URL, and username/password if the LDAP server is secured; The password in kylin.properties need be salted; You can Google "Generate a BCrypt Password" or run org.apache.kylin.rest.security.PasswordPlaceholderConfigurer to get a hash of your password.
 
 ```
 ldap.server=ldap://<your_ldap_host>:<port>
 ldap.username=<your_user_name>
-ldap.password=<your_password>
+ldap.password=<your_password_hash>
 ```
 
 Secondly, provide the user search patterns, this is by LDAP design, here is just a sample:


[09/52] [abbrv] kylin git commit: KYLIN-1296 Update restAPI document of build cube

Posted by li...@apache.org.
KYLIN-1296 Update restAPI document of build cube


Project: http://git-wip-us.apache.org/repos/asf/kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/785efd82
Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/785efd82
Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/785efd82

Branch: refs/heads/1.x-HBase1.1.3
Commit: 785efd82e0753d6a3fc4ff5fd23d1476f8a7cc5e
Parents: bfccac9
Author: lidongsjtu <li...@apache.org>
Authored: Mon Jan 25 12:52:40 2016 +0800
Committer: lidongsjtu <li...@apache.org>
Committed: Mon Jan 25 12:52:50 2016 +0800

----------------------------------------------------------------------
 website/_docs/howto/howto_use_restapi.md | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/785efd82/website/_docs/howto/howto_use_restapi.md
----------------------------------------------------------------------
diff --git a/website/_docs/howto/howto_use_restapi.md b/website/_docs/howto/howto_use_restapi.md
index 27d4e7a..a49a2db 100644
--- a/website/_docs/howto/howto_use_restapi.md
+++ b/website/_docs/howto/howto_use_restapi.md
@@ -634,7 +634,7 @@ Get descriptor for specified cube instance.
 #### Request Body
 * startTime - `required` `long` Start timestamp of data to build, e.g. 1388563200000 for 2014-1-1
 * endTime - `required` `long` End timestamp of data to build
-* buildType - `required` `string` Build type: 'BUILD' OR 'MERGE'
+* buildType - `required` `string` Supported build type: 'BUILD', 'MERGE', 'REFRESH'
 
 #### Response Sample
 ```


[08/52] [abbrv] kylin git commit: Feature changed: automatically append hive dependent jars to 'tmpjars'

Posted by li...@apache.org.
Feature changed: automatically append hive dependent jars to 'tmpjars'

Signed-off-by: shaofengshi <sh...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/bfccac90
Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/bfccac90
Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/bfccac90

Branch: refs/heads/1.x-HBase1.1.3
Commit: bfccac90f54a39f87738eb7a6955a0f0e404bc60
Parents: 73cfbef
Author: Zhong Yanghong <ya...@ebay.com>
Authored: Sun Jan 24 09:35:22 2016 +0000
Committer: shaofengshi <sh...@apache.org>
Committed: Mon Jan 25 09:47:11 2016 +0800

----------------------------------------------------------------------
 .../kylin/job/hadoop/AbstractHadoopJob.java     | 123 ++++++++++++++++---
 1 file changed, 103 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/bfccac90/job/src/main/java/org/apache/kylin/job/hadoop/AbstractHadoopJob.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/AbstractHadoopJob.java b/job/src/main/java/org/apache/kylin/job/hadoop/AbstractHadoopJob.java
index a851756..f5c85eb 100644
--- a/job/src/main/java/org/apache/kylin/job/hadoop/AbstractHadoopJob.java
+++ b/job/src/main/java/org/apache/kylin/job/hadoop/AbstractHadoopJob.java
@@ -30,6 +30,8 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.Map;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
 
 import org.apache.commons.cli.Option;
 import org.apache.commons.cli.OptionBuilder;
@@ -127,6 +129,26 @@ public abstract class AbstractHadoopJob extends Configured implements Tool {
         return retVal;
     }
 
+    private static final String KYLIN_HIVE_DEPENDENCY_JARS = "[^,]*hive-exec.jar|[^,]*hive-metastore.jar|[^,]*hive-hcatalog-core[0-9.-]*jar";
+
+    String filterKylinHiveDependency(String kylinHiveDependency) {
+        if (StringUtils.isBlank(kylinHiveDependency))
+            return "";
+
+        StringBuilder jarList = new StringBuilder();
+
+        Pattern hivePattern = Pattern.compile(KYLIN_HIVE_DEPENDENCY_JARS);
+        Matcher matcher = hivePattern.matcher(kylinHiveDependency);
+
+        while (matcher.find()) {
+            if (jarList.length() > 0)
+                jarList.append(",");
+            jarList.append(matcher.group());
+        }
+
+        return jarList.toString();
+    }
+
     private static final String MAP_REDUCE_CLASSPATH = "mapreduce.application.classpath";
 
     protected void setJobClasspath(Job job) {
@@ -141,14 +163,13 @@ public abstract class AbstractHadoopJob extends Configured implements Tool {
 
         String kylinHiveDependency = System.getProperty("kylin.hive.dependency");
         String kylinHBaseDependency = System.getProperty("kylin.hbase.dependency");
-        logger.info("append kylin.hive.dependency: " + kylinHiveDependency + " and kylin.hbase.dependency: " + kylinHBaseDependency + " to " + MAP_REDUCE_CLASSPATH);
+        logger.info("append kylin.hbase.dependency: " + kylinHBaseDependency + " to " + MAP_REDUCE_CLASSPATH);
 
         Configuration jobConf = job.getConfiguration();
         String classpath = jobConf.get(MAP_REDUCE_CLASSPATH);
         if (classpath == null || classpath.length() == 0) {
             logger.info("Didn't find " + MAP_REDUCE_CLASSPATH + " in job configuration, will run 'mapred classpath' to get the default value.");
             classpath = getDefaultMapRedClasspath();
-            classpath = classpath.replace(":", ","); // yarn classpath is comma separated
             logger.info("The default mapred classpath is: " + classpath);
         }
 
@@ -158,40 +179,102 @@ public abstract class AbstractHadoopJob extends Configured implements Tool {
             classpath = classpath + "," + kylinHBaseDependency;
         }
 
+        jobConf.set(MAP_REDUCE_CLASSPATH, classpath);
+        logger.info("Hadoop job classpath is: " + job.getConfiguration().get(MAP_REDUCE_CLASSPATH));
+
+        /*
+         *  set extra dependencies as tmpjars & tmpfiles if configured
+         */
+        StringBuilder kylinDependency = new StringBuilder();
+
+        // for hive dependencies
         if (kylinHiveDependency != null) {
             // yarn classpath is comma separated
             kylinHiveDependency = kylinHiveDependency.replace(":", ",");
-            classpath = classpath + "," + kylinHiveDependency;
+
+            logger.info("Hive Dependencies Before Filtered: " + kylinHiveDependency);
+            String filteredHive = filterKylinHiveDependency(kylinHiveDependency);
+            logger.info("Hive Dependencies After Filtered: " + filteredHive);
+
+            if (kylinDependency.length() > 0)
+                kylinDependency.append(",");
+            kylinDependency.append(filteredHive);
         }
 
-        jobConf.set(MAP_REDUCE_CLASSPATH, classpath);
-        logger.info("Hadoop job classpath is: " + job.getConfiguration().get(MAP_REDUCE_CLASSPATH));
+        // for KylinJobMRLibDir
+        String mrLibDir = KylinConfig.getInstanceFromEnv().getKylinJobMRLibDir();
+        if (!StringUtils.isBlank(mrLibDir)) {
+            File dirFileMRLIB = new File(mrLibDir);
+            if (dirFileMRLIB.exists()) {
+                if (kylinDependency.length() > 0)
+                    kylinDependency.append(",");
+                kylinDependency.append(mrLibDir);
+            } else {
+                logger.info("The directory '" + mrLibDir + "' for 'kylin.job.mr.lib.dir' does not exist!!!");
+            }
+        }
 
-        // set extra dependencies as tmpjars & tmpfiles if configured
-        setJobTmpJarsAndFiles(job);
+        setJobTmpJarsAndFiles(job, kylinDependency.toString());
     }
 
-    private void setJobTmpJarsAndFiles(Job job) {
-        String mrLibDir = KylinConfig.getInstanceFromEnv().getKylinJobMRLibDir();
-        if (StringUtils.isBlank(mrLibDir))
+    private void setJobTmpJarsAndFiles(Job job, String kylinDependency) {
+        if (StringUtils.isBlank(kylinDependency))
             return;
 
+        String[] fNameList = kylinDependency.split(",");
+
         try {
             Configuration jobConf = job.getConfiguration();
-            FileSystem fs = FileSystem.get(new Configuration(jobConf));
-            FileStatus[] fList = fs.listStatus(new Path(mrLibDir));
-            
+            FileSystem fs = FileSystem.getLocal(jobConf);
+
             StringBuilder jarList = new StringBuilder();
             StringBuilder fileList = new StringBuilder();
-            
+
+            for (String fileName : fNameList) {
+                Path p = new Path(fileName);
+                if (fs.getFileStatus(p).isDirectory()) {
+                    appendTmpDir(job, fileName);
+                    continue;
+                }
+
+                StringBuilder list = (p.getName().endsWith(".jar")) ? jarList : fileList;
+                if (list.length() > 0)
+                    list.append(",");
+                list.append(fs.getFileStatus(p).getPath().toString());
+            }
+
+            appendTmpFiles(fileList.toString(), jobConf);
+            appendTmpJars(jarList.toString(), jobConf);
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    private void appendTmpDir(Job job, String tmpDir) {
+        if (StringUtils.isBlank(tmpDir))
+            return;
+
+        try {
+            Configuration jobConf = job.getConfiguration();
+            FileSystem fs = FileSystem.getLocal(jobConf);
+            FileStatus[] fList = fs.listStatus(new Path(tmpDir));
+
+            StringBuilder jarList = new StringBuilder();
+            StringBuilder fileList = new StringBuilder();
+
             for (FileStatus file : fList) {
                 Path p = file.getPath();
+                if (fs.getFileStatus(p).isDirectory()) {
+                    appendTmpDir(job, p.toString());
+                    continue;
+                }
+
                 StringBuilder list = (p.getName().endsWith(".jar")) ? jarList : fileList;
                 if (list.length() > 0)
                     list.append(",");
-                list.append(mrLibDir + "/" + file.getPath().getName());
+                list.append(fs.getFileStatus(p).getPath().toString());
             }
-            
+
             appendTmpFiles(fileList.toString(), jobConf);
             appendTmpJars(jarList.toString(), jobConf);
 
@@ -203,7 +286,7 @@ public abstract class AbstractHadoopJob extends Configured implements Tool {
     private void appendTmpJars(String jarList, Configuration conf) {
         if (StringUtils.isBlank(jarList))
             return;
-        
+
         String tmpJars = conf.get("tmpjars", null);
         if (tmpJars == null) {
             tmpJars = jarList;
@@ -217,7 +300,7 @@ public abstract class AbstractHadoopJob extends Configured implements Tool {
     private void appendTmpFiles(String fileList, Configuration conf) {
         if (StringUtils.isBlank(fileList))
             return;
-        
+
         String tmpFiles = conf.get("tmpfiles", null);
         if (tmpFiles == null) {
             tmpFiles = fileList;
@@ -338,10 +421,10 @@ public abstract class AbstractHadoopJob extends Configured implements Tool {
         else
             hdfsMetaDir = "file:///" + hdfsMetaDir;
         logger.info("HDFS meta dir is: " + hdfsMetaDir);
-        
+
         appendTmpFiles(hdfsMetaDir, conf);
     }
-    
+
     private void dumpResources(KylinConfig kylinConfig, File metaDir, ArrayList<String> dumpList) throws IOException {
         ResourceStore from = ResourceStore.getStore(kylinConfig);
         KylinConfig localConfig = KylinConfig.createInstanceFromUri(metaDir.getAbsolutePath());


[03/52] [abbrv] kylin git commit: KYLIN-1343 Compile pass, pending IT

Posted by li...@apache.org.
KYLIN-1343 Compile pass, pending IT


Project: http://git-wip-us.apache.org/repos/asf/kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/df7ae17a
Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/df7ae17a
Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/df7ae17a

Branch: refs/heads/1.x-HBase1.1.3
Commit: df7ae17a7d90c0c37987f52a2cff4f42a46a1e66
Parents: 714bb4a
Author: Yang Li <li...@apache.org>
Authored: Sat Jan 23 13:24:32 2016 +0800
Committer: Yang Li <li...@apache.org>
Committed: Sat Jan 23 13:24:32 2016 +0800

----------------------------------------------------------------------
 .../calcite/sql2rel/SqlToRelConverter.java      | 8126 ++++++++++--------
 .../org/apache/kylin/jdbc/KylinConnection.java  |    2 +-
 .../org/apache/kylin/jdbc/KylinJdbcFactory.java |    5 +-
 .../java/org/apache/kylin/jdbc/KylinMeta.java   |   47 +-
 .../kylin/jdbc/KylinPreparedStatement.java      |    2 +
 .../org/apache/kylin/jdbc/KylinResultSet.java   |    5 +-
 pom.xml                                         |    2 +-
 .../kylin/query/optrule/OLAPJoinRule.java       |    2 +-
 .../kylin/query/relnode/OLAPAggregateRel.java   |    6 +-
 .../kylin/query/relnode/OLAPFilterRel.java      |    6 +-
 .../apache/kylin/query/relnode/OLAPJoinRel.java |   22 +-
 .../kylin/query/relnode/OLAPLimitRel.java       |    5 +-
 .../kylin/query/relnode/OLAPProjectRel.java     |    5 +-
 .../apache/kylin/query/relnode/OLAPSortRel.java |    7 +-
 .../kylin/query/relnode/OLAPTableScan.java      |    5 +-
 .../relnode/OLAPToEnumerableConverter.java      |    5 +-
 16 files changed, 4644 insertions(+), 3608 deletions(-)
----------------------------------------------------------------------



[13/52] [abbrv] kylin git commit: KYLIN-1363 Fix unary operator expected bug

Posted by li...@apache.org.
KYLIN-1363 Fix unary operator expected bug

Signed-off-by: honma <ho...@ebay.com>


Project: http://git-wip-us.apache.org/repos/asf/kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/4e18dfe2
Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/4e18dfe2
Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/4e18dfe2

Branch: refs/heads/1.x-HBase1.1.3
Commit: 4e18dfe2055ca0c23a3fbff4d7e3175a48e743be
Parents: 4a5d059
Author: Hao Chen <hc...@ebay.com>
Authored: Sun Jan 24 14:16:58 2016 +0800
Committer: honma <ho...@ebay.com>
Committed: Wed Jan 27 21:39:14 2016 +0800

----------------------------------------------------------------------
 bin/kylin.sh | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/4e18dfe2/bin/kylin.sh
----------------------------------------------------------------------
diff --git a/bin/kylin.sh b/bin/kylin.sh
index 8ca3c91..5bf3eab 100755
--- a/bin/kylin.sh
+++ b/bin/kylin.sh
@@ -22,7 +22,7 @@ source ${dir}/check-env.sh
 mkdir -p ${KYLIN_HOME}/logs
 
 # start command
-if [ $1 == "start" ]
+if [ "$1" == "start" ]
 then
 
     if [ -f "${KYLIN_HOME}/pid" ]
@@ -93,7 +93,7 @@ then
     exit 0
 
 # stop command
-elif [ $1 == "stop" ]
+elif [ "$1" == "stop" ]
 then
     if [ -f "${KYLIN_HOME}/pid" ]
     then


[35/52] [abbrv] kylin git commit: KYLIN-1416 remove website from coding branch

Posted by li...@apache.org.
http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/assets/fonts/fontawesome-webfont.svg
----------------------------------------------------------------------
diff --git a/website/assets/fonts/fontawesome-webfont.svg b/website/assets/fonts/fontawesome-webfont.svg
deleted file mode 100644
index d907b25..0000000
--- a/website/assets/fonts/fontawesome-webfont.svg
+++ /dev/null
@@ -1,520 +0,0 @@
-<?xml version="1.0" standalone="no"?>
-<!DOCTYPE svg PUBLIC "-//W3C//DTD SVG 1.1//EN" "http://www.w3.org/Graphics/SVG/1.1/DTD/svg11.dtd" >
-<svg xmlns="http://www.w3.org/2000/svg">
-<metadata></metadata>
-<defs>
-<font id="fontawesomeregular" horiz-adv-x="1536" >
-<font-face units-per-em="1792" ascent="1536" descent="-256" />
-<missing-glyph horiz-adv-x="448" />
-<glyph unicode=" "  horiz-adv-x="448" />
-<glyph unicode="&#x09;" horiz-adv-x="448" />
-<glyph unicode="&#xa0;" horiz-adv-x="448" />
-<glyph unicode="&#xa8;" horiz-adv-x="1792" />
-<glyph unicode="&#xa9;" horiz-adv-x="1792" />
-<glyph unicode="&#xae;" horiz-adv-x="1792" />
-<glyph unicode="&#xb4;" horiz-adv-x="1792" />
-<glyph unicode="&#xc6;" horiz-adv-x="1792" />
-<glyph unicode="&#xd8;" horiz-adv-x="1792" />
-<glyph unicode="&#x2000;" horiz-adv-x="768" />
-<glyph unicode="&#x2001;" horiz-adv-x="1537" />
-<glyph unicode="&#x2002;" horiz-adv-x="768" />
-<glyph unicode="&#x2003;" horiz-adv-x="1537" />
-<glyph unicode="&#x2004;" horiz-adv-x="512" />
-<glyph unicode="&#x2005;" horiz-adv-x="384" />
-<glyph unicode="&#x2006;" horiz-adv-x="256" />
-<glyph unicode="&#x2007;" horiz-adv-x="256" />
-<glyph unicode="&#x2008;" horiz-adv-x="192" />
-<glyph unicode="&#x2009;" horiz-adv-x="307" />
-<glyph unicode="&#x200a;" horiz-adv-x="85" />
-<glyph unicode="&#x202f;" horiz-adv-x="307" />
-<glyph unicode="&#x205f;" horiz-adv-x="384" />
-<glyph unicode="&#x2122;" horiz-adv-x="1792" />
-<glyph unicode="&#x221e;" horiz-adv-x="1792" />
-<glyph unicode="&#x2260;" horiz-adv-x="1792" />
-<glyph unicode="&#x25fc;" horiz-adv-x="500" d="M0 0z" />
-<glyph unicode="&#xf000;" horiz-adv-x="1792" d="M1699 1350q0 -35 -43 -78l-632 -632v-768h320q26 0 45 -19t19 -45t-19 -45t-45 -19h-896q-26 0 -45 19t-19 45t19 45t45 19h320v768l-632 632q-43 43 -43 78q0 23 18 36.5t38 17.5t43 4h1408q23 0 43 -4t38 -17.5t18 -36.5z" />
-<glyph unicode="&#xf001;" d="M1536 1312v-1120q0 -50 -34 -89t-86 -60.5t-103.5 -32t-96.5 -10.5t-96.5 10.5t-103.5 32t-86 60.5t-34 89t34 89t86 60.5t103.5 32t96.5 10.5q105 0 192 -39v537l-768 -237v-709q0 -50 -34 -89t-86 -60.5t-103.5 -32t-96.5 -10.5t-96.5 10.5t-103.5 32t-86 60.5t-34 89 t34 89t86 60.5t103.5 32t96.5 10.5q105 0 192 -39v967q0 31 19 56.5t49 35.5l832 256q12 4 28 4q40 0 68 -28t28 -68z" />
-<glyph unicode="&#xf002;" horiz-adv-x="1664" d="M1152 704q0 185 -131.5 316.5t-316.5 131.5t-316.5 -131.5t-131.5 -316.5t131.5 -316.5t316.5 -131.5t316.5 131.5t131.5 316.5zM1664 -128q0 -52 -38 -90t-90 -38q-54 0 -90 38l-343 342q-179 -124 -399 -124q-143 0 -273.5 55.5t-225 150t-150 225t-55.5 273.5 t55.5 273.5t150 225t225 150t273.5 55.5t273.5 -55.5t225 -150t150 -225t55.5 -273.5q0 -220 -124 -399l343 -343q37 -37 37 -90z" />
-<glyph unicode="&#xf003;" horiz-adv-x="1792" d="M1664 32v768q-32 -36 -69 -66q-268 -206 -426 -338q-51 -43 -83 -67t-86.5 -48.5t-102.5 -24.5h-1h-1q-48 0 -102.5 24.5t-86.5 48.5t-83 67q-158 132 -426 338q-37 30 -69 66v-768q0 -13 9.5 -22.5t22.5 -9.5h1472q13 0 22.5 9.5t9.5 22.5zM1664 1083v11v13.5t-0.5 13 t-3 12.5t-5.5 9t-9 7.5t-14 2.5h-1472q-13 0 -22.5 -9.5t-9.5 -22.5q0 -168 147 -284q193 -152 401 -317q6 -5 35 -29.5t46 -37.5t44.5 -31.5t50.5 -27.5t43 -9h1h1q20 0 43 9t50.5 27.5t44.5 31.5t46 37.5t35 29.5q208 165 401 317q54 43 100.5 115.5t46.5 131.5z M1792 1120v-1088q0 -66 -47 -113t-113 -47h-1472q-66 0 -113 47t-47 113v1088q0 66 47 113t113 47h1472q66 0 113 -47t47 -113z" />
-<glyph unicode="&#xf004;" horiz-adv-x="1792" d="M896 -128q-26 0 -44 18l-624 602q-10 8 -27.5 26t-55.5 65.5t-68 97.5t-53.5 121t-23.5 138q0 220 127 344t351 124q62 0 126.5 -21.5t120 -58t95.5 -68.5t76 -68q36 36 76 68t95.5 68.5t120 58t126.5 21.5q224 0 351 -124t127 -344q0 -221 -229 -450l-623 -600 q-18 -18 -44 -18z" />
-<glyph unicode="&#xf005;" horiz-adv-x="1664" d="M1664 889q0 -22 -26 -48l-363 -354l86 -500q1 -7 1 -20q0 -21 -10.5 -35.5t-30.5 -14.5q-19 0 -40 12l-449 236l-449 -236q-22 -12 -40 -12q-21 0 -31.5 14.5t-10.5 35.5q0 6 2 20l86 500l-364 354q-25 27 -25 48q0 37 56 46l502 73l225 455q19 41 49 41t49 -41l225 -455 l502 -73q56 -9 56 -46z" />
-<glyph unicode="&#xf006;" horiz-adv-x="1664" d="M1137 532l306 297l-422 62l-189 382l-189 -382l-422 -62l306 -297l-73 -421l378 199l377 -199zM1664 889q0 -22 -26 -48l-363 -354l86 -500q1 -7 1 -20q0 -50 -41 -50q-19 0 -40 12l-449 236l-449 -236q-22 -12 -40 -12q-21 0 -31.5 14.5t-10.5 35.5q0 6 2 20l86 500 l-364 354q-25 27 -25 48q0 37 56 46l502 73l225 455q19 41 49 41t49 -41l225 -455l502 -73q56 -9 56 -46z" />
-<glyph unicode="&#xf007;" horiz-adv-x="1408" d="M1408 131q0 -120 -73 -189.5t-194 -69.5h-874q-121 0 -194 69.5t-73 189.5q0 53 3.5 103.5t14 109t26.5 108.5t43 97.5t62 81t85.5 53.5t111.5 20q9 0 42 -21.5t74.5 -48t108 -48t133.5 -21.5t133.5 21.5t108 48t74.5 48t42 21.5q61 0 111.5 -20t85.5 -53.5t62 -81 t43 -97.5t26.5 -108.5t14 -109t3.5 -103.5zM1088 1024q0 -159 -112.5 -271.5t-271.5 -112.5t-271.5 112.5t-112.5 271.5t112.5 271.5t271.5 112.5t271.5 -112.5t112.5 -271.5z" />
-<glyph unicode="&#xf008;" horiz-adv-x="1920" d="M384 -64v128q0 26 -19 45t-45 19h-128q-26 0 -45 -19t-19 -45v-128q0 -26 19 -45t45 -19h128q26 0 45 19t19 45zM384 320v128q0 26 -19 45t-45 19h-128q-26 0 -45 -19t-19 -45v-128q0 -26 19 -45t45 -19h128q26 0 45 19t19 45zM384 704v128q0 26 -19 45t-45 19h-128 q-26 0 -45 -19t-19 -45v-128q0 -26 19 -45t45 -19h128q26 0 45 19t19 45zM1408 -64v512q0 26 -19 45t-45 19h-768q-26 0 -45 -19t-19 -45v-512q0 -26 19 -45t45 -19h768q26 0 45 19t19 45zM384 1088v128q0 26 -19 45t-45 19h-128q-26 0 -45 -19t-19 -45v-128q0 -26 19 -45 t45 -19h128q26 0 45 19t19 45zM1792 -64v128q0 26 -19 45t-45 19h-128q-26 0 -45 -19t-19 -45v-128q0 -26 19 -45t45 -19h128q26 0 45 19t19 45zM1408 704v512q0 26 -19 45t-45 19h-768q-26 0 -45 -19t-19 -45v-512q0 -26 19 -45t45 -19h768q26 0 45 19t19 45zM1792 320v128 q0 26 -19 45t-45 19h-128q-26 0 -45 -19t-19 -45v-128q0 -26 19 -45t45 -19h128q26 0 45 19t19 45zM1792 704v128q0 26 -19 45t-45 19h-128q-26 0 -45 -19t-19 -45v-128q0 -26 19 -45t45 -19h128q26 0 45 19t1
 9 45zM1792 1088v128q0 26 -19 45t-45 19h-128q-26 0 -45 -19 t-19 -45v-128q0 -26 19 -45t45 -19h128q26 0 45 19t19 45zM1920 1248v-1344q0 -66 -47 -113t-113 -47h-1600q-66 0 -113 47t-47 113v1344q0 66 47 113t113 47h1600q66 0 113 -47t47 -113z" />
-<glyph unicode="&#xf009;" horiz-adv-x="1664" d="M768 512v-384q0 -52 -38 -90t-90 -38h-512q-52 0 -90 38t-38 90v384q0 52 38 90t90 38h512q52 0 90 -38t38 -90zM768 1280v-384q0 -52 -38 -90t-90 -38h-512q-52 0 -90 38t-38 90v384q0 52 38 90t90 38h512q52 0 90 -38t38 -90zM1664 512v-384q0 -52 -38 -90t-90 -38 h-512q-52 0 -90 38t-38 90v384q0 52 38 90t90 38h512q52 0 90 -38t38 -90zM1664 1280v-384q0 -52 -38 -90t-90 -38h-512q-52 0 -90 38t-38 90v384q0 52 38 90t90 38h512q52 0 90 -38t38 -90z" />
-<glyph unicode="&#xf00a;" horiz-adv-x="1792" d="M512 288v-192q0 -40 -28 -68t-68 -28h-320q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h320q40 0 68 -28t28 -68zM512 800v-192q0 -40 -28 -68t-68 -28h-320q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h320q40 0 68 -28t28 -68zM1152 288v-192q0 -40 -28 -68t-68 -28h-320 q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h320q40 0 68 -28t28 -68zM512 1312v-192q0 -40 -28 -68t-68 -28h-320q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h320q40 0 68 -28t28 -68zM1152 800v-192q0 -40 -28 -68t-68 -28h-320q-40 0 -68 28t-28 68v192q0 40 28 68t68 28 h320q40 0 68 -28t28 -68zM1792 288v-192q0 -40 -28 -68t-68 -28h-320q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h320q40 0 68 -28t28 -68zM1152 1312v-192q0 -40 -28 -68t-68 -28h-320q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h320q40 0 68 -28t28 -68zM1792 800v-192 q0 -40 -28 -68t-68 -28h-320q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h320q40 0 68 -28t28 -68zM1792 1312v-192q0 -40 -28 -68t-68 -28h-320q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h320q40 0 68 -28
 t28 -68z" />
-<glyph unicode="&#xf00b;" horiz-adv-x="1792" d="M512 288v-192q0 -40 -28 -68t-68 -28h-320q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h320q40 0 68 -28t28 -68zM512 800v-192q0 -40 -28 -68t-68 -28h-320q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h320q40 0 68 -28t28 -68zM1792 288v-192q0 -40 -28 -68t-68 -28h-960 q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h960q40 0 68 -28t28 -68zM512 1312v-192q0 -40 -28 -68t-68 -28h-320q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h320q40 0 68 -28t28 -68zM1792 800v-192q0 -40 -28 -68t-68 -28h-960q-40 0 -68 28t-28 68v192q0 40 28 68t68 28 h960q40 0 68 -28t28 -68zM1792 1312v-192q0 -40 -28 -68t-68 -28h-960q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h960q40 0 68 -28t28 -68z" />
-<glyph unicode="&#xf00c;" horiz-adv-x="1792" d="M1671 970q0 -40 -28 -68l-724 -724l-136 -136q-28 -28 -68 -28t-68 28l-136 136l-362 362q-28 28 -28 68t28 68l136 136q28 28 68 28t68 -28l294 -295l656 657q28 28 68 28t68 -28l136 -136q28 -28 28 -68z" />
-<glyph unicode="&#xf00d;" horiz-adv-x="1408" d="M1298 214q0 -40 -28 -68l-136 -136q-28 -28 -68 -28t-68 28l-294 294l-294 -294q-28 -28 -68 -28t-68 28l-136 136q-28 28 -28 68t28 68l294 294l-294 294q-28 28 -28 68t28 68l136 136q28 28 68 28t68 -28l294 -294l294 294q28 28 68 28t68 -28l136 -136q28 -28 28 -68 t-28 -68l-294 -294l294 -294q28 -28 28 -68z" />
-<glyph unicode="&#xf00e;" horiz-adv-x="1664" d="M1024 736v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-224v-224q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v224h-224q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h224v224q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5v-224h224 q13 0 22.5 -9.5t9.5 -22.5zM1152 704q0 185 -131.5 316.5t-316.5 131.5t-316.5 -131.5t-131.5 -316.5t131.5 -316.5t316.5 -131.5t316.5 131.5t131.5 316.5zM1664 -128q0 -53 -37.5 -90.5t-90.5 -37.5q-54 0 -90 38l-343 342q-179 -124 -399 -124q-143 0 -273.5 55.5 t-225 150t-150 225t-55.5 273.5t55.5 273.5t150 225t225 150t273.5 55.5t273.5 -55.5t225 -150t150 -225t55.5 -273.5q0 -220 -124 -399l343 -343q37 -37 37 -90z" />
-<glyph unicode="&#xf010;" horiz-adv-x="1664" d="M1024 736v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-576q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h576q13 0 22.5 -9.5t9.5 -22.5zM1152 704q0 185 -131.5 316.5t-316.5 131.5t-316.5 -131.5t-131.5 -316.5t131.5 -316.5t316.5 -131.5t316.5 131.5t131.5 316.5z M1664 -128q0 -53 -37.5 -90.5t-90.5 -37.5q-54 0 -90 38l-343 342q-179 -124 -399 -124q-143 0 -273.5 55.5t-225 150t-150 225t-55.5 273.5t55.5 273.5t150 225t225 150t273.5 55.5t273.5 -55.5t225 -150t150 -225t55.5 -273.5q0 -220 -124 -399l343 -343q37 -37 37 -90z " />
-<glyph unicode="&#xf011;" d="M1536 640q0 -156 -61 -298t-164 -245t-245 -164t-298 -61t-298 61t-245 164t-164 245t-61 298q0 182 80.5 343t226.5 270q43 32 95.5 25t83.5 -50q32 -42 24.5 -94.5t-49.5 -84.5q-98 -74 -151.5 -181t-53.5 -228q0 -104 40.5 -198.5t109.5 -163.5t163.5 -109.5 t198.5 -40.5t198.5 40.5t163.5 109.5t109.5 163.5t40.5 198.5q0 121 -53.5 228t-151.5 181q-42 32 -49.5 84.5t24.5 94.5q31 43 84 50t95 -25q146 -109 226.5 -270t80.5 -343zM896 1408v-640q0 -52 -38 -90t-90 -38t-90 38t-38 90v640q0 52 38 90t90 38t90 -38t38 -90z" />
-<glyph unicode="&#xf012;" horiz-adv-x="1792" d="M256 96v-192q0 -14 -9 -23t-23 -9h-192q-14 0 -23 9t-9 23v192q0 14 9 23t23 9h192q14 0 23 -9t9 -23zM640 224v-320q0 -14 -9 -23t-23 -9h-192q-14 0 -23 9t-9 23v320q0 14 9 23t23 9h192q14 0 23 -9t9 -23zM1024 480v-576q0 -14 -9 -23t-23 -9h-192q-14 0 -23 9t-9 23 v576q0 14 9 23t23 9h192q14 0 23 -9t9 -23zM1408 864v-960q0 -14 -9 -23t-23 -9h-192q-14 0 -23 9t-9 23v960q0 14 9 23t23 9h192q14 0 23 -9t9 -23zM1792 1376v-1472q0 -14 -9 -23t-23 -9h-192q-14 0 -23 9t-9 23v1472q0 14 9 23t23 9h192q14 0 23 -9t9 -23z" />
-<glyph unicode="&#xf013;" d="M1024 640q0 106 -75 181t-181 75t-181 -75t-75 -181t75 -181t181 -75t181 75t75 181zM1536 749v-222q0 -12 -8 -23t-20 -13l-185 -28q-19 -54 -39 -91q35 -50 107 -138q10 -12 10 -25t-9 -23q-27 -37 -99 -108t-94 -71q-12 0 -26 9l-138 108q-44 -23 -91 -38 q-16 -136 -29 -186q-7 -28 -36 -28h-222q-14 0 -24.5 8.5t-11.5 21.5l-28 184q-49 16 -90 37l-141 -107q-10 -9 -25 -9q-14 0 -25 11q-126 114 -165 168q-7 10 -7 23q0 12 8 23q15 21 51 66.5t54 70.5q-27 50 -41 99l-183 27q-13 2 -21 12.5t-8 23.5v222q0 12 8 23t19 13 l186 28q14 46 39 92q-40 57 -107 138q-10 12 -10 24q0 10 9 23q26 36 98.5 107.5t94.5 71.5q13 0 26 -10l138 -107q44 23 91 38q16 136 29 186q7 28 36 28h222q14 0 24.5 -8.5t11.5 -21.5l28 -184q49 -16 90 -37l142 107q9 9 24 9q13 0 25 -10q129 -119 165 -170q7 -8 7 -22 q0 -12 -8 -23q-15 -21 -51 -66.5t-54 -70.5q26 -50 41 -98l183 -28q13 -2 21 -12.5t8 -23.5z" />
-<glyph unicode="&#xf014;" horiz-adv-x="1408" d="M512 800v-576q0 -14 -9 -23t-23 -9h-64q-14 0 -23 9t-9 23v576q0 14 9 23t23 9h64q14 0 23 -9t9 -23zM768 800v-576q0 -14 -9 -23t-23 -9h-64q-14 0 -23 9t-9 23v576q0 14 9 23t23 9h64q14 0 23 -9t9 -23zM1024 800v-576q0 -14 -9 -23t-23 -9h-64q-14 0 -23 9t-9 23v576 q0 14 9 23t23 9h64q14 0 23 -9t9 -23zM1152 76v948h-896v-948q0 -22 7 -40.5t14.5 -27t10.5 -8.5h832q3 0 10.5 8.5t14.5 27t7 40.5zM480 1152h448l-48 117q-7 9 -17 11h-317q-10 -2 -17 -11zM1408 1120v-64q0 -14 -9 -23t-23 -9h-96v-948q0 -83 -47 -143.5t-113 -60.5h-832 q-66 0 -113 58.5t-47 141.5v952h-96q-14 0 -23 9t-9 23v64q0 14 9 23t23 9h309l70 167q15 37 54 63t79 26h320q40 0 79 -26t54 -63l70 -167h309q14 0 23 -9t9 -23z" />
-<glyph unicode="&#xf015;" horiz-adv-x="1664" d="M1408 544v-480q0 -26 -19 -45t-45 -19h-384v384h-256v-384h-384q-26 0 -45 19t-19 45v480q0 1 0.5 3t0.5 3l575 474l575 -474q1 -2 1 -6zM1631 613l-62 -74q-8 -9 -21 -11h-3q-13 0 -21 7l-692 577l-692 -577q-12 -8 -24 -7q-13 2 -21 11l-62 74q-8 10 -7 23.5t11 21.5 l719 599q32 26 76 26t76 -26l244 -204v195q0 14 9 23t23 9h192q14 0 23 -9t9 -23v-408l219 -182q10 -8 11 -21.5t-7 -23.5z" />
-<glyph unicode="&#xf016;" d="M1468 1156q28 -28 48 -76t20 -88v-1152q0 -40 -28 -68t-68 -28h-1344q-40 0 -68 28t-28 68v1600q0 40 28 68t68 28h896q40 0 88 -20t76 -48zM1024 1400v-376h376q-10 29 -22 41l-313 313q-12 12 -41 22zM1408 -128v1024h-416q-40 0 -68 28t-28 68v416h-768v-1536h1280z " />
-<glyph unicode="&#xf017;" d="M896 992v-448q0 -14 -9 -23t-23 -9h-320q-14 0 -23 9t-9 23v64q0 14 9 23t23 9h224v352q0 14 9 23t23 9h64q14 0 23 -9t9 -23zM1312 640q0 148 -73 273t-198 198t-273 73t-273 -73t-198 -198t-73 -273t73 -273t198 -198t273 -73t273 73t198 198t73 273zM1536 640 q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
-<glyph unicode="&#xf018;" horiz-adv-x="1920" d="M1111 540v4l-24 320q-1 13 -11 22.5t-23 9.5h-186q-13 0 -23 -9.5t-11 -22.5l-24 -320v-4q-1 -12 8 -20t21 -8h244q12 0 21 8t8 20zM1870 73q0 -73 -46 -73h-704q13 0 22 9.5t8 22.5l-20 256q-1 13 -11 22.5t-23 9.5h-272q-13 0 -23 -9.5t-11 -22.5l-20 -256 q-1 -13 8 -22.5t22 -9.5h-704q-46 0 -46 73q0 54 26 116l417 1044q8 19 26 33t38 14h339q-13 0 -23 -9.5t-11 -22.5l-15 -192q-1 -14 8 -23t22 -9h166q13 0 22 9t8 23l-15 192q-1 13 -11 22.5t-23 9.5h339q20 0 38 -14t26 -33l417 -1044q26 -62 26 -116z" />
-<glyph unicode="&#xf019;" horiz-adv-x="1664" d="M1280 192q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1536 192q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1664 416v-320q0 -40 -28 -68t-68 -28h-1472q-40 0 -68 28t-28 68v320q0 40 28 68t68 28h465l135 -136 q58 -56 136 -56t136 56l136 136h464q40 0 68 -28t28 -68zM1339 985q17 -41 -14 -70l-448 -448q-18 -19 -45 -19t-45 19l-448 448q-31 29 -14 70q17 39 59 39h256v448q0 26 19 45t45 19h256q26 0 45 -19t19 -45v-448h256q42 0 59 -39z" />
-<glyph unicode="&#xf01a;" d="M1120 608q0 -12 -10 -24l-319 -319q-11 -9 -23 -9t-23 9l-320 320q-15 16 -7 35q8 20 30 20h192v352q0 14 9 23t23 9h192q14 0 23 -9t9 -23v-352h192q14 0 23 -9t9 -23zM768 1184q-148 0 -273 -73t-198 -198t-73 -273t73 -273t198 -198t273 -73t273 73t198 198t73 273 t-73 273t-198 198t-273 73zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
-<glyph unicode="&#xf01b;" d="M1118 660q-8 -20 -30 -20h-192v-352q0 -14 -9 -23t-23 -9h-192q-14 0 -23 9t-9 23v352h-192q-14 0 -23 9t-9 23q0 12 10 24l319 319q11 9 23 9t23 -9l320 -320q15 -16 7 -35zM768 1184q-148 0 -273 -73t-198 -198t-73 -273t73 -273t198 -198t273 -73t273 73t198 198 t73 273t-73 273t-198 198t-273 73zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
-<glyph unicode="&#xf01c;" d="M1023 576h316q-1 3 -2.5 8t-2.5 8l-212 496h-708l-212 -496q-1 -2 -2.5 -8t-2.5 -8h316l95 -192h320zM1536 546v-482q0 -26 -19 -45t-45 -19h-1408q-26 0 -45 19t-19 45v482q0 62 25 123l238 552q10 25 36.5 42t52.5 17h832q26 0 52.5 -17t36.5 -42l238 -552 q25 -61 25 -123z" />
-<glyph unicode="&#xf01d;" d="M1184 640q0 -37 -32 -55l-544 -320q-15 -9 -32 -9q-16 0 -32 8q-32 19 -32 56v640q0 37 32 56q33 18 64 -1l544 -320q32 -18 32 -55zM1312 640q0 148 -73 273t-198 198t-273 73t-273 -73t-198 -198t-73 -273t73 -273t198 -198t273 -73t273 73t198 198t73 273zM1536 640 q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
-<glyph unicode="&#xf01e;" d="M1536 1280v-448q0 -26 -19 -45t-45 -19h-448q-42 0 -59 40q-17 39 14 69l138 138q-148 137 -349 137q-104 0 -198.5 -40.5t-163.5 -109.5t-109.5 -163.5t-40.5 -198.5t40.5 -198.5t109.5 -163.5t163.5 -109.5t198.5 -40.5q119 0 225 52t179 147q7 10 23 12q14 0 25 -9 l137 -138q9 -8 9.5 -20.5t-7.5 -22.5q-109 -132 -264 -204.5t-327 -72.5q-156 0 -298 61t-245 164t-164 245t-61 298t61 298t164 245t245 164t298 61q147 0 284.5 -55.5t244.5 -156.5l130 129q29 31 70 14q39 -17 39 -59z" />
-<glyph unicode="&#xf021;" d="M1511 480q0 -5 -1 -7q-64 -268 -268 -434.5t-478 -166.5q-146 0 -282.5 55t-243.5 157l-129 -129q-19 -19 -45 -19t-45 19t-19 45v448q0 26 19 45t45 19h448q26 0 45 -19t19 -45t-19 -45l-137 -137q71 -66 161 -102t187 -36q134 0 250 65t186 179q11 17 53 117 q8 23 30 23h192q13 0 22.5 -9.5t9.5 -22.5zM1536 1280v-448q0 -26 -19 -45t-45 -19h-448q-26 0 -45 19t-19 45t19 45l138 138q-148 137 -349 137q-134 0 -250 -65t-186 -179q-11 -17 -53 -117q-8 -23 -30 -23h-199q-13 0 -22.5 9.5t-9.5 22.5v7q65 268 270 434.5t480 166.5 q146 0 284 -55.5t245 -156.5l130 129q19 19 45 19t45 -19t19 -45z" />
-<glyph unicode="&#xf022;" horiz-adv-x="1792" d="M384 352v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5zM384 608v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5z M384 864v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5zM1536 352v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-960q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h960q13 0 22.5 -9.5t9.5 -22.5z M1536 608v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-960q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h960q13 0 22.5 -9.5t9.5 -22.5zM1536 864v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-960q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h960q13 0 22.5 -9.5 t9.5 -22.5zM1664 160v832q0 13 -9.5 22.5t-22.5 9.5h-1472q-13 0 -22.5 -9.5t-9.5 -22.5v-832q0 -13 9.5 -22.5t22.5 -9.5h1472q13 0 22.5 9.5t9.5 22.5zM1792 1248v-1088q0 -66 -47 -113t-113 -47h-1472q-66 0 -1
 13 47t-47 113v1088q0 66 47 113t113 47h1472q66 0 113 -47 t47 -113z" />
-<glyph unicode="&#xf023;" horiz-adv-x="1152" d="M320 768h512v192q0 106 -75 181t-181 75t-181 -75t-75 -181v-192zM1152 672v-576q0 -40 -28 -68t-68 -28h-960q-40 0 -68 28t-28 68v576q0 40 28 68t68 28h32v192q0 184 132 316t316 132t316 -132t132 -316v-192h32q40 0 68 -28t28 -68z" />
-<glyph unicode="&#xf024;" horiz-adv-x="1792" d="M320 1280q0 -72 -64 -110v-1266q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v1266q-64 38 -64 110q0 53 37.5 90.5t90.5 37.5t90.5 -37.5t37.5 -90.5zM1792 1216v-763q0 -25 -12.5 -38.5t-39.5 -27.5q-215 -116 -369 -116q-61 0 -123.5 22t-108.5 48 t-115.5 48t-142.5 22q-192 0 -464 -146q-17 -9 -33 -9q-26 0 -45 19t-19 45v742q0 32 31 55q21 14 79 43q236 120 421 120q107 0 200 -29t219 -88q38 -19 88 -19q54 0 117.5 21t110 47t88 47t54.5 21q26 0 45 -19t19 -45z" />
-<glyph unicode="&#xf025;" horiz-adv-x="1664" d="M1664 650q0 -166 -60 -314l-20 -49l-185 -33q-22 -83 -90.5 -136.5t-156.5 -53.5v-32q0 -14 -9 -23t-23 -9h-64q-14 0 -23 9t-9 23v576q0 14 9 23t23 9h64q14 0 23 -9t9 -23v-32q71 0 130 -35.5t93 -95.5l68 12q29 95 29 193q0 148 -88 279t-236.5 209t-315.5 78 t-315.5 -78t-236.5 -209t-88 -279q0 -98 29 -193l68 -12q34 60 93 95.5t130 35.5v32q0 14 9 23t23 9h64q14 0 23 -9t9 -23v-576q0 -14 -9 -23t-23 -9h-64q-14 0 -23 9t-9 23v32q-88 0 -156.5 53.5t-90.5 136.5l-185 33l-20 49q-60 148 -60 314q0 151 67 291t179 242.5 t266 163.5t320 61t320 -61t266 -163.5t179 -242.5t67 -291z" />
-<glyph unicode="&#xf026;" horiz-adv-x="768" d="M768 1184v-1088q0 -26 -19 -45t-45 -19t-45 19l-333 333h-262q-26 0 -45 19t-19 45v384q0 26 19 45t45 19h262l333 333q19 19 45 19t45 -19t19 -45z" />
-<glyph unicode="&#xf027;" horiz-adv-x="1152" d="M768 1184v-1088q0 -26 -19 -45t-45 -19t-45 19l-333 333h-262q-26 0 -45 19t-19 45v384q0 26 19 45t45 19h262l333 333q19 19 45 19t45 -19t19 -45zM1152 640q0 -76 -42.5 -141.5t-112.5 -93.5q-10 -5 -25 -5q-26 0 -45 18.5t-19 45.5q0 21 12 35.5t29 25t34 23t29 35.5 t12 57t-12 57t-29 35.5t-34 23t-29 25t-12 35.5q0 27 19 45.5t45 18.5q15 0 25 -5q70 -27 112.5 -93t42.5 -142z" />
-<glyph unicode="&#xf028;" horiz-adv-x="1664" d="M768 1184v-1088q0 -26 -19 -45t-45 -19t-45 19l-333 333h-262q-26 0 -45 19t-19 45v384q0 26 19 45t45 19h262l333 333q19 19 45 19t45 -19t19 -45zM1152 640q0 -76 -42.5 -141.5t-112.5 -93.5q-10 -5 -25 -5q-26 0 -45 18.5t-19 45.5q0 21 12 35.5t29 25t34 23t29 35.5 t12 57t-12 57t-29 35.5t-34 23t-29 25t-12 35.5q0 27 19 45.5t45 18.5q15 0 25 -5q70 -27 112.5 -93t42.5 -142zM1408 640q0 -153 -85 -282.5t-225 -188.5q-13 -5 -25 -5q-27 0 -46 19t-19 45q0 39 39 59q56 29 76 44q74 54 115.5 135.5t41.5 173.5t-41.5 173.5 t-115.5 135.5q-20 15 -76 44q-39 20 -39 59q0 26 19 45t45 19q13 0 26 -5q140 -59 225 -188.5t85 -282.5zM1664 640q0 -230 -127 -422.5t-338 -283.5q-13 -5 -26 -5q-26 0 -45 19t-19 45q0 36 39 59q7 4 22.5 10.5t22.5 10.5q46 25 82 51q123 91 192 227t69 289t-69 289 t-192 227q-36 26 -82 51q-7 4 -22.5 10.5t-22.5 10.5q-39 23 -39 59q0 26 19 45t45 19q13 0 26 -5q211 -91 338 -283.5t127 -422.5z" />
-<glyph unicode="&#xf029;" horiz-adv-x="1408" d="M384 384v-128h-128v128h128zM384 1152v-128h-128v128h128zM1152 1152v-128h-128v128h128zM128 129h384v383h-384v-383zM128 896h384v384h-384v-384zM896 896h384v384h-384v-384zM640 640v-640h-640v640h640zM1152 128v-128h-128v128h128zM1408 128v-128h-128v128h128z M1408 640v-384h-384v128h-128v-384h-128v640h384v-128h128v128h128zM640 1408v-640h-640v640h640zM1408 1408v-640h-640v640h640z" />
-<glyph unicode="&#xf02a;" horiz-adv-x="1792" d="M63 0h-63v1408h63v-1408zM126 1h-32v1407h32v-1407zM220 1h-31v1407h31v-1407zM377 1h-31v1407h31v-1407zM534 1h-62v1407h62v-1407zM660 1h-31v1407h31v-1407zM723 1h-31v1407h31v-1407zM786 1h-31v1407h31v-1407zM943 1h-63v1407h63v-1407zM1100 1h-63v1407h63v-1407z M1226 1h-63v1407h63v-1407zM1352 1h-63v1407h63v-1407zM1446 1h-63v1407h63v-1407zM1635 1h-94v1407h94v-1407zM1698 1h-32v1407h32v-1407zM1792 0h-63v1408h63v-1408z" />
-<glyph unicode="&#xf02b;" d="M448 1088q0 53 -37.5 90.5t-90.5 37.5t-90.5 -37.5t-37.5 -90.5t37.5 -90.5t90.5 -37.5t90.5 37.5t37.5 90.5zM1515 512q0 -53 -37 -90l-491 -492q-39 -37 -91 -37q-53 0 -90 37l-715 716q-38 37 -64.5 101t-26.5 117v416q0 52 38 90t90 38h416q53 0 117 -26.5t102 -64.5 l715 -714q37 -39 37 -91z" />
-<glyph unicode="&#xf02c;" horiz-adv-x="1920" d="M448 1088q0 53 -37.5 90.5t-90.5 37.5t-90.5 -37.5t-37.5 -90.5t37.5 -90.5t90.5 -37.5t90.5 37.5t37.5 90.5zM1515 512q0 -53 -37 -90l-491 -492q-39 -37 -91 -37q-53 0 -90 37l-715 716q-38 37 -64.5 101t-26.5 117v416q0 52 38 90t90 38h416q53 0 117 -26.5t102 -64.5 l715 -714q37 -39 37 -91zM1899 512q0 -53 -37 -90l-491 -492q-39 -37 -91 -37q-36 0 -59 14t-53 45l470 470q37 37 37 90q0 52 -37 91l-715 714q-38 38 -102 64.5t-117 26.5h224q53 0 117 -26.5t102 -64.5l715 -714q37 -39 37 -91z" />
-<glyph unicode="&#xf02d;" horiz-adv-x="1664" d="M1639 1058q40 -57 18 -129l-275 -906q-19 -64 -76.5 -107.5t-122.5 -43.5h-923q-77 0 -148.5 53.5t-99.5 131.5q-24 67 -2 127q0 4 3 27t4 37q1 8 -3 21.5t-3 19.5q2 11 8 21t16.5 23.5t16.5 23.5q23 38 45 91.5t30 91.5q3 10 0.5 30t-0.5 28q3 11 17 28t17 23 q21 36 42 92t25 90q1 9 -2.5 32t0.5 28q4 13 22 30.5t22 22.5q19 26 42.5 84.5t27.5 96.5q1 8 -3 25.5t-2 26.5q2 8 9 18t18 23t17 21q8 12 16.5 30.5t15 35t16 36t19.5 32t26.5 23.5t36 11.5t47.5 -5.5l-1 -3q38 9 51 9h761q74 0 114 -56t18 -130l-274 -906 q-36 -119 -71.5 -153.5t-128.5 -34.5h-869q-27 0 -38 -15q-11 -16 -1 -43q24 -70 144 -70h923q29 0 56 15.5t35 41.5l300 987q7 22 5 57q38 -15 59 -43zM575 1056q-4 -13 2 -22.5t20 -9.5h608q13 0 25.5 9.5t16.5 22.5l21 64q4 13 -2 22.5t-20 9.5h-608q-13 0 -25.5 -9.5 t-16.5 -22.5zM492 800q-4 -13 2 -22.5t20 -9.5h608q13 0 25.5 9.5t16.5 22.5l21 64q4 13 -2 22.5t-20 9.5h-608q-13 0 -25.5 -9.5t-16.5 -22.5z" />
-<glyph unicode="&#xf02e;" horiz-adv-x="1280" d="M1164 1408q23 0 44 -9q33 -13 52.5 -41t19.5 -62v-1289q0 -34 -19.5 -62t-52.5 -41q-19 -8 -44 -8q-48 0 -83 32l-441 424l-441 -424q-36 -33 -83 -33q-23 0 -44 9q-33 13 -52.5 41t-19.5 62v1289q0 34 19.5 62t52.5 41q21 9 44 9h1048z" />
-<glyph unicode="&#xf02f;" horiz-adv-x="1664" d="M384 0h896v256h-896v-256zM384 640h896v384h-160q-40 0 -68 28t-28 68v160h-640v-640zM1536 576q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1664 576v-416q0 -13 -9.5 -22.5t-22.5 -9.5h-224v-160q0 -40 -28 -68t-68 -28h-960q-40 0 -68 28t-28 68 v160h-224q-13 0 -22.5 9.5t-9.5 22.5v416q0 79 56.5 135.5t135.5 56.5h64v544q0 40 28 68t68 28h672q40 0 88 -20t76 -48l152 -152q28 -28 48 -76t20 -88v-256h64q79 0 135.5 -56.5t56.5 -135.5z" />
-<glyph unicode="&#xf030;" horiz-adv-x="1920" d="M960 864q119 0 203.5 -84.5t84.5 -203.5t-84.5 -203.5t-203.5 -84.5t-203.5 84.5t-84.5 203.5t84.5 203.5t203.5 84.5zM1664 1280q106 0 181 -75t75 -181v-896q0 -106 -75 -181t-181 -75h-1408q-106 0 -181 75t-75 181v896q0 106 75 181t181 75h224l51 136 q19 49 69.5 84.5t103.5 35.5h512q53 0 103.5 -35.5t69.5 -84.5l51 -136h224zM960 128q185 0 316.5 131.5t131.5 316.5t-131.5 316.5t-316.5 131.5t-316.5 -131.5t-131.5 -316.5t131.5 -316.5t316.5 -131.5z" />
-<glyph unicode="&#xf031;" horiz-adv-x="1664" d="M725 977l-170 -450q33 0 136.5 -2t160.5 -2q19 0 57 2q-87 253 -184 452zM0 -128l2 79q23 7 56 12.5t57 10.5t49.5 14.5t44.5 29t31 50.5l237 616l280 724h75h53q8 -14 11 -21l205 -480q33 -78 106 -257.5t114 -274.5q15 -34 58 -144.5t72 -168.5q20 -45 35 -57 q19 -15 88 -29.5t84 -20.5q6 -38 6 -57q0 -4 -0.5 -13t-0.5 -13q-63 0 -190 8t-191 8q-76 0 -215 -7t-178 -8q0 43 4 78l131 28q1 0 12.5 2.5t15.5 3.5t14.5 4.5t15 6.5t11 8t9 11t2.5 14q0 16 -31 96.5t-72 177.5t-42 100l-450 2q-26 -58 -76.5 -195.5t-50.5 -162.5 q0 -22 14 -37.5t43.5 -24.5t48.5 -13.5t57 -8.5t41 -4q1 -19 1 -58q0 -9 -2 -27q-58 0 -174.5 10t-174.5 10q-8 0 -26.5 -4t-21.5 -4q-80 -14 -188 -14z" />
-<glyph unicode="&#xf032;" horiz-adv-x="1408" d="M555 15q74 -32 140 -32q376 0 376 335q0 114 -41 180q-27 44 -61.5 74t-67.5 46.5t-80.5 25t-84 10.5t-94.5 2q-73 0 -101 -10q0 -53 -0.5 -159t-0.5 -158q0 -8 -1 -67.5t-0.5 -96.5t4.5 -83.5t12 -66.5zM541 761q42 -7 109 -7q82 0 143 13t110 44.5t74.5 89.5t25.5 142 q0 70 -29 122.5t-79 82t-108 43.5t-124 14q-50 0 -130 -13q0 -50 4 -151t4 -152q0 -27 -0.5 -80t-0.5 -79q0 -46 1 -69zM0 -128l2 94q15 4 85 16t106 27q7 12 12.5 27t8.5 33.5t5.5 32.5t3 37.5t0.5 34v35.5v30q0 982 -22 1025q-4 8 -22 14.5t-44.5 11t-49.5 7t-48.5 4.5 t-30.5 3l-4 83q98 2 340 11.5t373 9.5q23 0 68.5 -0.5t67.5 -0.5q70 0 136.5 -13t128.5 -42t108 -71t74 -104.5t28 -137.5q0 -52 -16.5 -95.5t-39 -72t-64.5 -57.5t-73 -45t-84 -40q154 -35 256.5 -134t102.5 -248q0 -100 -35 -179.5t-93.5 -130.5t-138 -85.5t-163.5 -48.5 t-176 -14q-44 0 -132 3t-132 3q-106 0 -307 -11t-231 -12z" />
-<glyph unicode="&#xf033;" horiz-adv-x="1024" d="M0 -126l17 85q6 2 81.5 21.5t111.5 37.5q28 35 41 101q1 7 62 289t114 543.5t52 296.5v25q-24 13 -54.5 18.5t-69.5 8t-58 5.5l19 103q33 -2 120 -6.5t149.5 -7t120.5 -2.5q48 0 98.5 2.5t121 7t98.5 6.5q-5 -39 -19 -89q-30 -10 -101.5 -28.5t-108.5 -33.5 q-8 -19 -14 -42.5t-9 -40t-7.5 -45.5t-6.5 -42q-27 -148 -87.5 -419.5t-77.5 -355.5q-2 -9 -13 -58t-20 -90t-16 -83.5t-6 -57.5l1 -18q17 -4 185 -31q-3 -44 -16 -99q-11 0 -32.5 -1.5t-32.5 -1.5q-29 0 -87 10t-86 10q-138 2 -206 2q-51 0 -143 -9t-121 -11z" />
-<glyph unicode="&#xf034;" horiz-adv-x="1792" d="M1744 128q33 0 42 -18.5t-11 -44.5l-126 -162q-20 -26 -49 -26t-49 26l-126 162q-20 26 -11 44.5t42 18.5h80v1024h-80q-33 0 -42 18.5t11 44.5l126 162q20 26 49 26t49 -26l126 -162q20 -26 11 -44.5t-42 -18.5h-80v-1024h80zM81 1407l54 -27q12 -5 211 -5q44 0 132 2 t132 2q36 0 107.5 -0.5t107.5 -0.5h293q6 0 21 -0.5t20.5 0t16 3t17.5 9t15 17.5l42 1q4 0 14 -0.5t14 -0.5q2 -112 2 -336q0 -80 -5 -109q-39 -14 -68 -18q-25 44 -54 128q-3 9 -11 48t-14.5 73.5t-7.5 35.5q-6 8 -12 12.5t-15.5 6t-13 2.5t-18 0.5t-16.5 -0.5 q-17 0 -66.5 0.5t-74.5 0.5t-64 -2t-71 -6q-9 -81 -8 -136q0 -94 2 -388t2 -455q0 -16 -2.5 -71.5t0 -91.5t12.5 -69q40 -21 124 -42.5t120 -37.5q5 -40 5 -50q0 -14 -3 -29l-34 -1q-76 -2 -218 8t-207 10q-50 0 -151 -9t-152 -9q-3 51 -3 52v9q17 27 61.5 43t98.5 29t78 27 q19 42 19 383q0 101 -3 303t-3 303v117q0 2 0.5 15.5t0.5 25t-1 25.5t-3 24t-5 14q-11 12 -162 12q-33 0 -93 -12t-80 -26q-19 -13 -34 -72.5t-31.5 -111t-42.5 -53.5q-42 26 -56 44v383z" />
-<glyph unicode="&#xf035;" d="M81 1407l54 -27q12 -5 211 -5q44 0 132 2t132 2q70 0 246.5 1t304.5 0.5t247 -4.5q33 -1 56 31l42 1q4 0 14 -0.5t14 -0.5q2 -112 2 -336q0 -80 -5 -109q-39 -14 -68 -18q-25 44 -54 128q-3 9 -11 47.5t-15 73.5t-7 36q-10 13 -27 19q-5 2 -66 2q-30 0 -93 1t-103 1 t-94 -2t-96 -7q-9 -81 -8 -136l1 -152v52q0 -55 1 -154t1.5 -180t0.5 -153q0 -16 -2.5 -71.5t0 -91.5t12.5 -69q40 -21 124 -42.5t120 -37.5q5 -40 5 -50q0 -14 -3 -29l-34 -1q-76 -2 -218 8t-207 10q-50 0 -151 -9t-152 -9q-3 51 -3 52v9q17 27 61.5 43t98.5 29t78 27 q7 16 11.5 74t6 145.5t1.5 155t-0.5 153.5t-0.5 89q0 7 -2.5 21.5t-2.5 22.5q0 7 0.5 44t1 73t0 76.5t-3 67.5t-6.5 32q-11 12 -162 12q-41 0 -163 -13.5t-138 -24.5q-19 -12 -34 -71.5t-31.5 -111.5t-42.5 -54q-42 26 -56 44v383zM1310 125q12 0 42 -19.5t57.5 -41.5 t59.5 -49t36 -30q26 -21 26 -49t-26 -49q-4 -3 -36 -30t-59.5 -49t-57.5 -41.5t-42 -19.5q-13 0 -20.5 10.5t-10 28.5t-2.5 33.5t1.5 33t1.5 19.5h-1024q0 -2 1.5 -19.5t1.5 -33t-2.5 -33.5t-10 -28.5t-20.5 -10.5q-12 0 -42 19.5t-57.5 41
 .5t-59.5 49t-36 30q-26 21 -26 49 t26 49q4 3 36 30t59.5 49t57.5 41.5t42 19.5q13 0 20.5 -10.5t10 -28.5t2.5 -33.5t-1.5 -33t-1.5 -19.5h1024q0 2 -1.5 19.5t-1.5 33t2.5 33.5t10 28.5t20.5 10.5z" />
-<glyph unicode="&#xf036;" horiz-adv-x="1792" d="M1792 192v-128q0 -26 -19 -45t-45 -19h-1664q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1664q26 0 45 -19t19 -45zM1408 576v-128q0 -26 -19 -45t-45 -19h-1280q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1280q26 0 45 -19t19 -45zM1664 960v-128q0 -26 -19 -45 t-45 -19h-1536q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1536q26 0 45 -19t19 -45zM1280 1344v-128q0 -26 -19 -45t-45 -19h-1152q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1152q26 0 45 -19t19 -45z" />
-<glyph unicode="&#xf037;" horiz-adv-x="1792" d="M1792 192v-128q0 -26 -19 -45t-45 -19h-1664q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1664q26 0 45 -19t19 -45zM1408 576v-128q0 -26 -19 -45t-45 -19h-896q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h896q26 0 45 -19t19 -45zM1664 960v-128q0 -26 -19 -45t-45 -19 h-1408q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1408q26 0 45 -19t19 -45zM1280 1344v-128q0 -26 -19 -45t-45 -19h-640q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h640q26 0 45 -19t19 -45z" />
-<glyph unicode="&#xf038;" horiz-adv-x="1792" d="M1792 192v-128q0 -26 -19 -45t-45 -19h-1664q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1664q26 0 45 -19t19 -45zM1792 576v-128q0 -26 -19 -45t-45 -19h-1280q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1280q26 0 45 -19t19 -45zM1792 960v-128q0 -26 -19 -45 t-45 -19h-1536q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1536q26 0 45 -19t19 -45zM1792 1344v-128q0 -26 -19 -45t-45 -19h-1152q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1152q26 0 45 -19t19 -45z" />
-<glyph unicode="&#xf039;" horiz-adv-x="1792" d="M1792 192v-128q0 -26 -19 -45t-45 -19h-1664q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1664q26 0 45 -19t19 -45zM1792 576v-128q0 -26 -19 -45t-45 -19h-1664q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1664q26 0 45 -19t19 -45zM1792 960v-128q0 -26 -19 -45 t-45 -19h-1664q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1664q26 0 45 -19t19 -45zM1792 1344v-128q0 -26 -19 -45t-45 -19h-1664q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1664q26 0 45 -19t19 -45z" />
-<glyph unicode="&#xf03a;" horiz-adv-x="1792" d="M256 224v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-192q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h192q13 0 22.5 -9.5t9.5 -22.5zM256 608v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-192q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h192q13 0 22.5 -9.5 t9.5 -22.5zM256 992v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-192q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h192q13 0 22.5 -9.5t9.5 -22.5zM1792 224v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1344q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h1344 q13 0 22.5 -9.5t9.5 -22.5zM256 1376v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-192q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h192q13 0 22.5 -9.5t9.5 -22.5zM1792 608v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1344q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5 t22.5 9.5h1344q13 0 22.5 -9.5t9.5 -22.5zM1792 992v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1344q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h1344q13 0 22.5 -9.5t9.5 -22.5zM1792 1376v-192q0 -13 -9.5 -22.5t
 -22.5 -9.5h-1344q-13 0 -22.5 9.5t-9.5 22.5v192 q0 13 9.5 22.5t22.5 9.5h1344q13 0 22.5 -9.5t9.5 -22.5z" />
-<glyph unicode="&#xf03b;" horiz-adv-x="1792" d="M384 992v-576q0 -13 -9.5 -22.5t-22.5 -9.5q-14 0 -23 9l-288 288q-9 9 -9 23t9 23l288 288q9 9 23 9q13 0 22.5 -9.5t9.5 -22.5zM1792 224v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1728q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h1728q13 0 22.5 -9.5 t9.5 -22.5zM1792 608v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1088q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h1088q13 0 22.5 -9.5t9.5 -22.5zM1792 992v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1088q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h1088 q13 0 22.5 -9.5t9.5 -22.5zM1792 1376v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1728q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h1728q13 0 22.5 -9.5t9.5 -22.5z" />
-<glyph unicode="&#xf03c;" horiz-adv-x="1792" d="M352 704q0 -14 -9 -23l-288 -288q-9 -9 -23 -9q-13 0 -22.5 9.5t-9.5 22.5v576q0 13 9.5 22.5t22.5 9.5q14 0 23 -9l288 -288q9 -9 9 -23zM1792 224v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1728q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h1728q13 0 22.5 -9.5 t9.5 -22.5zM1792 608v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1088q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h1088q13 0 22.5 -9.5t9.5 -22.5zM1792 992v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1088q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h1088 q13 0 22.5 -9.5t9.5 -22.5zM1792 1376v-192q0 -13 -9.5 -22.5t-22.5 -9.5h-1728q-13 0 -22.5 9.5t-9.5 22.5v192q0 13 9.5 22.5t22.5 9.5h1728q13 0 22.5 -9.5t9.5 -22.5z" />
-<glyph unicode="&#xf03d;" horiz-adv-x="1792" d="M1792 1184v-1088q0 -42 -39 -59q-13 -5 -25 -5q-27 0 -45 19l-403 403v-166q0 -119 -84.5 -203.5t-203.5 -84.5h-704q-119 0 -203.5 84.5t-84.5 203.5v704q0 119 84.5 203.5t203.5 84.5h704q119 0 203.5 -84.5t84.5 -203.5v-165l403 402q18 19 45 19q12 0 25 -5 q39 -17 39 -59z" />
-<glyph unicode="&#xf03e;" horiz-adv-x="1920" d="M640 960q0 -80 -56 -136t-136 -56t-136 56t-56 136t56 136t136 56t136 -56t56 -136zM1664 576v-448h-1408v192l320 320l160 -160l512 512zM1760 1280h-1600q-13 0 -22.5 -9.5t-9.5 -22.5v-1216q0 -13 9.5 -22.5t22.5 -9.5h1600q13 0 22.5 9.5t9.5 22.5v1216 q0 13 -9.5 22.5t-22.5 9.5zM1920 1248v-1216q0 -66 -47 -113t-113 -47h-1600q-66 0 -113 47t-47 113v1216q0 66 47 113t113 47h1600q66 0 113 -47t47 -113z" />
-<glyph unicode="&#xf040;" d="M363 0l91 91l-235 235l-91 -91v-107h128v-128h107zM886 928q0 22 -22 22q-10 0 -17 -7l-542 -542q-7 -7 -7 -17q0 -22 22 -22q10 0 17 7l542 542q7 7 7 17zM832 1120l416 -416l-832 -832h-416v416zM1515 1024q0 -53 -37 -90l-166 -166l-416 416l166 165q36 38 90 38 q53 0 91 -38l235 -234q37 -39 37 -91z" />
-<glyph unicode="&#xf041;" horiz-adv-x="1024" d="M768 896q0 106 -75 181t-181 75t-181 -75t-75 -181t75 -181t181 -75t181 75t75 181zM1024 896q0 -109 -33 -179l-364 -774q-16 -33 -47.5 -52t-67.5 -19t-67.5 19t-46.5 52l-365 774q-33 70 -33 179q0 212 150 362t362 150t362 -150t150 -362z" />
-<glyph unicode="&#xf042;" d="M768 96v1088q-148 0 -273 -73t-198 -198t-73 -273t73 -273t198 -198t273 -73zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
-<glyph unicode="&#xf043;" horiz-adv-x="1024" d="M512 384q0 36 -20 69q-1 1 -15.5 22.5t-25.5 38t-25 44t-21 50.5q-4 16 -21 16t-21 -16q-7 -23 -21 -50.5t-25 -44t-25.5 -38t-15.5 -22.5q-20 -33 -20 -69q0 -53 37.5 -90.5t90.5 -37.5t90.5 37.5t37.5 90.5zM1024 512q0 -212 -150 -362t-362 -150t-362 150t-150 362 q0 145 81 275q6 9 62.5 90.5t101 151t99.5 178t83 201.5q9 30 34 47t51 17t51.5 -17t33.5 -47q28 -93 83 -201.5t99.5 -178t101 -151t62.5 -90.5q81 -127 81 -275z" />
-<glyph unicode="&#xf044;" horiz-adv-x="1792" d="M888 352l116 116l-152 152l-116 -116v-56h96v-96h56zM1328 1072q-16 16 -33 -1l-350 -350q-17 -17 -1 -33t33 1l350 350q17 17 1 33zM1408 478v-190q0 -119 -84.5 -203.5t-203.5 -84.5h-832q-119 0 -203.5 84.5t-84.5 203.5v832q0 119 84.5 203.5t203.5 84.5h832 q63 0 117 -25q15 -7 18 -23q3 -17 -9 -29l-49 -49q-14 -14 -32 -8q-23 6 -45 6h-832q-66 0 -113 -47t-47 -113v-832q0 -66 47 -113t113 -47h832q66 0 113 47t47 113v126q0 13 9 22l64 64q15 15 35 7t20 -29zM1312 1216l288 -288l-672 -672h-288v288zM1756 1084l-92 -92 l-288 288l92 92q28 28 68 28t68 -28l152 -152q28 -28 28 -68t-28 -68z" />
-<glyph unicode="&#xf045;" horiz-adv-x="1664" d="M1408 547v-259q0 -119 -84.5 -203.5t-203.5 -84.5h-832q-119 0 -203.5 84.5t-84.5 203.5v832q0 119 84.5 203.5t203.5 84.5h255v0q13 0 22.5 -9.5t9.5 -22.5q0 -27 -26 -32q-77 -26 -133 -60q-10 -4 -16 -4h-112q-66 0 -113 -47t-47 -113v-832q0 -66 47 -113t113 -47h832 q66 0 113 47t47 113v214q0 19 18 29q28 13 54 37q16 16 35 8q21 -9 21 -29zM1645 1043l-384 -384q-18 -19 -45 -19q-12 0 -25 5q-39 17 -39 59v192h-160q-323 0 -438 -131q-119 -137 -74 -473q3 -23 -20 -34q-8 -2 -12 -2q-16 0 -26 13q-10 14 -21 31t-39.5 68.5t-49.5 99.5 t-38.5 114t-17.5 122q0 49 3.5 91t14 90t28 88t47 81.5t68.5 74t94.5 61.5t124.5 48.5t159.5 30.5t196.5 11h160v192q0 42 39 59q13 5 25 5q26 0 45 -19l384 -384q19 -19 19 -45t-19 -45z" />
-<glyph unicode="&#xf046;" horiz-adv-x="1664" d="M1408 606v-318q0 -119 -84.5 -203.5t-203.5 -84.5h-832q-119 0 -203.5 84.5t-84.5 203.5v832q0 119 84.5 203.5t203.5 84.5h832q63 0 117 -25q15 -7 18 -23q3 -17 -9 -29l-49 -49q-10 -10 -23 -10q-3 0 -9 2q-23 6 -45 6h-832q-66 0 -113 -47t-47 -113v-832 q0 -66 47 -113t113 -47h832q66 0 113 47t47 113v254q0 13 9 22l64 64q10 10 23 10q6 0 12 -3q20 -8 20 -29zM1639 1095l-814 -814q-24 -24 -57 -24t-57 24l-430 430q-24 24 -24 57t24 57l110 110q24 24 57 24t57 -24l263 -263l647 647q24 24 57 24t57 -24l110 -110 q24 -24 24 -57t-24 -57z" />
-<glyph unicode="&#xf047;" horiz-adv-x="1792" d="M1792 640q0 -26 -19 -45l-256 -256q-19 -19 -45 -19t-45 19t-19 45v128h-384v-384h128q26 0 45 -19t19 -45t-19 -45l-256 -256q-19 -19 -45 -19t-45 19l-256 256q-19 19 -19 45t19 45t45 19h128v384h-384v-128q0 -26 -19 -45t-45 -19t-45 19l-256 256q-19 19 -19 45 t19 45l256 256q19 19 45 19t45 -19t19 -45v-128h384v384h-128q-26 0 -45 19t-19 45t19 45l256 256q19 19 45 19t45 -19l256 -256q19 -19 19 -45t-19 -45t-45 -19h-128v-384h384v128q0 26 19 45t45 19t45 -19l256 -256q19 -19 19 -45z" />
-<glyph unicode="&#xf048;" horiz-adv-x="1024" d="M979 1395q19 19 32 13t13 -32v-1472q0 -26 -13 -32t-32 13l-710 710q-9 9 -13 19v-678q0 -26 -19 -45t-45 -19h-128q-26 0 -45 19t-19 45v1408q0 26 19 45t45 19h128q26 0 45 -19t19 -45v-678q4 11 13 19z" />
-<glyph unicode="&#xf049;" horiz-adv-x="1792" d="M1747 1395q19 19 32 13t13 -32v-1472q0 -26 -13 -32t-32 13l-710 710q-9 9 -13 19v-710q0 -26 -13 -32t-32 13l-710 710q-9 9 -13 19v-678q0 -26 -19 -45t-45 -19h-128q-26 0 -45 19t-19 45v1408q0 26 19 45t45 19h128q26 0 45 -19t19 -45v-678q4 11 13 19l710 710 q19 19 32 13t13 -32v-710q4 11 13 19z" />
-<glyph unicode="&#xf04a;" horiz-adv-x="1664" d="M1619 1395q19 19 32 13t13 -32v-1472q0 -26 -13 -32t-32 13l-710 710q-8 9 -13 19v-710q0 -26 -13 -32t-32 13l-710 710q-19 19 -19 45t19 45l710 710q19 19 32 13t13 -32v-710q5 11 13 19z" />
-<glyph unicode="&#xf04b;" horiz-adv-x="1408" d="M1384 609l-1328 -738q-23 -13 -39.5 -3t-16.5 36v1472q0 26 16.5 36t39.5 -3l1328 -738q23 -13 23 -31t-23 -31z" />
-<glyph unicode="&#xf04c;" d="M1536 1344v-1408q0 -26 -19 -45t-45 -19h-512q-26 0 -45 19t-19 45v1408q0 26 19 45t45 19h512q26 0 45 -19t19 -45zM640 1344v-1408q0 -26 -19 -45t-45 -19h-512q-26 0 -45 19t-19 45v1408q0 26 19 45t45 19h512q26 0 45 -19t19 -45z" />
-<glyph unicode="&#xf04d;" d="M1536 1344v-1408q0 -26 -19 -45t-45 -19h-1408q-26 0 -45 19t-19 45v1408q0 26 19 45t45 19h1408q26 0 45 -19t19 -45z" />
-<glyph unicode="&#xf04e;" horiz-adv-x="1664" d="M45 -115q-19 -19 -32 -13t-13 32v1472q0 26 13 32t32 -13l710 -710q8 -8 13 -19v710q0 26 13 32t32 -13l710 -710q19 -19 19 -45t-19 -45l-710 -710q-19 -19 -32 -13t-13 32v710q-5 -10 -13 -19z" />
-<glyph unicode="&#xf050;" horiz-adv-x="1792" d="M45 -115q-19 -19 -32 -13t-13 32v1472q0 26 13 32t32 -13l710 -710q8 -8 13 -19v710q0 26 13 32t32 -13l710 -710q8 -8 13 -19v678q0 26 19 45t45 19h128q26 0 45 -19t19 -45v-1408q0 -26 -19 -45t-45 -19h-128q-26 0 -45 19t-19 45v678q-5 -10 -13 -19l-710 -710 q-19 -19 -32 -13t-13 32v710q-5 -10 -13 -19z" />
-<glyph unicode="&#xf051;" horiz-adv-x="1024" d="M45 -115q-19 -19 -32 -13t-13 32v1472q0 26 13 32t32 -13l710 -710q8 -8 13 -19v678q0 26 19 45t45 19h128q26 0 45 -19t19 -45v-1408q0 -26 -19 -45t-45 -19h-128q-26 0 -45 19t-19 45v678q-5 -10 -13 -19z" />
-<glyph unicode="&#xf052;" horiz-adv-x="1538" d="M14 557l710 710q19 19 45 19t45 -19l710 -710q19 -19 13 -32t-32 -13h-1472q-26 0 -32 13t13 32zM1473 0h-1408q-26 0 -45 19t-19 45v256q0 26 19 45t45 19h1408q26 0 45 -19t19 -45v-256q0 -26 -19 -45t-45 -19z" />
-<glyph unicode="&#xf053;" horiz-adv-x="1280" d="M1171 1235l-531 -531l531 -531q19 -19 19 -45t-19 -45l-166 -166q-19 -19 -45 -19t-45 19l-742 742q-19 19 -19 45t19 45l742 742q19 19 45 19t45 -19l166 -166q19 -19 19 -45t-19 -45z" />
-<glyph unicode="&#xf054;" horiz-adv-x="1280" d="M1107 659l-742 -742q-19 -19 -45 -19t-45 19l-166 166q-19 19 -19 45t19 45l531 531l-531 531q-19 19 -19 45t19 45l166 166q19 19 45 19t45 -19l742 -742q19 -19 19 -45t-19 -45z" />
-<glyph unicode="&#xf055;" d="M1216 576v128q0 26 -19 45t-45 19h-256v256q0 26 -19 45t-45 19h-128q-26 0 -45 -19t-19 -45v-256h-256q-26 0 -45 -19t-19 -45v-128q0 -26 19 -45t45 -19h256v-256q0 -26 19 -45t45 -19h128q26 0 45 19t19 45v256h256q26 0 45 19t19 45zM1536 640q0 -209 -103 -385.5 t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
-<glyph unicode="&#xf056;" d="M1216 576v128q0 26 -19 45t-45 19h-768q-26 0 -45 -19t-19 -45v-128q0 -26 19 -45t45 -19h768q26 0 45 19t19 45zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5 t103 -385.5z" />
-<glyph unicode="&#xf057;" d="M1149 414q0 26 -19 45l-181 181l181 181q19 19 19 45q0 27 -19 46l-90 90q-19 19 -46 19q-26 0 -45 -19l-181 -181l-181 181q-19 19 -45 19q-27 0 -46 -19l-90 -90q-19 -19 -19 -46q0 -26 19 -45l181 -181l-181 -181q-19 -19 -19 -45q0 -27 19 -46l90 -90q19 -19 46 -19 q26 0 45 19l181 181l181 -181q19 -19 45 -19q27 0 46 19l90 90q19 19 19 46zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
-<glyph unicode="&#xf058;" d="M1284 802q0 28 -18 46l-91 90q-19 19 -45 19t-45 -19l-408 -407l-226 226q-19 19 -45 19t-45 -19l-91 -90q-18 -18 -18 -46q0 -27 18 -45l362 -362q19 -19 45 -19q27 0 46 19l543 543q18 18 18 45zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103 t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
-<glyph unicode="&#xf059;" d="M896 160v192q0 14 -9 23t-23 9h-192q-14 0 -23 -9t-9 -23v-192q0 -14 9 -23t23 -9h192q14 0 23 9t9 23zM1152 832q0 88 -55.5 163t-138.5 116t-170 41q-243 0 -371 -213q-15 -24 8 -42l132 -100q7 -6 19 -6q16 0 25 12q53 68 86 92q34 24 86 24q48 0 85.5 -26t37.5 -59 q0 -38 -20 -61t-68 -45q-63 -28 -115.5 -86.5t-52.5 -125.5v-36q0 -14 9 -23t23 -9h192q14 0 23 9t9 23q0 19 21.5 49.5t54.5 49.5q32 18 49 28.5t46 35t44.5 48t28 60.5t12.5 81zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5 t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
-<glyph unicode="&#xf05a;" d="M1024 160v160q0 14 -9 23t-23 9h-96v512q0 14 -9 23t-23 9h-320q-14 0 -23 -9t-9 -23v-160q0 -14 9 -23t23 -9h96v-320h-96q-14 0 -23 -9t-9 -23v-160q0 -14 9 -23t23 -9h448q14 0 23 9t9 23zM896 1056v160q0 14 -9 23t-23 9h-192q-14 0 -23 -9t-9 -23v-160q0 -14 9 -23 t23 -9h192q14 0 23 9t9 23zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
-<glyph unicode="&#xf05b;" d="M1197 512h-109q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h109q-32 108 -112.5 188.5t-188.5 112.5v-109q0 -26 -19 -45t-45 -19h-128q-26 0 -45 19t-19 45v109q-108 -32 -188.5 -112.5t-112.5 -188.5h109q26 0 45 -19t19 -45v-128q0 -26 -19 -45t-45 -19h-109 q32 -108 112.5 -188.5t188.5 -112.5v109q0 26 19 45t45 19h128q26 0 45 -19t19 -45v-109q108 32 188.5 112.5t112.5 188.5zM1536 704v-128q0 -26 -19 -45t-45 -19h-143q-37 -161 -154.5 -278.5t-278.5 -154.5v-143q0 -26 -19 -45t-45 -19h-128q-26 0 -45 19t-19 45v143 q-161 37 -278.5 154.5t-154.5 278.5h-143q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h143q37 161 154.5 278.5t278.5 154.5v143q0 26 19 45t45 19h128q26 0 45 -19t19 -45v-143q161 -37 278.5 -154.5t154.5 -278.5h143q26 0 45 -19t19 -45z" />
-<glyph unicode="&#xf05c;" d="M1097 457l-146 -146q-10 -10 -23 -10t-23 10l-137 137l-137 -137q-10 -10 -23 -10t-23 10l-146 146q-10 10 -10 23t10 23l137 137l-137 137q-10 10 -10 23t10 23l146 146q10 10 23 10t23 -10l137 -137l137 137q10 10 23 10t23 -10l146 -146q10 -10 10 -23t-10 -23 l-137 -137l137 -137q10 -10 10 -23t-10 -23zM1312 640q0 148 -73 273t-198 198t-273 73t-273 -73t-198 -198t-73 -273t73 -273t198 -198t273 -73t273 73t198 198t73 273zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5 t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
-<glyph unicode="&#xf05d;" d="M1171 723l-422 -422q-19 -19 -45 -19t-45 19l-294 294q-19 19 -19 45t19 45l102 102q19 19 45 19t45 -19l147 -147l275 275q19 19 45 19t45 -19l102 -102q19 -19 19 -45t-19 -45zM1312 640q0 148 -73 273t-198 198t-273 73t-273 -73t-198 -198t-73 -273t73 -273t198 -198 t273 -73t273 73t198 198t73 273zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
-<glyph unicode="&#xf05e;" d="M1312 643q0 161 -87 295l-754 -753q137 -89 297 -89q111 0 211.5 43.5t173.5 116.5t116 174.5t43 212.5zM313 344l755 754q-135 91 -300 91q-148 0 -273 -73t-198 -199t-73 -274q0 -162 89 -299zM1536 643q0 -157 -61 -300t-163.5 -246t-245 -164t-298.5 -61t-298.5 61 t-245 164t-163.5 246t-61 300t61 299.5t163.5 245.5t245 164t298.5 61t298.5 -61t245 -164t163.5 -245.5t61 -299.5z" />
-<glyph unicode="&#xf060;" d="M1536 640v-128q0 -53 -32.5 -90.5t-84.5 -37.5h-704l293 -294q38 -36 38 -90t-38 -90l-75 -76q-37 -37 -90 -37q-52 0 -91 37l-651 652q-37 37 -37 90q0 52 37 91l651 650q38 38 91 38q52 0 90 -38l75 -74q38 -38 38 -91t-38 -91l-293 -293h704q52 0 84.5 -37.5 t32.5 -90.5z" />
-<glyph unicode="&#xf061;" d="M1472 576q0 -54 -37 -91l-651 -651q-39 -37 -91 -37q-51 0 -90 37l-75 75q-38 38 -38 91t38 91l293 293h-704q-52 0 -84.5 37.5t-32.5 90.5v128q0 53 32.5 90.5t84.5 37.5h704l-293 294q-38 36 -38 90t38 90l75 75q38 38 90 38q53 0 91 -38l651 -651q37 -35 37 -90z" />
-<glyph unicode="&#xf062;" horiz-adv-x="1664" d="M1611 565q0 -51 -37 -90l-75 -75q-38 -38 -91 -38q-54 0 -90 38l-294 293v-704q0 -52 -37.5 -84.5t-90.5 -32.5h-128q-53 0 -90.5 32.5t-37.5 84.5v704l-294 -293q-36 -38 -90 -38t-90 38l-75 75q-38 38 -38 90q0 53 38 91l651 651q35 37 90 37q54 0 91 -37l651 -651 q37 -39 37 -91z" />
-<glyph unicode="&#xf063;" horiz-adv-x="1664" d="M1611 704q0 -53 -37 -90l-651 -652q-39 -37 -91 -37q-53 0 -90 37l-651 652q-38 36 -38 90q0 53 38 91l74 75q39 37 91 37q53 0 90 -37l294 -294v704q0 52 38 90t90 38h128q52 0 90 -38t38 -90v-704l294 294q37 37 90 37q52 0 91 -37l75 -75q37 -39 37 -91z" />
-<glyph unicode="&#xf064;" horiz-adv-x="1792" d="M1792 896q0 -26 -19 -45l-512 -512q-19 -19 -45 -19t-45 19t-19 45v256h-224q-98 0 -175.5 -6t-154 -21.5t-133 -42.5t-105.5 -69.5t-80 -101t-48.5 -138.5t-17.5 -181q0 -55 5 -123q0 -6 2.5 -23.5t2.5 -26.5q0 -15 -8.5 -25t-23.5 -10q-16 0 -28 17q-7 9 -13 22 t-13.5 30t-10.5 24q-127 285 -127 451q0 199 53 333q162 403 875 403h224v256q0 26 19 45t45 19t45 -19l512 -512q19 -19 19 -45z" />
-<glyph unicode="&#xf065;" d="M755 480q0 -13 -10 -23l-332 -332l144 -144q19 -19 19 -45t-19 -45t-45 -19h-448q-26 0 -45 19t-19 45v448q0 26 19 45t45 19t45 -19l144 -144l332 332q10 10 23 10t23 -10l114 -114q10 -10 10 -23zM1536 1344v-448q0 -26 -19 -45t-45 -19t-45 19l-144 144l-332 -332 q-10 -10 -23 -10t-23 10l-114 114q-10 10 -10 23t10 23l332 332l-144 144q-19 19 -19 45t19 45t45 19h448q26 0 45 -19t19 -45z" />
-<glyph unicode="&#xf066;" d="M768 576v-448q0 -26 -19 -45t-45 -19t-45 19l-144 144l-332 -332q-10 -10 -23 -10t-23 10l-114 114q-10 10 -10 23t10 23l332 332l-144 144q-19 19 -19 45t19 45t45 19h448q26 0 45 -19t19 -45zM1523 1248q0 -13 -10 -23l-332 -332l144 -144q19 -19 19 -45t-19 -45 t-45 -19h-448q-26 0 -45 19t-19 45v448q0 26 19 45t45 19t45 -19l144 -144l332 332q10 10 23 10t23 -10l114 -114q10 -10 10 -23z" />
-<glyph unicode="&#xf067;" horiz-adv-x="1408" d="M1408 800v-192q0 -40 -28 -68t-68 -28h-416v-416q0 -40 -28 -68t-68 -28h-192q-40 0 -68 28t-28 68v416h-416q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h416v416q0 40 28 68t68 28h192q40 0 68 -28t28 -68v-416h416q40 0 68 -28t28 -68z" />
-<glyph unicode="&#xf068;" horiz-adv-x="1408" d="M1408 800v-192q0 -40 -28 -68t-68 -28h-1216q-40 0 -68 28t-28 68v192q0 40 28 68t68 28h1216q40 0 68 -28t28 -68z" />
-<glyph unicode="&#xf069;" horiz-adv-x="1664" d="M1482 486q46 -26 59.5 -77.5t-12.5 -97.5l-64 -110q-26 -46 -77.5 -59.5t-97.5 12.5l-266 153v-307q0 -52 -38 -90t-90 -38h-128q-52 0 -90 38t-38 90v307l-266 -153q-46 -26 -97.5 -12.5t-77.5 59.5l-64 110q-26 46 -12.5 97.5t59.5 77.5l266 154l-266 154 q-46 26 -59.5 77.5t12.5 97.5l64 110q26 46 77.5 59.5t97.5 -12.5l266 -153v307q0 52 38 90t90 38h128q52 0 90 -38t38 -90v-307l266 153q46 26 97.5 12.5t77.5 -59.5l64 -110q26 -46 12.5 -97.5t-59.5 -77.5l-266 -154z" />
-<glyph unicode="&#xf06a;" d="M768 1408q209 0 385.5 -103t279.5 -279.5t103 -385.5t-103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103zM896 161v190q0 14 -9 23.5t-22 9.5h-192q-13 0 -23 -10t-10 -23v-190q0 -13 10 -23t23 -10h192 q13 0 22 9.5t9 23.5zM894 505l18 621q0 12 -10 18q-10 8 -24 8h-220q-14 0 -24 -8q-10 -6 -10 -18l17 -621q0 -10 10 -17.5t24 -7.5h185q14 0 23.5 7.5t10.5 17.5z" />
-<glyph unicode="&#xf06b;" d="M928 180v56v468v192h-320v-192v-468v-56q0 -25 18 -38.5t46 -13.5h192q28 0 46 13.5t18 38.5zM472 1024h195l-126 161q-26 31 -69 31q-40 0 -68 -28t-28 -68t28 -68t68 -28zM1160 1120q0 40 -28 68t-68 28q-43 0 -69 -31l-125 -161h194q40 0 68 28t28 68zM1536 864v-320 q0 -14 -9 -23t-23 -9h-96v-416q0 -40 -28 -68t-68 -28h-1088q-40 0 -68 28t-28 68v416h-96q-14 0 -23 9t-9 23v320q0 14 9 23t23 9h440q-93 0 -158.5 65.5t-65.5 158.5t65.5 158.5t158.5 65.5q107 0 168 -77l128 -165l128 165q61 77 168 77q93 0 158.5 -65.5t65.5 -158.5 t-65.5 -158.5t-158.5 -65.5h440q14 0 23 -9t9 -23z" />
-<glyph unicode="&#xf06c;" horiz-adv-x="1792" d="M1280 832q0 26 -19 45t-45 19q-172 0 -318 -49.5t-259.5 -134t-235.5 -219.5q-19 -21 -19 -45q0 -26 19 -45t45 -19q24 0 45 19q27 24 74 71t67 66q137 124 268.5 176t313.5 52q26 0 45 19t19 45zM1792 1030q0 -95 -20 -193q-46 -224 -184.5 -383t-357.5 -268 q-214 -108 -438 -108q-148 0 -286 47q-15 5 -88 42t-96 37q-16 0 -39.5 -32t-45 -70t-52.5 -70t-60 -32q-30 0 -51 11t-31 24t-27 42q-2 4 -6 11t-5.5 10t-3 9.5t-1.5 13.5q0 35 31 73.5t68 65.5t68 56t31 48q0 4 -14 38t-16 44q-9 51 -9 104q0 115 43.5 220t119 184.5 t170.5 139t204 95.5q55 18 145 25.5t179.5 9t178.5 6t163.5 24t113.5 56.5l29.5 29.5t29.5 28t27 20t36.5 16t43.5 4.5q39 0 70.5 -46t47.5 -112t24 -124t8 -96z" />
-<glyph unicode="&#xf06d;" horiz-adv-x="1408" d="M1408 -160v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-1344q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h1344q13 0 22.5 -9.5t9.5 -22.5zM1152 896q0 -78 -24.5 -144t-64 -112.5t-87.5 -88t-96 -77.5t-87.5 -72t-64 -81.5t-24.5 -96.5q0 -96 67 -224l-4 1l1 -1 q-90 41 -160 83t-138.5 100t-113.5 122.5t-72.5 150.5t-27.5 184q0 78 24.5 144t64 112.5t87.5 88t96 77.5t87.5 72t64 81.5t24.5 96.5q0 94 -66 224l3 -1l-1 1q90 -41 160 -83t138.5 -100t113.5 -122.5t72.5 -150.5t27.5 -184z" />
-<glyph unicode="&#xf06e;" horiz-adv-x="1792" d="M1664 576q-152 236 -381 353q61 -104 61 -225q0 -185 -131.5 -316.5t-316.5 -131.5t-316.5 131.5t-131.5 316.5q0 121 61 225q-229 -117 -381 -353q133 -205 333.5 -326.5t434.5 -121.5t434.5 121.5t333.5 326.5zM944 960q0 20 -14 34t-34 14q-125 0 -214.5 -89.5 t-89.5 -214.5q0 -20 14 -34t34 -14t34 14t14 34q0 86 61 147t147 61q20 0 34 14t14 34zM1792 576q0 -34 -20 -69q-140 -230 -376.5 -368.5t-499.5 -138.5t-499.5 139t-376.5 368q-20 35 -20 69t20 69q140 229 376.5 368t499.5 139t499.5 -139t376.5 -368q20 -35 20 -69z" />
-<glyph unicode="&#xf070;" horiz-adv-x="1792" d="M555 201l78 141q-87 63 -136 159t-49 203q0 121 61 225q-229 -117 -381 -353q167 -258 427 -375zM944 960q0 20 -14 34t-34 14q-125 0 -214.5 -89.5t-89.5 -214.5q0 -20 14 -34t34 -14t34 14t14 34q0 86 61 147t147 61q20 0 34 14t14 34zM1307 1151q0 -7 -1 -9 q-105 -188 -315 -566t-316 -567l-49 -89q-10 -16 -28 -16q-12 0 -134 70q-16 10 -16 28q0 12 44 87q-143 65 -263.5 173t-208.5 245q-20 31 -20 69t20 69q153 235 380 371t496 136q89 0 180 -17l54 97q10 16 28 16q5 0 18 -6t31 -15.5t33 -18.5t31.5 -18.5t19.5 -11.5 q16 -10 16 -27zM1344 704q0 -139 -79 -253.5t-209 -164.5l280 502q8 -45 8 -84zM1792 576q0 -35 -20 -69q-39 -64 -109 -145q-150 -172 -347.5 -267t-419.5 -95l74 132q212 18 392.5 137t301.5 307q-115 179 -282 294l63 112q95 -64 182.5 -153t144.5 -184q20 -34 20 -69z " />
-<glyph unicode="&#xf071;" horiz-adv-x="1792" d="M1024 161v190q0 14 -9.5 23.5t-22.5 9.5h-192q-13 0 -22.5 -9.5t-9.5 -23.5v-190q0 -14 9.5 -23.5t22.5 -9.5h192q13 0 22.5 9.5t9.5 23.5zM1022 535l18 459q0 12 -10 19q-13 11 -24 11h-220q-11 0 -24 -11q-10 -7 -10 -21l17 -457q0 -10 10 -16.5t24 -6.5h185 q14 0 23.5 6.5t10.5 16.5zM1008 1469l768 -1408q35 -63 -2 -126q-17 -29 -46.5 -46t-63.5 -17h-1536q-34 0 -63.5 17t-46.5 46q-37 63 -2 126l768 1408q17 31 47 49t65 18t65 -18t47 -49z" />
-<glyph unicode="&#xf072;" horiz-adv-x="1408" d="M1376 1376q44 -52 12 -148t-108 -172l-161 -161l160 -696q5 -19 -12 -33l-128 -96q-7 -6 -19 -6q-4 0 -7 1q-15 3 -21 16l-279 508l-259 -259l53 -194q5 -17 -8 -31l-96 -96q-9 -9 -23 -9h-2q-15 2 -24 13l-189 252l-252 189q-11 7 -13 23q-1 13 9 25l96 97q9 9 23 9 q6 0 8 -1l194 -53l259 259l-508 279q-14 8 -17 24q-2 16 9 27l128 128q14 13 30 8l665 -159l160 160q76 76 172 108t148 -12z" />
-<glyph unicode="&#xf073;" horiz-adv-x="1664" d="M128 -128h288v288h-288v-288zM480 -128h320v288h-320v-288zM128 224h288v320h-288v-320zM480 224h320v320h-320v-320zM128 608h288v288h-288v-288zM864 -128h320v288h-320v-288zM480 608h320v288h-320v-288zM1248 -128h288v288h-288v-288zM864 224h320v320h-320v-320z M512 1088v288q0 13 -9.5 22.5t-22.5 9.5h-64q-13 0 -22.5 -9.5t-9.5 -22.5v-288q0 -13 9.5 -22.5t22.5 -9.5h64q13 0 22.5 9.5t9.5 22.5zM1248 224h288v320h-288v-320zM864 608h320v288h-320v-288zM1248 608h288v288h-288v-288zM1280 1088v288q0 13 -9.5 22.5t-22.5 9.5h-64 q-13 0 -22.5 -9.5t-9.5 -22.5v-288q0 -13 9.5 -22.5t22.5 -9.5h64q13 0 22.5 9.5t9.5 22.5zM1664 1152v-1280q0 -52 -38 -90t-90 -38h-1408q-52 0 -90 38t-38 90v1280q0 52 38 90t90 38h128v96q0 66 47 113t113 47h64q66 0 113 -47t47 -113v-96h384v96q0 66 47 113t113 47 h64q66 0 113 -47t47 -113v-96h128q52 0 90 -38t38 -90z" />
-<glyph unicode="&#xf074;" horiz-adv-x="1792" d="M666 1055q-60 -92 -137 -273q-22 45 -37 72.5t-40.5 63.5t-51 56.5t-63 35t-81.5 14.5h-224q-14 0 -23 9t-9 23v192q0 14 9 23t23 9h224q250 0 410 -225zM1792 256q0 -14 -9 -23l-320 -320q-9 -9 -23 -9q-13 0 -22.5 9.5t-9.5 22.5v192q-32 0 -85 -0.5t-81 -1t-73 1 t-71 5t-64 10.5t-63 18.5t-58 28.5t-59 40t-55 53.5t-56 69.5q59 93 136 273q22 -45 37 -72.5t40.5 -63.5t51 -56.5t63 -35t81.5 -14.5h256v192q0 14 9 23t23 9q12 0 24 -10l319 -319q9 -9 9 -23zM1792 1152q0 -14 -9 -23l-320 -320q-9 -9 -23 -9q-13 0 -22.5 9.5t-9.5 22.5 v192h-256q-48 0 -87 -15t-69 -45t-51 -61.5t-45 -77.5q-32 -62 -78 -171q-29 -66 -49.5 -111t-54 -105t-64 -100t-74 -83t-90 -68.5t-106.5 -42t-128 -16.5h-224q-14 0 -23 9t-9 23v192q0 14 9 23t23 9h224q48 0 87 15t69 45t51 61.5t45 77.5q32 62 78 171q29 66 49.5 111 t54 105t64 100t74 83t90 68.5t106.5 42t128 16.5h256v192q0 14 9 23t23 9q12 0 24 -10l319 -319q9 -9 9 -23z" />
-<glyph unicode="&#xf075;" horiz-adv-x="1792" d="M1792 640q0 -174 -120 -321.5t-326 -233t-450 -85.5q-70 0 -145 8q-198 -175 -460 -242q-49 -14 -114 -22q-17 -2 -30.5 9t-17.5 29v1q-3 4 -0.5 12t2 10t4.5 9.5l6 9t7 8.5t8 9q7 8 31 34.5t34.5 38t31 39.5t32.5 51t27 59t26 76q-157 89 -247.5 220t-90.5 281 q0 130 71 248.5t191 204.5t286 136.5t348 50.5q244 0 450 -85.5t326 -233t120 -321.5z" />
-<glyph unicode="&#xf076;" d="M1536 704v-128q0 -201 -98.5 -362t-274 -251.5t-395.5 -90.5t-395.5 90.5t-274 251.5t-98.5 362v128q0 26 19 45t45 19h384q26 0 45 -19t19 -45v-128q0 -52 23.5 -90t53.5 -57t71 -30t64 -13t44 -2t44 2t64 13t71 30t53.5 57t23.5 90v128q0 26 19 45t45 19h384 q26 0 45 -19t19 -45zM512 1344v-384q0 -26 -19 -45t-45 -19h-384q-26 0 -45 19t-19 45v384q0 26 19 45t45 19h384q26 0 45 -19t19 -45zM1536 1344v-384q0 -26 -19 -45t-45 -19h-384q-26 0 -45 19t-19 45v384q0 26 19 45t45 19h384q26 0 45 -19t19 -45z" />
-<glyph unicode="&#xf077;" horiz-adv-x="1792" d="M1683 205l-166 -165q-19 -19 -45 -19t-45 19l-531 531l-531 -531q-19 -19 -45 -19t-45 19l-166 165q-19 19 -19 45.5t19 45.5l742 741q19 19 45 19t45 -19l742 -741q19 -19 19 -45.5t-19 -45.5z" />
-<glyph unicode="&#xf078;" horiz-adv-x="1792" d="M1683 728l-742 -741q-19 -19 -45 -19t-45 19l-742 741q-19 19 -19 45.5t19 45.5l166 165q19 19 45 19t45 -19l531 -531l531 531q19 19 45 19t45 -19l166 -165q19 -19 19 -45.5t-19 -45.5z" />
-<glyph unicode="&#xf079;" horiz-adv-x="1920" d="M1280 32q0 -13 -9.5 -22.5t-22.5 -9.5h-960q-8 0 -13.5 2t-9 7t-5.5 8t-3 11.5t-1 11.5v13v11v160v416h-192q-26 0 -45 19t-19 45q0 24 15 41l320 384q19 22 49 22t49 -22l320 -384q15 -17 15 -41q0 -26 -19 -45t-45 -19h-192v-384h576q16 0 25 -11l160 -192q7 -11 7 -21 zM1920 448q0 -24 -15 -41l-320 -384q-20 -23 -49 -23t-49 23l-320 384q-15 17 -15 41q0 26 19 45t45 19h192v384h-576q-16 0 -25 12l-160 192q-7 9 -7 20q0 13 9.5 22.5t22.5 9.5h960q8 0 13.5 -2t9 -7t5.5 -8t3 -11.5t1 -11.5v-13v-11v-160v-416h192q26 0 45 -19t19 -45z " />
-<glyph unicode="&#xf07a;" horiz-adv-x="1664" d="M640 0q0 -53 -37.5 -90.5t-90.5 -37.5t-90.5 37.5t-37.5 90.5t37.5 90.5t90.5 37.5t90.5 -37.5t37.5 -90.5zM1536 0q0 -53 -37.5 -90.5t-90.5 -37.5t-90.5 37.5t-37.5 90.5t37.5 90.5t90.5 37.5t90.5 -37.5t37.5 -90.5zM1664 1088v-512q0 -24 -16 -42.5t-41 -21.5 l-1044 -122q1 -7 4.5 -21.5t6 -26.5t2.5 -22q0 -16 -24 -64h920q26 0 45 -19t19 -45t-19 -45t-45 -19h-1024q-26 0 -45 19t-19 45q0 14 11 39.5t29.5 59.5t20.5 38l-177 823h-204q-26 0 -45 19t-19 45t19 45t45 19h256q16 0 28.5 -6.5t20 -15.5t13 -24.5t7.5 -26.5 t5.5 -29.5t4.5 -25.5h1201q26 0 45 -19t19 -45z" />
-<glyph unicode="&#xf07b;" horiz-adv-x="1664" d="M1664 928v-704q0 -92 -66 -158t-158 -66h-1216q-92 0 -158 66t-66 158v960q0 92 66 158t158 66h320q92 0 158 -66t66 -158v-32h672q92 0 158 -66t66 -158z" />
-<glyph unicode="&#xf07c;" horiz-adv-x="1920" d="M1879 584q0 -31 -31 -66l-336 -396q-43 -51 -120.5 -86.5t-143.5 -35.5h-1088q-34 0 -60.5 13t-26.5 43q0 31 31 66l336 396q43 51 120.5 86.5t143.5 35.5h1088q34 0 60.5 -13t26.5 -43zM1536 928v-160h-832q-94 0 -197 -47.5t-164 -119.5l-337 -396l-5 -6q0 4 -0.5 12.5 t-0.5 12.5v960q0 92 66 158t158 66h320q92 0 158 -66t66 -158v-32h544q92 0 158 -66t66 -158z" />
-<glyph unicode="&#xf07d;" horiz-adv-x="768" d="M704 1216q0 -26 -19 -45t-45 -19h-128v-1024h128q26 0 45 -19t19 -45t-19 -45l-256 -256q-19 -19 -45 -19t-45 19l-256 256q-19 19 -19 45t19 45t45 19h128v1024h-128q-26 0 -45 19t-19 45t19 45l256 256q19 19 45 19t45 -19l256 -256q19 -19 19 -45z" />
-<glyph unicode="&#xf07e;" horiz-adv-x="1792" d="M1792 640q0 -26 -19 -45l-256 -256q-19 -19 -45 -19t-45 19t-19 45v128h-1024v-128q0 -26 -19 -45t-45 -19t-45 19l-256 256q-19 19 -19 45t19 45l256 256q19 19 45 19t45 -19t19 -45v-128h1024v128q0 26 19 45t45 19t45 -19l256 -256q19 -19 19 -45z" />
-<glyph unicode="&#xf080;" horiz-adv-x="2048" d="M640 640v-512h-256v512h256zM1024 1152v-1024h-256v1024h256zM2048 0v-128h-2048v1536h128v-1408h1920zM1408 896v-768h-256v768h256zM1792 1280v-1152h-256v1152h256z" />
-<glyph unicode="&#xf081;" d="M1280 926q-56 -25 -121 -34q68 40 93 117q-65 -38 -134 -51q-61 66 -153 66q-87 0 -148.5 -61.5t-61.5 -148.5q0 -29 5 -48q-129 7 -242 65t-192 155q-29 -50 -29 -106q0 -114 91 -175q-47 1 -100 26v-2q0 -75 50 -133.5t123 -72.5q-29 -8 -51 -8q-13 0 -39 4 q21 -63 74.5 -104t121.5 -42q-116 -90 -261 -90q-26 0 -50 3q148 -94 322 -94q112 0 210 35.5t168 95t120.5 137t75 162t24.5 168.5q0 18 -1 27q63 45 105 109zM1536 1120v-960q0 -119 -84.5 -203.5t-203.5 -84.5h-960q-119 0 -203.5 84.5t-84.5 203.5v960q0 119 84.5 203.5 t203.5 84.5h960q119 0 203.5 -84.5t84.5 -203.5z" />
-<glyph unicode="&#xf082;" d="M1536 160q0 -119 -84.5 -203.5t-203.5 -84.5h-192v608h203l30 224h-233v143q0 54 28 83t96 29l132 1v207q-96 9 -180 9q-136 0 -218 -80.5t-82 -225.5v-166h-224v-224h224v-608h-544q-119 0 -203.5 84.5t-84.5 203.5v960q0 119 84.5 203.5t203.5 84.5h960 q119 0 203.5 -84.5t84.5 -203.5v-960z" />
-<glyph unicode="&#xf083;" horiz-adv-x="1792" d="M928 704q0 14 -9 23t-23 9q-66 0 -113 -47t-47 -113q0 -14 9 -23t23 -9t23 9t9 23q0 40 28 68t68 28q14 0 23 9t9 23zM1152 574q0 -106 -75 -181t-181 -75t-181 75t-75 181t75 181t181 75t181 -75t75 -181zM128 0h1536v128h-1536v-128zM1280 574q0 159 -112.5 271.5 t-271.5 112.5t-271.5 -112.5t-112.5 -271.5t112.5 -271.5t271.5 -112.5t271.5 112.5t112.5 271.5zM256 1216h384v128h-384v-128zM128 1024h1536v118v138h-828l-64 -128h-644v-128zM1792 1280v-1280q0 -53 -37.5 -90.5t-90.5 -37.5h-1536q-53 0 -90.5 37.5t-37.5 90.5v1280 q0 53 37.5 90.5t90.5 37.5h1536q53 0 90.5 -37.5t37.5 -90.5z" />
-<glyph unicode="&#xf084;" horiz-adv-x="1792" d="M832 1024q0 80 -56 136t-136 56t-136 -56t-56 -136q0 -42 19 -83q-41 19 -83 19q-80 0 -136 -56t-56 -136t56 -136t136 -56t136 56t56 136q0 42 -19 83q41 -19 83 -19q80 0 136 56t56 136zM1683 320q0 -17 -49 -66t-66 -49q-9 0 -28.5 16t-36.5 33t-38.5 40t-24.5 26 l-96 -96l220 -220q28 -28 28 -68q0 -42 -39 -81t-81 -39q-40 0 -68 28l-671 671q-176 -131 -365 -131q-163 0 -265.5 102.5t-102.5 265.5q0 160 95 313t248 248t313 95q163 0 265.5 -102.5t102.5 -265.5q0 -189 -131 -365l355 -355l96 96q-3 3 -26 24.5t-40 38.5t-33 36.5 t-16 28.5q0 17 49 66t66 49q13 0 23 -10q6 -6 46 -44.5t82 -79.5t86.5 -86t73 -78t28.5 -41z" />
-<glyph unicode="&#xf085;" horiz-adv-x="1920" d="M896 640q0 106 -75 181t-181 75t-181 -75t-75 -181t75 -181t181 -75t181 75t75 181zM1664 128q0 52 -38 90t-90 38t-90 -38t-38 -90q0 -53 37.5 -90.5t90.5 -37.5t90.5 37.5t37.5 90.5zM1664 1152q0 52 -38 90t-90 38t-90 -38t-38 -90q0 -53 37.5 -90.5t90.5 -37.5 t90.5 37.5t37.5 90.5zM1280 731v-185q0 -10 -7 -19.5t-16 -10.5l-155 -24q-11 -35 -32 -76q34 -48 90 -115q7 -10 7 -20q0 -12 -7 -19q-23 -30 -82.5 -89.5t-78.5 -59.5q-11 0 -21 7l-115 90q-37 -19 -77 -31q-11 -108 -23 -155q-7 -24 -30 -24h-186q-11 0 -20 7.5t-10 17.5 l-23 153q-34 10 -75 31l-118 -89q-7 -7 -20 -7q-11 0 -21 8q-144 133 -144 160q0 9 7 19q10 14 41 53t47 61q-23 44 -35 82l-152 24q-10 1 -17 9.5t-7 19.5v185q0 10 7 19.5t16 10.5l155 24q11 35 32 76q-34 48 -90 115q-7 11 -7 20q0 12 7 20q22 30 82 89t79 59q11 0 21 -7 l115 -90q34 18 77 32q11 108 23 154q7 24 30 24h186q11 0 20 -7.5t10 -17.5l23 -153q34 -10 75 -31l118 89q8 7 20 7q11 0 21 -8q144 -133 144 -160q0 -9 -7 -19q-12 -16 -42 -54t-45 -60q23 -48 34 -82l152 
 -23q10 -2 17 -10.5t7 -19.5zM1920 198v-140q0 -16 -149 -31 q-12 -27 -30 -52q51 -113 51 -138q0 -4 -4 -7q-122 -71 -124 -71q-8 0 -46 47t-52 68q-20 -2 -30 -2t-30 2q-14 -21 -52 -68t-46 -47q-2 0 -124 71q-4 3 -4 7q0 25 51 138q-18 25 -30 52q-149 15 -149 31v140q0 16 149 31q13 29 30 52q-51 113 -51 138q0 4 4 7q4 2 35 20 t59 34t30 16q8 0 46 -46.5t52 -67.5q20 2 30 2t30 -2q51 71 92 112l6 2q4 0 124 -70q4 -3 4 -7q0 -25 -51 -138q17 -23 30 -52q149 -15 149 -31zM1920 1222v-140q0 -16 -149 -31q-12 -27 -30 -52q51 -113 51 -138q0 -4 -4 -7q-122 -71 -124 -71q-8 0 -46 47t-52 68 q-20 -2 -30 -2t-30 2q-14 -21 -52 -68t-46 -47q-2 0 -124 71q-4 3 -4 7q0 25 51 138q-18 25 -30 52q-149 15 -149 31v140q0 16 149 31q13 29 30 52q-51 113 -51 138q0 4 4 7q4 2 35 20t59 34t30 16q8 0 46 -46.5t52 -67.5q20 2 30 2t30 -2q51 71 92 112l6 2q4 0 124 -70 q4 -3 4 -7q0 -25 -51 -138q17 -23 30 -52q149 -15 149 -31z" />
-<glyph unicode="&#xf086;" horiz-adv-x="1792" d="M1408 768q0 -139 -94 -257t-256.5 -186.5t-353.5 -68.5q-86 0 -176 16q-124 -88 -278 -128q-36 -9 -86 -16h-3q-11 0 -20.5 8t-11.5 21q-1 3 -1 6.5t0.5 6.5t2 6l2.5 5t3.5 5.5t4 5t4.5 5t4 4.5q5 6 23 25t26 29.5t22.5 29t25 38.5t20.5 44q-124 72 -195 177t-71 224 q0 139 94 257t256.5 186.5t353.5 68.5t353.5 -68.5t256.5 -186.5t94 -257zM1792 512q0 -120 -71 -224.5t-195 -176.5q10 -24 20.5 -44t25 -38.5t22.5 -29t26 -29.5t23 -25q1 -1 4 -4.5t4.5 -5t4 -5t3.5 -5.5l2.5 -5t2 -6t0.5 -6.5t-1 -6.5q-3 -14 -13 -22t-22 -7 q-50 7 -86 16q-154 40 -278 128q-90 -16 -176 -16q-271 0 -472 132q58 -4 88 -4q161 0 309 45t264 129q125 92 192 212t67 254q0 77 -23 152q129 -71 204 -178t75 -230z" />
-<glyph unicode="&#xf087;" d="M256 192q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1408 768q0 51 -39 89.5t-89 38.5h-352q0 58 48 159.5t48 160.5q0 98 -32 145t-128 47q-26 -26 -38 -85t-30.5 -125.5t-59.5 -109.5q-22 -23 -77 -91q-4 -5 -23 -30t-31.5 -41t-34.5 -42.5 t-40 -44t-38.5 -35.5t-40 -27t-35.5 -9h-32v-640h32q13 0 31.5 -3t33 -6.5t38 -11t35 -11.5t35.5 -12.5t29 -10.5q211 -73 342 -73h121q192 0 192 167q0 26 -5 56q30 16 47.5 52.5t17.5 73.5t-18 69q53 50 53 119q0 25 -10 55.5t-25 47.5q32 1 53.5 47t21.5 81zM1536 769 q0 -89 -49 -163q9 -33 9 -69q0 -77 -38 -144q3 -21 3 -43q0 -101 -60 -178q1 -139 -85 -219.5t-227 -80.5h-36h-93q-96 0 -189.5 22.5t-216.5 65.5q-116 40 -138 40h-288q-53 0 -90.5 37.5t-37.5 90.5v640q0 53 37.5 90.5t90.5 37.5h274q36 24 137 155q58 75 107 128 q24 25 35.5 85.5t30.5 126.5t62 108q39 37 90 37q84 0 151 -32.5t102 -101.5t35 -186q0 -93 -48 -192h176q104 0 180 -76t76 -179z" />
-<glyph unicode="&#xf088;" d="M256 1088q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1408 512q0 35 -21.5 81t-53.5 47q15 17 25 47.5t10 55.5q0 69 -53 119q18 32 18 69t-17.5 73.5t-47.5 52.5q5 30 5 56q0 85 -49 126t-136 41h-128q-131 0 -342 -73q-5 -2 -29 -10.5 t-35.5 -12.5t-35 -11.5t-38 -11t-33 -6.5t-31.5 -3h-32v-640h32q16 0 35.5 -9t40 -27t38.5 -35.5t40 -44t34.5 -42.5t31.5 -41t23 -30q55 -68 77 -91q41 -43 59.5 -109.5t30.5 -125.5t38 -85q96 0 128 47t32 145q0 59 -48 160.5t-48 159.5h352q50 0 89 38.5t39 89.5z M1536 511q0 -103 -76 -179t-180 -76h-176q48 -99 48 -192q0 -118 -35 -186q-35 -69 -102 -101.5t-151 -32.5q-51 0 -90 37q-34 33 -54 82t-25.5 90.5t-17.5 84.5t-31 64q-48 50 -107 127q-101 131 -137 155h-274q-53 0 -90.5 37.5t-37.5 90.5v640q0 53 37.5 90.5t90.5 37.5 h288q22 0 138 40q128 44 223 66t200 22h112q140 0 226.5 -79t85.5 -216v-5q60 -77 60 -178q0 -22 -3 -43q38 -67 38 -144q0 -36 -9 -69q49 -74 49 -163z" />
-<glyph unicode="&#xf089;" horiz-adv-x="896" d="M832 1504v-1339l-449 -236q-22 -12 -40 -12q-21 0 -31.5 14.5t-10.5 35.5q0 6 2 20l86 500l-364 354q-25 27 -25 48q0 37 56 46l502 73l225 455q19 41 49 41z" />
-<glyph unicode="&#xf08a;" horiz-adv-x="1792" d="M1664 940q0 81 -21.5 143t-55 98.5t-81.5 59.5t-94 31t-98 8t-112 -25.5t-110.5 -64t-86.5 -72t-60 -61.5q-18 -22 -49 -22t-49 22q-24 28 -60 61.5t-86.5 72t-110.5 64t-112 25.5t-98 -8t-94 -31t-81.5 -59.5t-55 -98.5t-21.5 -143q0 -168 187 -355l581 -560l580 559 q188 188 188 356zM1792 940q0 -221 -229 -450l-623 -600q-18 -18 -44 -18t-44 18l-624 602q-10 8 -27.5 26t-55.5 65.5t-68 97.5t-53.5 121t-23.5 138q0 220 127 344t351 124q62 0 126.5 -21.5t120 -58t95.5 -68.5t76 -68q36 36 76 68t95.5 68.5t120 58t126.5 21.5 q224 0 351 -124t127 -344z" />
-<glyph unicode="&#xf08b;" horiz-adv-x="1664" d="M640 96q0 -4 1 -20t0.5 -26.5t-3 -23.5t-10 -19.5t-20.5 -6.5h-320q-119 0 -203.5 84.5t-84.5 203.5v704q0 119 84.5 203.5t203.5 84.5h320q13 0 22.5 -9.5t9.5 -22.5q0 -4 1 -20t0.5 -26.5t-3 -23.5t-10 -19.5t-20.5 -6.5h-320q-66 0 -113 -47t-47 -113v-704 q0 -66 47 -113t113 -47h288h11h13t11.5 -1t11.5 -3t8 -5.5t7 -9t2 -13.5zM1568 640q0 -26 -19 -45l-544 -544q-19 -19 -45 -19t-45 19t-19 45v288h-448q-26 0 -45 19t-19 45v384q0 26 19 45t45 19h448v288q0 26 19 45t45 19t45 -19l544 -544q19 -19 19 -45z" />
-<glyph unicode="&#xf08c;" d="M237 122h231v694h-231v-694zM483 1030q-1 52 -36 86t-93 34t-94.5 -34t-36.5 -86q0 -51 35.5 -85.5t92.5 -34.5h1q59 0 95 34.5t36 85.5zM1068 122h231v398q0 154 -73 233t-193 79q-136 0 -209 -117h2v101h-231q3 -66 0 -694h231v388q0 38 7 56q15 35 45 59.5t74 24.5 q116 0 116 -157v-371zM1536 1120v-960q0 -119 -84.5 -203.5t-203.5 -84.5h-960q-119 0 -203.5 84.5t-84.5 203.5v960q0 119 84.5 203.5t203.5 84.5h960q119 0 203.5 -84.5t84.5 -203.5z" />
-<glyph unicode="&#xf08d;" horiz-adv-x="1152" d="M480 672v448q0 14 -9 23t-23 9t-23 -9t-9 -23v-448q0 -14 9 -23t23 -9t23 9t9 23zM1152 320q0 -26 -19 -45t-45 -19h-429l-51 -483q-2 -12 -10.5 -20.5t-20.5 -8.5h-1q-27 0 -32 27l-76 485h-404q-26 0 -45 19t-19 45q0 123 78.5 221.5t177.5 98.5v512q-52 0 -90 38 t-38 90t38 90t90 38h640q52 0 90 -38t38 -90t-38 -90t-90 -38v-512q99 0 177.5 -98.5t78.5 -221.5z" />
-<glyph unicode="&#xf08e;" horiz-adv-x="1792" d="M1408 608v-320q0 -119 -84.5 -203.5t-203.5 -84.5h-832q-119 0 -203.5 84.5t-84.5 203.5v832q0 119 84.5 203.5t203.5 84.5h704q14 0 23 -9t9 -23v-64q0 -14 -9 -23t-23 -9h-704q-66 0 -113 -47t-47 -113v-832q0 -66 47 -113t113 -47h832q66 0 113 47t47 113v320 q0 14 9 23t23 9h64q14 0 23 -9t9 -23zM1792 1472v-512q0 -26 -19 -45t-45 -19t-45 19l-176 176l-652 -652q-10 -10 -23 -10t-23 10l-114 114q-10 10 -10 23t10 23l652 652l-176 176q-19 19 -19 45t19 45t45 19h512q26 0 45 -19t19 -45z" />
-<glyph unicode="&#xf090;" d="M1184 640q0 -26 -19 -45l-544 -544q-19 -19 -45 -19t-45 19t-19 45v288h-448q-26 0 -45 19t-19 45v384q0 26 19 45t45 19h448v288q0 26 19 45t45 19t45 -19l544 -544q19 -19 19 -45zM1536 992v-704q0 -119 -84.5 -203.5t-203.5 -84.5h-320q-13 0 -22.5 9.5t-9.5 22.5 q0 4 -1 20t-0.5 26.5t3 23.5t10 19.5t20.5 6.5h320q66 0 113 47t47 113v704q0 66 -47 113t-113 47h-288h-11h-13t-11.5 1t-11.5 3t-8 5.5t-7 9t-2 13.5q0 4 -1 20t-0.5 26.5t3 23.5t10 19.5t20.5 6.5h320q119 0 203.5 -84.5t84.5 -203.5z" />
-<glyph unicode="&#xf091;" horiz-adv-x="1664" d="M458 653q-74 162 -74 371h-256v-96q0 -78 94.5 -162t235.5 -113zM1536 928v96h-256q0 -209 -74 -371q141 29 235.5 113t94.5 162zM1664 1056v-128q0 -71 -41.5 -143t-112 -130t-173 -97.5t-215.5 -44.5q-42 -54 -95 -95q-38 -34 -52.5 -72.5t-14.5 -89.5q0 -54 30.5 -91 t97.5 -37q75 0 133.5 -45.5t58.5 -114.5v-64q0 -14 -9 -23t-23 -9h-832q-14 0 -23 9t-9 23v64q0 69 58.5 114.5t133.5 45.5q67 0 97.5 37t30.5 91q0 51 -14.5 89.5t-52.5 72.5q-53 41 -95 95q-113 5 -215.5 44.5t-173 97.5t-112 130t-41.5 143v128q0 40 28 68t68 28h288v96 q0 66 47 113t113 47h576q66 0 113 -47t47 -113v-96h288q40 0 68 -28t28 -68z" />
-<glyph unicode="&#xf092;" d="M394 184q-8 -9 -20 3q-13 11 -4 19q8 9 20 -3q12 -11 4 -19zM352 245q9 -12 0 -19q-8 -6 -17 7t0 18q9 7 17 -6zM291 305q-5 -7 -13 -2q-10 5 -7 12q3 5 13 2q10 -5 7 -12zM322 271q-6 -7 -16 3q-9 11 -2 16q6 6 16 -3q9 -11 2 -16zM451 159q-4 -12 -19 -6q-17 4 -13 15 t19 7q16 -5 13 -16zM514 154q0 -11 -16 -11q-17 -2 -17 11q0 11 16 11q17 2 17 -11zM572 164q2 -10 -14 -14t-18 8t14 15q16 2 18 -9zM1536 1120v-960q0 -119 -84.5 -203.5t-203.5 -84.5h-224q-16 0 -24.5 1t-19.5 5t-16 14.5t-5 27.5v239q0 97 -52 142q57 6 102.5 18t94 39 t81 66.5t53 105t20.5 150.5q0 121 -79 206q37 91 -8 204q-28 9 -81 -11t-92 -44l-38 -24q-93 26 -192 26t-192 -26q-16 11 -42.5 27t-83.5 38.5t-86 13.5q-44 -113 -7 -204q-79 -85 -79 -206q0 -85 20.5 -150t52.5 -105t80.5 -67t94 -39t102.5 -18q-40 -36 -49 -103 q-21 -10 -45 -15t-57 -5t-65.5 21.5t-55.5 62.5q-19 32 -48.5 52t-49.5 24l-20 3q-21 0 -29 -4.5t-5 -11.5t9 -14t13 -12l7 -5q22 -10 43.5 -38t31.5 -51l10 -23q13 -38 44 -61.5t67 -30t69.5 -7t55.5 3.5l23 4q0 -38 0.5 -103t0.5 
 -68q0 -22 -11 -33.5t-22 -13t-33 -1.5 h-224q-119 0 -203.5 84.5t-84.5 203.5v960q0 119 84.5 203.5t203.5 84.5h960q119 0 203.5 -84.5t84.5 -203.5z" />
-<glyph unicode="&#xf093;" horiz-adv-x="1664" d="M1280 64q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1536 64q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1664 288v-320q0 -40 -28 -68t-68 -28h-1472q-40 0 -68 28t-28 68v320q0 40 28 68t68 28h427q21 -56 70.5 -92 t110.5 -36h256q61 0 110.5 36t70.5 92h427q40 0 68 -28t28 -68zM1339 936q-17 -40 -59 -40h-256v-448q0 -26 -19 -45t-45 -19h-256q-26 0 -45 19t-19 45v448h-256q-42 0 -59 40q-17 39 14 69l448 448q18 19 45 19t45 -19l448 -448q31 -30 14 -69z" />
-<glyph unicode="&#xf094;" d="M1407 710q0 44 -7 113.5t-18 96.5q-12 30 -17 44t-9 36.5t-4 48.5q0 23 5 68.5t5 67.5q0 37 -10 55q-4 1 -13 1q-19 0 -58 -4.5t-59 -4.5q-60 0 -176 24t-175 24q-43 0 -94.5 -11.5t-85 -23.5t-89.5 -34q-137 -54 -202 -103q-96 -73 -159.5 -189.5t-88 -236t-24.5 -248.5 q0 -40 12.5 -120t12.5 -121q0 -23 -11 -66.5t-11 -65.5t12 -36.5t34 -14.5q24 0 72.5 11t73.5 11q57 0 169.5 -15.5t169.5 -15.5q181 0 284 36q129 45 235.5 152.5t166 245.5t59.5 275zM1535 712q0 -165 -70 -327.5t-196 -288t-281 -180.5q-124 -44 -326 -44 q-57 0 -170 14.5t-169 14.5q-24 0 -72.5 -14.5t-73.5 -14.5q-73 0 -123.5 55.5t-50.5 128.5q0 24 11 68t11 67q0 40 -12.5 120.5t-12.5 121.5q0 111 18 217.5t54.5 209.5t100.5 194t150 156q78 59 232 120q194 78 316 78q60 0 175.5 -24t173.5 -24q19 0 57 5t58 5 q81 0 118 -50.5t37 -134.5q0 -23 -5 -68t-5 -68q0 -10 1 -18.5t3 -17t4 -13.5t6.5 -16t6.5 -17q16 -40 25 -118.5t9 -136.5z" />
-<glyph unicode="&#xf095;" horiz-adv-x="1408" d="M1408 296q0 -27 -10 -70.5t-21 -68.5q-21 -50 -122 -106q-94 -51 -186 -51q-27 0 -52.5 3.5t-57.5 12.5t-47.5 14.5t-55.5 20.5t-49 18q-98 35 -175 83q-128 79 -264.5 215.5t-215.5 264.5q-48 77 -83 175q-3 9 -18 49t-20.5 55.5t-14.5 47.5t-12.5 57.5t-3.5 52.5 q0 92 51 186q56 101 106 122q25 11 68.5 21t70.5 10q14 0 21 -3q18 -6 53 -76q11 -19 30 -54t35 -63.5t31 -53.5q3 -4 17.5 -25t21.5 -35.5t7 -28.5q0 -20 -28.5 -50t-62 -55t-62 -53t-28.5 -46q0 -9 5 -22.5t8.5 -20.5t14 -24t11.5 -19q76 -137 174 -235t235 -174 q2 -1 19 -11.5t24 -14t20.5 -8.5t22.5 -5q18 0 46 28.5t53 62t55 62t50 28.5q14 0 28.5 -7t35.5 -21.5t25 -17.5q25 -15 53.5 -31t63.5 -35t54 -30q70 -35 76 -53q3 -7 3 -21z" />
-<glyph unicode="&#xf096;" horiz-adv-x="1408" d="M1120 1280h-832q-66 0 -113 -47t-47 -113v-832q0 -66 47 -113t113 -47h832q66 0 113 47t47 113v832q0 66 -47 113t-113 47zM1408 1120v-832q0 -119 -84.5 -203.5t-203.5 -84.5h-832q-119 0 -203.5 84.5t-84.5 203.5v832q0 119 84.5 203.5t203.5 84.5h832 q119 0 203.5 -84.5t84.5 -203.5z" />
-<glyph unicode="&#xf097;" horiz-adv-x="1280" d="M1152 1280h-1024v-1242l423 406l89 85l89 -85l423 -406v1242zM1164 1408q23 0 44 -9q33 -13 52.5 -41t19.5 -62v-1289q0 -34 -19.5 -62t-52.5 -41q-19 -8 -44 -8q-48 0 -83 32l-441 424l-441 -424q-36 -33 -83 -33q-23 0 -44 9q-33 13 -52.5 41t-19.5 62v1289 q0 34 19.5 62t52.5 41q21 9 44 9h1048z" />
-<glyph unicode="&#xf098;" d="M1280 343q0 11 -2 16q-3 8 -38.5 29.5t-88.5 49.5l-53 29q-5 3 -19 13t-25 15t-21 5q-18 0 -47 -32.5t-57 -65.5t-44 -33q-7 0 -16.5 3.5t-15.5 6.5t-17 9.5t-14 8.5q-99 55 -170.5 126.5t-126.5 170.5q-2 3 -8.5 14t-9.5 17t-6.5 15.5t-3.5 16.5q0 13 20.5 33.5t45 38.5 t45 39.5t20.5 36.5q0 10 -5 21t-15 25t-13 19q-3 6 -15 28.5t-25 45.5t-26.5 47.5t-25 40.5t-16.5 18t-16 2q-48 0 -101 -22q-46 -21 -80 -94.5t-34 -130.5q0 -16 2.5 -34t5 -30.5t9 -33t10 -29.5t12.5 -33t11 -30q60 -164 216.5 -320.5t320.5 -216.5q6 -2 30 -11t33 -12.5 t29.5 -10t33 -9t30.5 -5t34 -2.5q57 0 130.5 34t94.5 80q22 53 22 101zM1536 1120v-960q0 -119 -84.5 -203.5t-203.5 -84.5h-960q-119 0 -203.5 84.5t-84.5 203.5v960q0 119 84.5 203.5t203.5 84.5h960q119 0 203.5 -84.5t84.5 -203.5z" />
-<glyph unicode="&#xf099;" horiz-adv-x="1664" d="M1620 1128q-67 -98 -162 -167q1 -14 1 -42q0 -130 -38 -259.5t-115.5 -248.5t-184.5 -210.5t-258 -146t-323 -54.5q-271 0 -496 145q35 -4 78 -4q225 0 401 138q-105 2 -188 64.5t-114 159.5q33 -5 61 -5q43 0 85 11q-112 23 -185.5 111.5t-73.5 205.5v4q68 -38 146 -41 q-66 44 -105 115t-39 154q0 88 44 163q121 -149 294.5 -238.5t371.5 -99.5q-8 38 -8 74q0 134 94.5 228.5t228.5 94.5q140 0 236 -102q109 21 205 78q-37 -115 -142 -178q93 10 186 50z" />
-<glyph unicode="&#xf09a;" horiz-adv-x="1024" d="M959 1524v-264h-157q-86 0 -116 -36t-30 -108v-189h293l-39 -296h-254v-759h-306v759h-255v296h255v218q0 186 104 288.5t277 102.5q147 0 228 -12z" />
-<glyph unicode="&#xf09b;" d="M1536 640q0 -251 -146.5 -451.5t-378.5 -277.5q-27 -5 -39.5 7t-12.5 30v211q0 97 -52 142q57 6 102.5 18t94 39t81 66.5t53 105t20.5 150.5q0 121 -79 206q37 91 -8 204q-28 9 -81 -11t-92 -44l-38 -24q-93 26 -192 26t-192 -26q-16 11 -42.5 27t-83.5 38.5t-86 13.5 q-44 -113 -7 -204q-79 -85 -79 -206q0 -85 20.5 -150t52.5 -105t80.5 -67t94 -39t102.5 -18q-40 -36 -49 -103q-21 -10 -45 -15t-57 -5t-65.5 21.5t-55.5 62.5q-19 32 -48.5 52t-49.5 24l-20 3q-21 0 -29 -4.5t-5 -11.5t9 -14t13 -12l7 -5q22 -10 43.5 -38t31.5 -51l10 -23 q13 -38 44 -61.5t67 -30t69.5 -7t55.5 3.5l23 4q0 -38 0.5 -89t0.5 -54q0 -18 -13 -30t-40 -7q-232 77 -378.5 277.5t-146.5 451.5q0 209 103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
-<glyph unicode="&#xf09c;" horiz-adv-x="1664" d="M1664 960v-256q0 -26 -19 -45t-45 -19h-64q-26 0 -45 19t-19 45v256q0 106 -75 181t-181 75t-181 -75t-75 -181v-192h96q40 0 68 -28t28 -68v-576q0 -40 -28 -68t-68 -28h-960q-40 0 -68 28t-28 68v576q0 40 28 68t68 28h672v192q0 185 131.5 316.5t316.5 131.5 t316.5 -131.5t131.5 -316.5z" />
-<glyph unicode="&#xf09d;" horiz-adv-x="1920" d="M1760 1408q66 0 113 -47t47 -113v-1216q0 -66 -47 -113t-113 -47h-1600q-66 0 -113 47t-47 113v1216q0 66 47 113t113 47h1600zM160 1280q-13 0 -22.5 -9.5t-9.5 -22.5v-224h1664v224q0 13 -9.5 22.5t-22.5 9.5h-1600zM1760 0q13 0 22.5 9.5t9.5 22.5v608h-1664v-608 q0 -13 9.5 -22.5t22.5 -9.5h1600zM256 128v128h256v-128h-256zM640 128v128h384v-128h-384z" />
-<glyph unicode="&#xf09e;" horiz-adv-x="1408" d="M384 192q0 -80 -56 -136t-136 -56t-136 56t-56 136t56 136t136 56t136 -56t56 -136zM896 69q2 -28 -17 -48q-18 -21 -47 -21h-135q-25 0 -43 16.5t-20 41.5q-22 229 -184.5 391.5t-391.5 184.5q-25 2 -41.5 20t-16.5 43v135q0 29 21 47q17 17 43 17h5q160 -13 306 -80.5 t259 -181.5q114 -113 181.5 -259t80.5 -306zM1408 67q2 -27 -18 -47q-18 -20 -46 -20h-143q-26 0 -44.5 17.5t-19.5 42.5q-12 215 -101 408.5t-231.5 336t-336 231.5t-408.5 102q-25 1 -42.5 19.5t-17.5 43.5v143q0 28 20 46q18 18 44 18h3q262 -13 501.5 -120t425.5 -294 q187 -186 294 -425.5t120 -501.5z" />
-<glyph unicode="&#xf0a0;" d="M1040 320q0 -33 -23.5 -56.5t-56.5 -23.5t-56.5 23.5t-23.5 56.5t23.5 56.5t56.5 23.5t56.5 -23.5t23.5 -56.5zM1296 320q0 -33 -23.5 -56.5t-56.5 -23.5t-56.5 23.5t-23.5 56.5t23.5 56.5t56.5 23.5t56.5 -23.5t23.5 -56.5zM1408 160v320q0 13 -9.5 22.5t-22.5 9.5 h-1216q-13 0 -22.5 -9.5t-9.5 -22.5v-320q0 -13 9.5 -22.5t22.5 -9.5h1216q13 0 22.5 9.5t9.5 22.5zM178 640h1180l-157 482q-4 13 -16 21.5t-26 8.5h-782q-14 0 -26 -8.5t-16 -21.5zM1536 480v-320q0 -66 -47 -113t-113 -47h-1216q-66 0 -113 47t-47 113v320q0 25 16 75 l197 606q17 53 63 86t101 33h782q55 0 101 -33t63 -86l197 -606q16 -50 16 -75z" />
-<glyph unicode="&#xf0a1;" horiz-adv-x="1792" d="M1664 896q53 0 90.5 -37.5t37.5 -90.5t-37.5 -90.5t-90.5 -37.5v-384q0 -52 -38 -90t-90 -38q-417 347 -812 380q-58 -19 -91 -66t-31 -100.5t40 -92.5q-20 -33 -23 -65.5t6 -58t33.5 -55t48 -50t61.5 -50.5q-29 -58 -111.5 -83t-168.5 -11.5t-132 55.5q-7 23 -29.5 87.5 t-32 94.5t-23 89t-15 101t3.5 98.5t22 110.5h-122q-66 0 -113 47t-47 113v192q0 66 47 113t113 47h480q435 0 896 384q52 0 90 -38t38 -90v-384zM1536 292v954q-394 -302 -768 -343v-270q377 -42 768 -341z" />
-<glyph unicode="&#xf0a2;" horiz-adv-x="1792" d="M912 -160q0 16 -16 16q-59 0 -101.5 42.5t-42.5 101.5q0 16 -16 16t-16 -16q0 -73 51.5 -124.5t124.5 -51.5q16 0 16 16zM246 128h1300q-266 300 -266 832q0 51 -24 105t-69 103t-121.5 80.5t-169.5 31.5t-169.5 -31.5t-121.5 -80.5t-69 -103t-24 -105q0 -532 -266 -832z M1728 128q0 -52 -38 -90t-90 -38h-448q0 -106 -75 -181t-181 -75t-181 75t-75 181h-448q-52 0 -90 38t-38 90q50 42 91 88t85 119.5t74.5 158.5t50 206t19.5 260q0 152 117 282.5t307 158.5q-8 19 -8 39q0 40 28 68t68 28t68 -28t28 -68q0 -20 -8 -39q190 -28 307 -158.5 t117 -282.5q0 -139 19.5 -260t50 -206t74.5 -158.5t85 -119.5t91 -88z" />
-<glyph unicode="&#xf0a3;" d="M1376 640l138 -135q30 -28 20 -70q-12 -41 -52 -51l-188 -48l53 -186q12 -41 -19 -70q-29 -31 -70 -19l-186 53l-48 -188q-10 -40 -51 -52q-12 -2 -19 -2q-31 0 -51 22l-135 138l-135 -138q-28 -30 -70 -20q-41 11 -51 52l-48 188l-186 -53q-41 -12 -70 19q-31 29 -19 70 l53 186l-188 48q-40 10 -52 51q-10 42 20 70l138 135l-138 135q-30 28 -20 70q12 41 52 51l188 48l-53 186q-12 41 19 70q29 31 70 19l186 -53l48 188q10 41 51 51q41 12 70 -19l135 -139l135 139q29 30 70 19q41 -10 51 -51l48 -188l186 53q41 12 70 -19q31 -29 19 -70 l-53 -186l188 -48q40 -10 52 -51q10 -42 -20 -70z" />
-<glyph unicode="&#xf0a4;" horiz-adv-x="1792" d="M256 192q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1664 768q0 51 -39 89.5t-89 38.5h-576q0 20 15 48.5t33 55t33 68t15 84.5q0 67 -44.5 97.5t-115.5 30.5q-24 0 -90 -139q-24 -44 -37 -65q-40 -64 -112 -145q-71 -81 -101 -106 q-69 -57 -140 -57h-32v-640h32q72 0 167 -32t193.5 -64t179.5 -32q189 0 189 167q0 26 -5 56q30 16 47.5 52.5t17.5 73.5t-18 69q53 50 53 119q0 25 -10 55.5t-25 47.5h331q52 0 90 38t38 90zM1792 769q0 -105 -75.5 -181t-180.5 -76h-169q-4 -62 -37 -119q3 -21 3 -43 q0 -101 -60 -178q1 -139 -85 -219.5t-227 -80.5q-133 0 -322 69q-164 59 -223 59h-288q-53 0 -90.5 37.5t-37.5 90.5v640q0 53 37.5 90.5t90.5 37.5h288q10 0 21.5 4.5t23.5 14t22.5 18t24 22.5t20.5 21.5t19 21.5t14 17q65 74 100 129q13 21 33 62t37 72t40.5 63t55 49.5 t69.5 17.5q125 0 206.5 -67t81.5 -189q0 -68 -22 -128h374q104 0 180 -76t76 -179z" />
-<glyph unicode="&#xf0a5;" horiz-adv-x="1792" d="M1376 128h32v640h-32q-35 0 -67.5 12t-62.5 37t-50 46t-49 54q-2 3 -3.5 4.5t-4 4.5t-4.5 5q-72 81 -112 145q-14 22 -38 68q-1 3 -10.5 22.5t-18.5 36t-20 35.5t-21.5 30.5t-18.5 11.5q-71 0 -115.5 -30.5t-44.5 -97.5q0 -43 15 -84.5t33 -68t33 -55t15 -48.5h-576 q-50 0 -89 -38.5t-39 -89.5q0 -52 38 -90t90 -38h331q-15 -17 -25 -47.5t-10 -55.5q0 -69 53 -119q-18 -32 -18 -69t17.5 -73.5t47.5 -52.5q-4 -24 -4 -56q0 -85 48.5 -126t135.5 -41q84 0 183 32t194 64t167 32zM1664 192q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45 t45 -19t45 19t19 45zM1792 768v-640q0 -53 -37.5 -90.5t-90.5 -37.5h-288q-59 0 -223 -59q-190 -69 -317 -69q-142 0 -230 77.5t-87 217.5l1 5q-61 76 -61 178q0 22 3 43q-33 57 -37 119h-169q-105 0 -180.5 76t-75.5 181q0 103 76 179t180 76h374q-22 60 -22 128 q0 122 81.5 189t206.5 67q38 0 69.5 -17.5t55 -49.5t40.5 -63t37 -72t33 -62q35 -55 100 -129q2 -3 14 -17t19 -21.5t20.5 -21.5t24 -22.5t22.5 -18t23.5 -14t21.5 -4.5h288q53 0 90.5 -37.5t37.5 -90.5z" />
-<glyph unicode="&#xf0a6;" d="M1280 -64q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1408 700q0 189 -167 189q-26 0 -56 -5q-16 30 -52.5 47.5t-73.5 17.5t-69 -18q-50 53 -119 53q-25 0 -55.5 -10t-47.5 -25v331q0 52 -38 90t-90 38q-51 0 -89.5 -39t-38.5 -89v-576 q-20 0 -48.5 15t-55 33t-68 33t-84.5 15q-67 0 -97.5 -44.5t-30.5 -115.5q0 -24 139 -90q44 -24 65 -37q64 -40 145 -112q81 -71 106 -101q57 -69 57 -140v-32h640v32q0 72 32 167t64 193.5t32 179.5zM1536 705q0 -133 -69 -322q-59 -164 -59 -223v-288q0 -53 -37.5 -90.5 t-90.5 -37.5h-640q-53 0 -90.5 37.5t-37.5 90.5v288q0 10 -4.5 21.5t-14 23.5t-18 22.5t-22.5 24t-21.5 20.5t-21.5 19t-17 14q-74 65 -129 100q-21 13 -62 33t-72 37t-63 40.5t-49.5 55t-17.5 69.5q0 125 67 206.5t189 81.5q68 0 128 -22v374q0 104 76 180t179 76 q105 0 181 -75.5t76 -180.5v-169q62 -4 119 -37q21 3 43 3q101 0 178 -60q139 1 219.5 -85t80.5 -227z" />
-<glyph unicode="&#xf0a7;" d="M1408 576q0 84 -32 183t-64 194t-32 167v32h-640v-32q0 -35 -12 -67.5t-37 -62.5t-46 -50t-54 -49q-9 -8 -14 -12q-81 -72 -145 -112q-22 -14 -68 -38q-3 -1 -22.5 -10.5t-36 -18.5t-35.5 -20t-30.5 -21.5t-11.5 -18.5q0 -71 30.5 -115.5t97.5 -44.5q43 0 84.5 15t68 33 t55 33t48.5 15v-576q0 -50 38.5 -89t89.5 -39q52 0 90 38t38 90v331q46 -35 103 -35q69 0 119 53q32 -18 69 -18t73.5 17.5t52.5 47.5q24 -4 56 -4q85 0 126 48.5t41 135.5zM1280 1344q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1536 580 q0 -142 -77.5 -230t-217.5 -87l-5 1q-76 -61 -178 -61q-22 0 -43 3q-54 -30 -119 -37v-169q0 -105 -76 -180.5t-181 -75.5q-103 0 -179 76t-76 180v374q-54 -22 -128 -22q-121 0 -188.5 81.5t-67.5 206.5q0 38 17.5 69.5t49.5 55t63 40.5t72 37t62 33q55 35 129 100 q3 2 17 14t21.5 19t21.5 20.5t22.5 24t18 22.5t14 23.5t4.5 21.5v288q0 53 37.5 90.5t90.5 37.5h640q53 0 90.5 -37.5t37.5 -90.5v-288q0 -59 59 -223q69 -190 69 -317z" />
-<glyph unicode="&#xf0a8;" d="M1280 576v128q0 26 -19 45t-45 19h-502l189 189q19 19 19 45t-19 45l-91 91q-18 18 -45 18t-45 -18l-362 -362l-91 -91q-18 -18 -18 -45t18 -45l91 -91l362 -362q18 -18 45 -18t45 18l91 91q18 18 18 45t-18 45l-189 189h502q26 0 45 19t19 45zM1536 640 q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
-<glyph unicode="&#xf0a9;" d="M1285 640q0 27 -18 45l-91 91l-362 362q-18 18 -45 18t-45 -18l-91 -91q-18 -18 -18 -45t18 -45l189 -189h-502q-26 0 -45 -19t-19 -45v-128q0 -26 19 -45t45 -19h502l-189 -189q-19 -19 -19 -45t19 -45l91 -91q18 -18 45 -18t45 18l362 362l91 91q18 18 18 45zM1536 640 q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
-<glyph unicode="&#xf0aa;" d="M1284 641q0 27 -18 45l-362 362l-91 91q-18 18 -45 18t-45 -18l-91 -91l-362 -362q-18 -18 -18 -45t18 -45l91 -91q18 -18 45 -18t45 18l189 189v-502q0 -26 19 -45t45 -19h128q26 0 45 19t19 45v502l189 -189q19 -19 45 -19t45 19l91 91q18 18 18 45zM1536 640 q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
-<glyph unicode="&#xf0ab;" d="M1284 639q0 27 -18 45l-91 91q-18 18 -45 18t-45 -18l-189 -189v502q0 26 -19 45t-45 19h-128q-26 0 -45 -19t-19 -45v-502l-189 189q-19 19 -45 19t-45 -19l-91 -91q-18 -18 -18 -45t18 -45l362 -362l91 -91q18 -18 45 -18t45 18l91 91l362 362q18 18 18 45zM1536 640 q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
-<glyph unicode="&#xf0ac;" d="M768 1408q209 0 385.5 -103t279.5 -279.5t103 -385.5t-103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103zM1042 887q-2 -1 -9.5 -9.5t-13.5 -9.5q2 0 4.5 5t5 11t3.5 7q6 7 22 15q14 6 52 12q34 8 51 -11 q-2 2 9.5 13t14.5 12q3 2 15 4.5t15 7.5l2 22q-12 -1 -17.5 7t-6.5 21q0 -2 -6 -8q0 7 -4.5 8t-11.5 -1t-9 -1q-10 3 -15 7.5t-8 16.5t-4 15q-2 5 -9.5 10.5t-9.5 10.5q-1 2 -2.5 5.5t-3 6.5t-4 5.5t-5.5 2.5t-7 -5t-7.5 -10t-4.5 -5q-3 2 -6 1.5t-4.5 -1t-4.5 -3t-5 -3.5 q-3 -2 -8.5 -3t-8.5 -2q15 5 -1 11q-10 4 -16 3q9 4 7.5 12t-8.5 14h5q-1 4 -8.5 8.5t-17.5 8.5t-13 6q-8 5 -34 9.5t-33 0.5q-5 -6 -4.5 -10.5t4 -14t3.5 -12.5q1 -6 -5.5 -13t-6.5 -12q0 -7 14 -15.5t10 -21.5q-3 -8 -16 -16t-16 -12q-5 -8 -1.5 -18.5t10.5 -16.5 q2 -2 1.5 -4t-3.5 -4.5t-5.5 -4t-6.5 -3.5l-3 -2q-11 -5 -20.5 6t-13.5 26q-7 25 -16 30q-23 8 -29 -1q-5 13 -41 26q-25 9 -58 4q6 1 0 15q-7 15 -19 12q3 6 4 17.5t1 13.5q3 13 12 23q1 1 7 8.5t9.5 13.5t0.5 6q35 -4 50 11q5 5 11.5 17
 t10.5 17q9 6 14 5.5t14.5 -5.5 t14.5 -5q14 -1 15.5 11t-7.5 20q12 -1 3 17q-5 7 -8 9q-12 4 -27 -5q-8 -4 2 -8q-1 1 -9.5 -10.5t-16.5 -17.5t-16 5q-1 1 -5.5 13.5t-9.5 13.5q-8 0 -16 -15q3 8 -11 15t-24 8q19 12 -8 27q-7 4 -20.5 5t-19.5 -4q-5 -7 -5.5 -11.5t5 -8t10.5 -5.5t11.5 -4t8.5 -3 q14 -10 8 -14q-2 -1 -8.5 -3.5t-11.5 -4.5t-6 -4q-3 -4 0 -14t-2 -14q-5 5 -9 17.5t-7 16.5q7 -9 -25 -6l-10 1q-4 0 -16 -2t-20.5 -1t-13.5 8q-4 8 0 20q1 4 4 2q-4 3 -11 9.5t-10 8.5q-46 -15 -94 -41q6 -1 12 1q5 2 13 6.5t10 5.5q34 14 42 7l5 5q14 -16 20 -25 q-7 4 -30 1q-20 -6 -22 -12q7 -12 5 -18q-4 3 -11.5 10t-14.5 11t-15 5q-16 0 -22 -1q-146 -80 -235 -222q7 -7 12 -8q4 -1 5 -9t2.5 -11t11.5 3q9 -8 3 -19q1 1 44 -27q19 -17 21 -21q3 -11 -10 -18q-1 2 -9 9t-9 4q-3 -5 0.5 -18.5t10.5 -12.5q-7 0 -9.5 -16t-2.5 -35.5 t-1 -23.5l2 -1q-3 -12 5.5 -34.5t21.5 -19.5q-13 -3 20 -43q6 -8 8 -9q3 -2 12 -7.5t15 -10t10 -10.5q4 -5 10 -22.5t14 -23.5q-2 -6 9.5 -20t10.5 -23q-1 0 -2.5 -1t-2.5 -1q3 -7 15.5 -14t15.5 -13q1 -3 2 -10t3 -11t8 -2q2 20 -24 62q-1
 5 25 -17 29q-3 5 -5.5 15.5 t-4.5 14.5q2 0 6 -1.5t8.5 -3.5t7.5 -4t2 -3q-3 -7 2 -17.5t12 -18.5t17 -19t12 -13q6 -6 14 -19.5t0 -13.5q9 0 20 -10t17 -20q5 -8 8 -26t5 -24q2 -7 8.5 -13.5t12.5 -9.5l16 -8t13 -7q5 -2 18.5 -10.5t21.5 -11.5q10 -4 16 -4t14.5 2.5t13.5 3.5q15 2 29 -15t21 -21 q36 -19 55 -11q-2 -1 0.5 -7.5t8 -15.5t9 -14.5t5.5 -8.5q5 -6 18 -15t18 -15q6 4 7 9q-3 -8 7 -20t18 -10q14 3 14 32q-31 -15 -49 18q0 1 -2.5 5.5t-4 8.5t-2.5 8.5t0 7.5t5 3q9 0 10 3.5t-2 12.5t-4 13q-1 8 -11 20t-12 15q-5 -9 -16 -8t-16 9q0 -1 -1.5 -5.5t-1.5 -6.5 q-13 0 -15 1q1 3 2.5 17.5t3.5 22.5q1 4 5.5 12t7.5 14.5t4 12.5t-4.5 9.5t-17.5 2.5q-19 -1 -26 -20q-1 -3 -3 -10.5t-5 -11.5t-9 -7q-7 -3 -24 -2t-24 5q-13 8 -22.5 29t-9.5 37q0 10 2.5 26.5t3 25t-5.5 24.5q3 2 9 9.5t10 10.5q2 1 4.5 1.5t4.5 0t4 1.5t3 6q-1 1 -4 3 q-3 3 -4 3q7 -3 28.5 1.5t27.5 -1.5q15 -11 22 2q0 1 -2.5 9.5t-0.5 13.5q5 -27 29 -9q3 -3 15.5 -5t17.5 -5q3 -2 7 -5.5t5.5 -4.5t5 0.5t8.5 6.5q10 -14 12 -24q11 -40 19 -44q7 -3 11 -2t4.5 9.5t0 14t-1.5 12.5l-1 8v18l-1 8q
 -15 3 -18.5 12t1.5 18.5t15 18.5q1 1 8 3.5 t15.5 6.5t12.5 8q21 19 15 35q7 0 11 9q-1 0 -5 3t-7.5 5t-4.5 2q9 5 2 16q5 3 7.5 11t7.5 10q9 -12 21 -2q7 8 1 16q5 7 20.5 10.5t18.5 9.5q7 -2 8 2t1 12t3 12q4 5 15 9t13 5l17 11q3 4 0 4q18 -2 31 11q10 11 -6 20q3 6 -3 9.5t-15 5.5q3 1 11.5 0.5t10.5 1.5 q15 10 -7 16q-17 5 -43 -12zM879 10q206 36 351 189q-3 3 -12.5 4.5t-12.5 3.5q-18 7 -24 8q1 7 -2.5 13t-8 9t-12.5 8t-11 7q-2 2 -7 6t-7 5.5t-7.5 4.5t-8.5 2t-10 -1l-3 -1q-3 -1 -5.5 -2.5t-5.5 -3t-4 -3t0 -2.5q-21 17 -36 22q-5 1 -11 5.5t-10.5 7t-10 1.5t-11.5 -7 q-5 -5 -6 -15t-2 -13q-7 5 0 17.5t2 18.5q-3 6 -10.5 4.5t-12 -4.5t-11.5 -8.5t-9 -6.5t-8.5 -5.5t-8.5 -7.5q-3 -4 -6 -12t-5 -11q-2 4 -11.5 6.5t-9.5 5.5q2 -10

<TRUNCATED>
http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/assets/fonts/fontawesome-webfont.ttf
----------------------------------------------------------------------
diff --git a/website/assets/fonts/fontawesome-webfont.ttf b/website/assets/fonts/fontawesome-webfont.ttf
deleted file mode 100644
index 96a3639..0000000
Binary files a/website/assets/fonts/fontawesome-webfont.ttf and /dev/null differ


[40/52] [abbrv] kylin git commit: KYLIN-1416 remove website from coding branch

Posted by li...@apache.org.
http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_docs/tutorial/powerbi.cn.md
----------------------------------------------------------------------
diff --git a/website/_docs/tutorial/powerbi.cn.md b/website/_docs/tutorial/powerbi.cn.md
deleted file mode 100644
index 9c2a539..0000000
--- a/website/_docs/tutorial/powerbi.cn.md
+++ /dev/null
@@ -1,56 +0,0 @@
----
-layout: docs-cn
-title:  微软Excel及Power BI教程
-categories: tutorial
-permalink: /cn/docs/tutorial/powerbi.html
-version: v1.2
-since: v1.2
----
-
-Microsoft Excel是当今Windows平台上最流行的数据处理软件之一,支持多种数据处理功能,可以利用Power Query从ODBC数据源读取数据并返回到数据表中。
-
-Microsoft Power BI 是由微软推出的商业智能的专业分析工具,给用户提供简单且丰富的数据可视化及分析功能。
-
-> Apache Kylin目前版本不支持原始数据的查询,部分查询会因此失败,导致应用程序发生异常,建议打上KYLIN-1075补丁包以优化查询结果的显示。
-
-
-> Power BI及Excel不支持"connect live"模式,请注意并添加where条件在查询超大数据集时候,以避免从服务器拉去过多的数据到本地,甚至在某些情况下查询执行失败。
-
-### Install ODBC Driver
-参考页面[Kylin ODBC 驱动程序教程](./odbc.html),请确保下载并安装Kylin ODBC Driver __v1.2__. 如果你安装有早前版本,请卸载后再安装。 
-
-### 连接Excel到Kylin
-1. 从微软官网下载和安装Power Query,安装完成后在Excel中会看到Power Query的Fast Tab,单击`From other sources`下拉按钮,并选择`From ODBC`项
-![](/images/tutorial/odbc/ms_tool/Picture1.png)
-
-2. 在弹出的`From ODBC`数据连接向导中输入Apache Kylin服务器的连接字符串,也可以在`SQL`文本框中输入您想要执行的SQL语句,单击`OK`,SQL的执行结果就会立即加载到Excel的数据表中
-![](/images/tutorial/odbc/ms_tool/Picture2.png)
-
-> 为了简化连接字符串的输入,推荐创建Apache Kylin的DSN,可以将连接字符串简化为DSN=[YOUR_DSN_NAME],有关DSN的创建请参考:[https://support.microsoft.com/en-us/kb/305599](https://support.microsoft.com/en-us/kb/305599)。
-
- 
-3. 如果您选择不输入SQL语句,Power Query将会列出所有的数据库表,您可以根据需要对整张表的数据进行加载。但是,Apache Kylin暂不支持原数据的查询,部分表的加载可能因此受限
-![](/images/tutorial/odbc/ms_tool/Picture3.png)
-
-4. 稍等片刻,数据已成功加载到Excel中
-![](/images/tutorial/odbc/ms_tool/Picture4.png)
-
-5.  一旦服务器端数据产生更新,则需要对Excel中的数据进行同步,右键单击右侧列表中的数据源,选择`Refresh`,最新的数据便会更新到数据表中.
-
-6.  1.  为了提升性能,可以在Power Query中打开`Query Options`设置,然后开启`Fast data load`,这将提高数据加载速度,但可能造成界面的暂时无响应
-
-### Power BI
-1.  启动您已经安装的Power BI桌面版程序,单击`Get data`按钮,并选中ODBC数据源.
-![](/images/tutorial/odbc/ms_tool/Picture5.png)
-
-2.  在弹出的`From ODBC`数据连接向导中输入Apache Kylin服务器的数据库连接字符串,也可以在`SQL`文本框中输入您想要执行的SQL语句。单击`OK`,SQL的执行结果就会立即加载到Power BI中
-![](/images/tutorial/odbc/ms_tool/Picture6.png)
-
-3.  如果您选择不输入SQL语句,Power BI将会列出项目中所有的表,您可以根据需要将整张表的数据进行加载。但是,Apache Kylin暂不支持原数据的查询,部分表的加载可能因此受限
-![](/images/tutorial/odbc/ms_tool/Picture7.png)
-
-4.  现在你可以进一步使用Power BI进行可视化分析:
-![](/images/tutorial/odbc/ms_tool/Picture8.png)
-
-5.  单击工具栏的`Refresh`按钮即可重新加载数据并对图表进行更新
-

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_docs/tutorial/powerbi.md
----------------------------------------------------------------------
diff --git a/website/_docs/tutorial/powerbi.md b/website/_docs/tutorial/powerbi.md
deleted file mode 100644
index 20cd782..0000000
--- a/website/_docs/tutorial/powerbi.md
+++ /dev/null
@@ -1,55 +0,0 @@
----
-layout: docs
-title:  MS Excel and Power BI Tutorial
-categories: tutorial
-permalink: /docs/tutorial/powerbi.html
-version: v1.2
-since: v1.2
----
-
-Microsoft Excel is one of the most famous data tool on Windows platform, and has plenty of data analyzing functions. With Power Query installed as plug-in, excel can easily read data from ODBC data source and fill spreadsheets. 
-
-Microsoft Power BI is a business intelligence tool providing rich functionality and experience for data visualization and processing to user.
-
-> Apache Kylin currently doesn't support query on raw data yet, some queries might fail and cause some exceptions in application. Patch KYLIN-1075 is recommended to get better look of query result.
-
-> Power BI and Excel do not support "connect live" model for other ODBC driver yet, please pay attention when you query on huge dataset, it may pull too many data into your client which will take a while even fail at the end.
-
-### Install ODBC Driver
-Refer to this guide: [Kylin ODBC Driver Tutorial](./odbc.html).
-Please make sure to download and install Kylin ODBC Driver __v1.2__. If you already installed ODBC Driver in your system, please uninstall it first. 
-
-### Kylin and Excel
-1. Download Power Query from Microsoft’s Website and install it. Then run Excel, switch to `Power Query` fast tab, click `From Other Sources` dropdown list, and select `ODBC` item.
-![](/images/tutorial/odbc/ms_tool/Picture1.png)
-
-2.  You’ll see `From ODBC` dialog, just type Database Connection String of Apache Kylin Server in the `Connection String` textbox. Optionally you can type a SQL statement in `SQL statement` textbox. Click `OK`, result set will run to your spreadsheet now.
-![](/images/tutorial/odbc/ms_tool/Picture2.png)
-
-> Tips: In order to simplify the Database Connection String, DSN is recommended, which can shorten the Connection String like `DSN=[YOUR_DSN_NAME]`. Details about DSN, refer to [https://support.microsoft.com/en-us/kb/305599](https://support.microsoft.com/en-us/kb/305599).
- 
-3. If you didn’t input the SQL statement in last step, Power Query will list all tables in the project, which means you can load data from the whole table. But, since Apache Kylin cannot query on raw data currently, this function may be limited.
-![](/images/tutorial/odbc/ms_tool/Picture3.png)
-
-4.  Hold on for a while, the data is lying in Excel now.
-![](/images/tutorial/odbc/ms_tool/Picture4.png)
-
-5.  If you want to sync data with Kylin Server, just right click the data source in right panel, and select `Refresh`, then you’ll see the latest data.
-
-6.  To improve data loading performance, you can enable `Fast data load` in Power Query, but this will make your UI unresponsive for a while. 
-
-### Power BI
-1.  Run Power BI Desktop, and click `Get Data` button, then select `ODBC` as data source type.
-![](/images/tutorial/odbc/ms_tool/Picture5.png)
-
-2.  Same with Excel, just type Database Connection String of Apache Kylin Server in the `Connection String` textbox, and optionally type a SQL statement in `SQL statement` textbox. Click `OK`, the result set will come to Power BI as a new data source query.
-![](/images/tutorial/odbc/ms_tool/Picture6.png)
-
-3.  If you didn’t input the SQL statement in last step, Power BI will list all tables in the project, which means you can load data from the whole table. But, since Apache Kylin cannot query on raw data currently, this function may be limited.
-![](/images/tutorial/odbc/ms_tool/Picture7.png)
-
-4.  Now you can start to enjoy analyzing with Power BI.
-![](/images/tutorial/odbc/ms_tool/Picture8.png)
-
-5.  To reload the data and redraw the charts, just click `Refresh` button in `Home` fast tab.
-

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_docs/tutorial/tableau.cn.md
----------------------------------------------------------------------
diff --git a/website/_docs/tutorial/tableau.cn.md b/website/_docs/tutorial/tableau.cn.md
deleted file mode 100644
index 0ca4a39..0000000
--- a/website/_docs/tutorial/tableau.cn.md
+++ /dev/null
@@ -1,116 +0,0 @@
----
-layout: docs-cn
-title:  Tableau教程
-categories: 教程
-permalink: /cn/docs/tutorial/tableau.html
-version: v1.2
-since: v0.7.1
----
-
-> Kylin ODBC驱动程序与Tableau存在一些限制,请在尝试前仔细阅读本说明书。
-> * 仅支持“managed”分析路径,Kylin引擎将对意外的维度或度量报错
-> * 请始终优先选择事实表,然后使用正确的连接条件添加查找表(cube中已定义的连接类型)
-> * 请勿尝试在多个事实表或多个查找表之间进行连接;
-> * 你可以尝试使用类似Tableau过滤器中seller id这样的高基数维度,但引擎现在将只返回有限个Tableau过滤器中的seller id。
-> 
-> 如需更多详细信息或有任何问题,请联系Kylin团队:`kylinolap@gmail.com`
-
-
-### 使用Tableau 9.x的用户
-请参考[Tableau 9 教程](./tableau_91.html)以获得更详细帮助。
-
-### 步骤1. 安装Kylin ODBC驱动程序
-参考页面[Kylin ODBC 驱动程序教程](./odbc.html)。
-
-### 步骤2. 连接到Kylin服务器
-> 我们建议使用Connect Using Driver而不是Using DSN。
-
-Connect Using Driver: 选择左侧面板中的“Other Database(ODBC)”和弹出窗口的“KylinODBCDriver”。
-
-![](/images/Kylin-and-Tableau-Tutorial/1 odbc.png)
-
-输入你的服务器位置和证书:服务器主机,端口,用户名和密码。
-
-![](/images/Kylin-and-Tableau-Tutorial/2 serverhost.jpg)
-
-点击“Connect”获取你有权限访问的项目列表。有关权限的详细信息请参考[Kylin Cube Permission Grant Tutorial](https://github.com/KylinOLAP/Kylin/wiki/Kylin-Cube-Permission-Grant-Tutorial)。然后在下拉列表中选择你想要连接的项目。
-
-![](/images/Kylin-and-Tableau-Tutorial/3 project.jpg)
-
-点击“Done”完成连接。
-
-![](/images/Kylin-and-Tableau-Tutorial/4 done.jpg)
-
-### 步骤3. 使用单表或多表
-> 限制
->    * 必须首先选择事实表
->    * 请勿仅支持从查找表选择
->    * 连接条件必须与cube定义匹配
-
-**选择事实表**
-
-选择`Multiple Tables`。
-
-![](/images/Kylin-and-Tableau-Tutorial/5 multipleTable.jpg)
-
-然后点击`Add Table...`添加一张事实表。
-
-![](/images/Kylin-and-Tableau-Tutorial/6 facttable.jpg)
-
-![](/images/Kylin-and-Tableau-Tutorial/6 facttable2.jpg)
-
-**选择查找表**
-
-点击`Add Table...`添加一张查找表。
-
-![](/images/Kylin-and-Tableau-Tutorial/7 lkptable.jpg)
-
-仔细建立连接条款。
-
-![](/images/Kylin-and-Tableau-Tutorial/8 join.jpg)
-
-继续通过点击`Add Table...`添加表直到所有的查找表都被正确添加。命名此连接以在Tableau中使用。
-
-![](/images/Kylin-and-Tableau-Tutorial/9 connName.jpg)
-
-**使用Connect Live**
-
-`Data Connection`共有三种类型。选择`Connect Live`选项。
-
-![](/images/Kylin-and-Tableau-Tutorial/10 connectLive.jpg)
-
-然后你就能够尽情使用Tableau进行分析。
-
-![](/images/Kylin-and-Tableau-Tutorial/11 analysis.jpg)
-
-**添加额外查找表**
-
-点击顶部菜单栏的`Data`,选择`Edit Tables...`更新查找表信息。
-
-![](/images/Kylin-and-Tableau-Tutorial/12 edit tables.jpg)
-
-### 步骤4. 使用自定义SQL
-使用自定义SQL类似于使用单表/多表,但你需要在`Custom SQL`标签复制你的SQL后采取同上指令。
-
-![](/images/Kylin-and-Tableau-Tutorial/19 custom.jpg)
-
-### 步骤5. 发布到Tableau服务器
-如果你已经完成使用Tableau制作一个仪表板,你可以将它发布到Tableau服务器上。
-点击顶部菜单栏的`Server`,选择`Publish Workbook...`。
-
-![](/images/Kylin-and-Tableau-Tutorial/14 publish.jpg)
-
-然后登陆你的Tableau服务器并准备发布。
-
-![](/images/Kylin-and-Tableau-Tutorial/16 prepare-publish.png)
-
-如果你正在使用Connect Using Driver而不是DSN连接,你还将需要嵌入你的密码。点击左下方的`Authentication`按钮并选择`Embedded Password`。点击`Publish`然后你将看到结果。
-
-![](/images/Kylin-and-Tableau-Tutorial/17 embedded-pwd.png)
-
-### 小贴士
-* 在Tableau中隐藏表名
-
-    * Tableau将会根据源表名分组显示列,但用户可能希望根据其他不同的安排组织列。使用Tableau中的"Group by Folder"并创建文件夹来对不同的列分组。
-
-     ![](/images/Kylin-and-Tableau-Tutorial/18 groupby-folder.jpg)

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_docs/tutorial/tableau.md
----------------------------------------------------------------------
diff --git a/website/_docs/tutorial/tableau.md b/website/_docs/tutorial/tableau.md
deleted file mode 100644
index f805bcc..0000000
--- a/website/_docs/tutorial/tableau.md
+++ /dev/null
@@ -1,115 +0,0 @@
----
-layout: docs
-title:  Tableau Tutorial
-categories: tutorial
-permalink: /docs/tutorial/tableau.html
-version: v1.2
-since: v0.7.1
----
-
-> There are some limitations of Kylin ODBC driver with Tableau, please read carefully this instruction before you try it.
-> 
-> * Only support "managed" analysis path, Kylin engine will raise exception for unexpected dimension or metric
-> * Please always select Fact Table first, then add lookup tables with correct join condition (defined join type in cube)
-> * Do not try to join between fact tables or lookup tables;
-> * You can try to use high cardinality dimensions like seller id as Tableau Filter, but the engine will only return limited seller id in Tableau's filter now.
-
-### For Tableau 9.x User
-Please refer to [Tableau 9.x Tutorial](./tableau_91.html) for detail guide.
-
-### Step 1. Install Kylin ODBC Driver
-Refer to this guide: [Kylin ODBC Driver Tutorial](./odbc.html).
-
-### Step 2. Connect to Kylin Server
-> We recommended to use Connect Using Driver instead of Using DSN.
-
-Connect Using Driver: Select "Other Database(ODBC)" in the left panel and choose KylinODBCDriver in the pop-up window. 
-
-![](/images/Kylin-and-Tableau-Tutorial/1 odbc.png)
-
-Enter your Sever location and credentials: server host, port, username and password.
-
-![]( /images/Kylin-and-Tableau-Tutorial/2 serverhost.jpg)
-
-Click "Connect" to get the list of projects that you have permission to access. See details about permission in [Kylin Cube Permission Grant Tutorial](./acl.html). Then choose the project you want to connect in the drop down list. 
-
-![]( /images/Kylin-and-Tableau-Tutorial/3 project.jpg)
-
-Click "Done" to complete the connection.
-
-![]( /images/Kylin-and-Tableau-Tutorial/4 done.jpg)
-
-### Step 3. Using Single Table or Multiple Tables
-> Limitation
-> 
->    * Must select FACT table first
->    * Do not support select from lookup table only
->    * The join condition must match within cube definition
-
-**Select Fact Table**
-
-Select `Multiple Tables`.
-
-![]( /images/Kylin-and-Tableau-Tutorial/5 multipleTable.jpg)
-
-Then click `Add Table...` to add a fact table.
-
-![]( /images/Kylin-and-Tableau-Tutorial/6 facttable.jpg)
-
-![]( /images/Kylin-and-Tableau-Tutorial/6 facttable2.jpg)
-
-**Select Look-up Table**
-
-Click `Add Table...` to add a look-up table. 
-
-![]( /images/Kylin-and-Tableau-Tutorial/7 lkptable.jpg)
-
-Set up the join clause carefully. 
-
-![]( /images/Kylin-and-Tableau-Tutorial/8 join.jpg)
-
-Keep add tables through click `Add Table...` until all the look-up tables have been added properly. Give the connection a name for use in Tableau.
-
-![]( /images/Kylin-and-Tableau-Tutorial/9 connName.jpg)
-
-**Using Connect Live**
-
-There are three types of `Data Connection`. Choose the `Connect Live` option. 
-
-![]( /images/Kylin-and-Tableau-Tutorial/10 connectLive.jpg)
-
-Then you can enjoy analyzing with Tableau.
-
-![]( /images/Kylin-and-Tableau-Tutorial/11 analysis.jpg)
-
-**Add additional look-up Tables**
-
-Click `Data` in the top menu bar, select `Edit Tables...` to update the look-up table information.
-
-![]( /images/Kylin-and-Tableau-Tutorial/12 edit tables.jpg)
-
-### Step 4. Using Customized SQL
-To use customized SQL resembles using Single Table/Multiple Tables, except that you just need to paste your SQL in `Custom SQL` tab and take the same instruction as above.
-
-![]( /images/Kylin-and-Tableau-Tutorial/19 custom.jpg)
-
-### Step 5. Publish to Tableau Server
-Suppose you have finished making a dashboard with Tableau, you can publish it to Tableau Server.
-Click `Server` in the top menu bar, select `Publish Workbook...`. 
-
-![]( /images/Kylin-and-Tableau-Tutorial/14 publish.jpg)
-
-Then sign in your Tableau Server and prepare to publish. 
-
-![]( /images/Kylin-and-Tableau-Tutorial/16 prepare-publish.png)
-
-If you're Using Driver Connect instead of DSN connect, you'll need to additionally embed your password in. Click the `Authentication` button at left bottom and select `Embedded Password`. Click `Publish` and you will see the result.
-
-![]( /images/Kylin-and-Tableau-Tutorial/17 embedded-pwd.png)
-
-### Tips
-* Hide Table name in Tableau
-
-    * Tableau will display columns be grouped by source table name, but user may want to organize columns with different structure. Using "Group by Folder" in Tableau and Create Folders to group different columns.
-
-     ![]( /images/Kylin-and-Tableau-Tutorial/18 groupby-folder.jpg)

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_docs/tutorial/tableau_91.cn.md
----------------------------------------------------------------------
diff --git a/website/_docs/tutorial/tableau_91.cn.md b/website/_docs/tutorial/tableau_91.cn.md
deleted file mode 100644
index b6e4a45..0000000
--- a/website/_docs/tutorial/tableau_91.cn.md
+++ /dev/null
@@ -1,50 +0,0 @@
----
-layout: docs-cn
-title:  Tableau 9 教程
-categories: tutorial
-permalink: /cn/docs/tutorial/tableau_91.html
-version: v1.2
-since: v1.2
----
-
-Tableau 9已经发布一段时间了,社区有很多用户希望Apache Kylin能进一步支持该版本。现在可以通过更新Kylin ODBC驱动以使用Tableau 9来与Kylin服务进行交互。
-
-> Apache Kylin目前版本不支持原始数据的查询,部分查询会因此失败,导致应用程序发生异常,建议打上KYLIN-1075补丁包以优化查询结果的显示。
-
-### Tableau 8.x 用户
-请参考[Tableau 教程](./tableau.html)以获得更详细帮助。
-
-### Install ODBC Driver
-参考页面[Kylin ODBC 驱动程序教程](./odbc.html),请确保下载并安装Kylin ODBC Driver __v1.2__. 如果你安装有早前版本,请卸载后再安装。 
-
-### Connect to Kylin Server
-在Tableau 9.1创建新的数据连接,单击左侧面板中的`Other Database(ODBC)`,并在弹出窗口中选择`KylinODBCDriver` 
-![](/images/tutorial/odbc/tableau_91/1.png)
-
-输入你的服务器地址、端口、项目、用户名和密码,点击`Connect`可获取有权限访问的所有项目列表。有关权限的详细信息请参考[Kylin Cube 权限授予教程](./acl.html).
-![](/images/tutorial/odbc/tableau_91/2.png)
-
-### 映射数据模型
-在左侧的列表中,选择数据库`defaultCatalog`并单击”搜索“按钮,将列出所有可查询的表。用鼠标把表拖拽到右侧区域,就可以添加表作为数据源,并创建好表与表的连接关系
-![](/images/tutorial/odbc/tableau_91/3.png)
-
-### Connect Live
-Tableau 9.1中有两种数据源连接类型,选择`在线`选项以确保使用'Connect Live'模式
-![](/images/tutorial/odbc/tableau_91/4.png)
-
-### 自定义SQL
-如果需要使用自定义SQL,可以单击左侧`New Custom SQL`并在弹窗中输入SQL语句,就可添加为数据源.
-![](/images/tutorial/odbc/tableau_91/5.png)
-
-### 可视化
-现在你可以进一步使用Tableau进行可视化分析:
-![](/images/tutorial/odbc/tableau_91/6.png)
-
-### 发布到Tableau服务器
-如果希望发布到Tableau服务器, 点击`Server`菜单并选择`Publish Workbook`
-![](/images/tutorial/odbc/tableau_91/7.png)
-
-### 更多
-请参考[Tableau 教程](./tableau.html)以获得更多信息
-
-

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_docs/tutorial/tableau_91.md
----------------------------------------------------------------------
diff --git a/website/_docs/tutorial/tableau_91.md b/website/_docs/tutorial/tableau_91.md
deleted file mode 100755
index c0f013c..0000000
--- a/website/_docs/tutorial/tableau_91.md
+++ /dev/null
@@ -1,51 +0,0 @@
----
-layout: docs
-title:  Tableau 9 Tutorial
-categories: tutorial
-permalink: /docs/tutorial/tableau_91.html
-version: v1.2
-since: v1.2
----
-
-Tableau 9.x has been released a while, there are many users are asking about support this version with Apache Kylin. With updated Kylin ODBC Driver, now user could interactive with Kylin service through Tableau 9.x.
-
-> Apache Kylin currently doesn't support query on raw data yet, some queries might fail and cause some exceptions in application. Patch [KYLIN-1075](https://issues.apache.org/jira/browse/KYLIN-1075) is recommended to get better look of query result.
-
-### For Tableau 8.x User
-Please refer to [Kylin and Tableau Tutorial](./tableau.html) for detail guide.
-
-### Install Kylin ODBC Driver
-Refer to this guide: [Kylin ODBC Driver Tutorial](./odbc.html).
-Please make sure to download and install Kylin ODBC Driver __v1.2__. If you already installed ODBC Driver in your system, please uninstall it first. 
-
-### Connect to Kylin Server
-Connect Using Driver: Start Tableau 9.1 desktop, click `Other Database(ODBC)` in the left panel and choose KylinODBCDriver in the pop-up window. 
-![](/images/tutorial/odbc/tableau_91/1.png)
-
-Provide your Sever location, credentials and project. Clicking `Connect` button, you can get the list of projects that you have permission to access, see details at [Kylin Cube Permission Grant Tutorial](./acl.html).
-![](/images/tutorial/odbc/tableau_91/2.png)
-
-### Mapping Data Model
-In left panel, select `defaultCatalog` as Database, click `Search` button in Table search box, and all tables get listed. With drag and drop to the right region, tables will become data source. Make sure JOINs are configured correctly.
-![](/images/tutorial/odbc/tableau_91/3.png)
-
-### Connect Live
-There are two types of `Connection`, choose the `Live` option to make sure using Connect Live mode.
-![](/images/tutorial/odbc/tableau_91/4.png)
-
-### Custom SQL
-To use customized SQL, click `New Custom SQL` in left panel and type SQL statement in pop-up dialog.
-![](/images/tutorial/odbc/tableau_91/5.png)
-
-### Visualization
-Now you can start to enjou analyzing with Tableau 9.1.
-![](/images/tutorial/odbc/tableau_91/6.png)
-
-### Publish to Tableau Server
-If you want to publish local dashboard to a Tableau Server, just expand `Server` menu and select `Publish Workbook`.
-![](/images/tutorial/odbc/tableau_91/7.png)
-
-### More
-Please refer to [Kylin and Tableau Tutorial](./tableau.html) for more detail.
-
-

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_docs/tutorial/web.cn.md
----------------------------------------------------------------------
diff --git a/website/_docs/tutorial/web.cn.md b/website/_docs/tutorial/web.cn.md
deleted file mode 100644
index f3dbbac..0000000
--- a/website/_docs/tutorial/web.cn.md
+++ /dev/null
@@ -1,139 +0,0 @@
----
-layout: docs-cn
-title:  Kylin网页版教程
-categories: 教程
-permalink: /cn/docs/tutorial/web.html
-version: v1.2
-since: v0.7.1
----
-
-> **支持的浏览器**
-> 
-> Windows: Google Chrome, FireFox
-> 
-> Mac: Google Chrome, FireFox, Safari
-
-## 1. 访问 & 登陆
-访问主机: http://your_sandbox_ip:9080
-使用用户名/密码登陆:ADMIN/KYLIN
-
-![]( /images/Kylin-Web-Tutorial/1 login.png)
-
-## 2. Kylin中可用的Hive表
-虽然Kylin使用SQL作为查询接口并利用Hive元数据,Kylin不会让用户查询所有的hive表,因为到目前为止它是一个预构建OLAP(MOLAP)系统。为了使表在Kylin中可用,使用"Sync"方法能够方便地从Hive中同步表。
-
-![]( /images/Kylin-Web-Tutorial/2 tables.png)
-
-## 3. Kylin OLAP Cube
-Kylin的OLAP Cube是从星型模式的Hive表中获取的预计算数据集,这是供用户探索、管理所有cube的网页管理页面。由菜单栏进入`Cubes`页面,系统中所有可用的cube将被列出。
-
-![]( /images/Kylin-Web-Tutorial/3 cubes.png)
-
-探索更多关于Cube的详细信息
-
-* 表格视图:
-
-   ![]( /images/Kylin-Web-Tutorial/4 form-view.png)
-
-* SQL 视图 (Hive查询读取数据以生成cube):
-
-   ![]( /images/Kylin-Web-Tutorial/5 sql-view.png)
-
-* 可视化 (显示这个cube背后的星型模式):
-
-   ![]( /images/Kylin-Web-Tutorial/6 visualization.png)
-
-* 访问 (授予用户/角色权限,beta版中授予权限操作仅对管理员开放):
-
-   ![]( /images/Kylin-Web-Tutorial/7 access.png)
-
-## 4. 在网页上编写和运行SQL
-Kelin的网页版为用户提供了一个简单的查询工具来运行SQL以探索现存的cube,验证结果并探索使用#5中的Pivot analysis与可视化分析的结果集。
-
-> **查询限制**
-> 
-> 1. 仅支持SELECT查询
-> 
-> 2. 为了避免从服务器到客户端产生巨大的网络流量,beta版中的扫描范围阀值被设置为1,000,000。
-> 
-> 3. beta版中,SQL在cube中无法找到的数据将不会重定向到Hive
-
-由菜单栏进入“Query”页面:
-
-![]( /images/Kylin-Web-Tutorial/8 query.png)
-
-* 源表:
-
-   浏览器当前可用表(与Hive相同的结构和元数据):
-  
-   ![]( /images/Kylin-Web-Tutorial/9 query-table.png)
-
-* 新的查询:
-
-   你可以编写和运行你的查询并探索结果。这里提供一个查询供你参考:
-
-   ![]( /images/Kylin-Web-Tutorial/10 query-result.png)
-
-* 已保存的查询:
-
-   与用户账号关联,你将能够从不同的浏览器甚至机器上获取已保存的查询。
-   在结果区域点击“Save”,将会弹出名字和描述来保存当前查询:
-
-   ![]( /images/Kylin-Web-Tutorial/11 save-query.png)
-
-   点击“Saved Queries”探索所有已保存的查询,你可以直接重新提交它来运行或删除它:
-
-   ![]( /images/Kylin-Web-Tutorial/11 save-query-2.png)
-
-* 查询历史:
-
-   仅保存当前用户在当前浏览器中的查询历史,这将需要启用cookie,并且如果你清理浏览器缓存将会丢失数据。点击“Query History”标签,你可以直接重新提交其中的任何一条并再次运行。
-
-## 5. Pivot Analysis与可视化
-Kylin的网页版提供一个简单的Pivot与可视化分析工具供用户探索他们的查询结果:
-
-* 一般信息:
-
-   当查询运行成功后,它将呈现一个成功指标与被访问的cube名字。
-   同时它将会呈现这个查询在后台引擎运行了多久(不包括从Kylin服务器到浏览器的网络通信):
-
-   ![]( /images/Kylin-Web-Tutorial/12 general.png)
-
-* 查询结果:
-
-   能够方便地在一个列上排序。
-
-   ![]( /images/Kylin-Web-Tutorial/13 results.png)
-
-* 导出到CSV文件
-
-   点击“Export”按钮以CSV文件格式保存当前结果。
-
-* Pivot表:
-
-   将一个或多个列拖放到标头,结果将根据这些列的值分组:
-
-   ![]( /images/Kylin-Web-Tutorial/14 drag.png)
-
-* 可视化:
-
-   同时,结果集将被方便地显示在“可视化”的不同图表中:
-
-   注意:线形图仅当至少一个从Hive表中获取的维度有真实的“Date”数据类型列时才是可用的。
-
-   * 条形图:
-
-   ![]( /images/Kylin-Web-Tutorial/15 bar-chart.png)
-   
-   * 饼图:
-
-   ![]( /images/Kylin-Web-Tutorial/16 pie-chart.png)
-
-   * 线形图:
-
-   ![]( /images/Kylin-Web-Tutorial/17 line-chart.png)
-
-## 6. Cube建立 Job监控
-监控和管理cube建立过程与诊断的详细信息,甚至直接连接到Hadoop的job信息:
-
-![](/images/Kylin-Web-Tutorial/7 job-steps.png)

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_docs/tutorial/web.md
----------------------------------------------------------------------
diff --git a/website/_docs/tutorial/web.md b/website/_docs/tutorial/web.md
deleted file mode 100644
index f664a7b..0000000
--- a/website/_docs/tutorial/web.md
+++ /dev/null
@@ -1,139 +0,0 @@
----
-layout: docs
-title:  Kylin Web Tutorial
-categories: tutorial
-permalink: /docs/tutorial/web.html
-version: v1.2
-since: v0.7.1
----
-
-> **Supported Browsers**
-> 
-> Windows: Google Chrome, FireFox
-> 
-> Mac: Google Chrome, FireFox, Safari
-
-## 1. Access & Login
-Host to access: http://your_sandbox_ip:9080
-Login with username/password: ADMIN/KYLIN
-
-![](/images/Kylin-Web-Tutorial/1 login.png)
-
-## 2. Available Hive Tables in Kylin
-Although Kylin will using SQL as query interface and leverage Hive metadata, kylin will not enable user to query all hive tables since it's a pre-build OLAP (MOLAP) system so far. To enable Table in Kylin, it will be easy to using "Sync" function to sync up tables from Hive.
-
-![](/images/Kylin-Web-Tutorial/2 tables.png)
-
-## 3. Kylin OLAP Cube
-Kylin's OLAP Cubes are pre-calculation datasets from Star Schema Hive tables, Here's the web management interface for user to explorer, manage all cubes.Go to `Cubes` Menu, it will list all cubes available in system:
-
-![](/images/Kylin-Web-Tutorial/3 cubes.png)
-
-To explore more detail about the Cube
-
-* Form View:
-
-   ![](/images/Kylin-Web-Tutorial/4 form-view.png)
-
-* SQL View (Hive Query to read data to generate the cube):
-
-   ![](/images/Kylin-Web-Tutorial/5 sql-view.png)
-
-* Visualization (Showing the Star Schema behind of this cube):
-
-   ![](/images/Kylin-Web-Tutorial/6 visualization.png)
-
-* Access (Grant user/role privileges, Grant operation only open to Admin in beta):
-
-   ![](/images/Kylin-Web-Tutorial/7 access.png)
-
-## 4. Write and Execute SQL on web
-Kylin's web offer a simple query tool for user to run SQL to explorer existing cube, verify result and explorer the result set using #5's Pivot analysis and visualization
-
-> **Query Limit**
-> 
-> 1. Only SELECT query be supported
-> 
-> 2. To avoid huge network traffic from server to client, the scan range's threshold be set to 1,000,000 in beta.
-> 
-> 3. SQL can't found data from cube will not redirect to Hive in beta
-
-Go to "Query" menu:
-
-![](/images/Kylin-Web-Tutorial/8 query.png)
-
-* Source Tables:
-
-   Browser current available Tables (same structure and metadata as Hive):
-  
-   ![](/images/Kylin-Web-Tutorial/9 query-table.png)
-
-* New Query:
-
-   You can write and execute your query and explorer the result. One query for you reference:
-
-   ![](/images/Kylin-Web-Tutorial/10 query-result.png)
-
-* Saved Query:
-
-   Associate with user account, you can get saved query from different browsers even machines.
-   Click "Save" in Result area, it will popup for name and description to save current query:
-
-   ![](/images/Kylin-Web-Tutorial/11 save-query.png)
-
-   Click "Saved Queries" to browser all your saved queries, you could direct resubmit it to run or remove it:
-
-   ![](/images/Kylin-Web-Tutorial/11 save-query-2.png)
-
-* Query History:
-
-   Only keep the current user's query history in current bowser, it will require cookie enabled and will lost if you clean up bowser's cache.Click "Query History" tab, you could directly resubmit any of them to execute again.
-
-## 5. Pivot Analysis and Visualization
-There's one simple pivot and visualization analysis tool in Kylin's web for user to explore their query result:
-
-* General Information:
-
-   When the query execute success, it will present a success indictor and also a cube's name which be hit. 
-   Also it will present how long this query be executed in backend engine (not cover network traffic from Kylin server to browser):
-
-   ![](/images/Kylin-Web-Tutorial/12 general.png)
-
-* Query Result:
-
-   It's easy to order on one column.
-
-   ![](/images/Kylin-Web-Tutorial/13 results.png)
-
-* Export to CSV File
-
-   Click "Export" button to save current result as CSV file.
-
-* Pivot Table:
-
-   Drag and Drop one or more columns into the header, the result will grouping by such column's value:
-
-   ![](/images/Kylin-Web-Tutorial/14 drag.png)
-
-* Visualization:
-
-   Also, the result set will be easy to show with different charts in "Visualization":
-
-   note: line chart only available when there's at least one dimension with real "Date" data type of column from Hive Table.
-
-   * Bar Chart:
-
-   ![](/images/Kylin-Web-Tutorial/15 bar-chart.png)
-   
-   * Pie Chart:
-
-   ![](/images/Kylin-Web-Tutorial/16 pie-chart.png)
-
-   * Line Chart
-
-   ![](/images/Kylin-Web-Tutorial/17 line-chart.png)
-
-## 6. Cube Build Job Monitoring
-Monitor and manage cube build process, diagnostic into the detail and even link to Hadoop's job information directly:
-
-![](/images/Kylin-Web-Tutorial/7 job-steps.png)

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_includes/dev_nav.html
----------------------------------------------------------------------
diff --git a/website/_includes/dev_nav.html b/website/_includes/dev_nav.html
deleted file mode 100644
index ddca361..0000000
--- a/website/_includes/dev_nav.html
+++ /dev/null
@@ -1,33 +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.
--->
-
-<div class="col-md-3 col-lg-3 col-xs-4 aside1 visible-md visible-lg" id="nside1" style=" padding-top: 2em">
-    <ul class="nav nav-pills nav-stacked">
-    {% for section in site.data.development %}
-    <li><a href="#{{ section | first }}" data-toggle="collapse" id="navtitle">{{ section.title }}</a></li>
-    <div class="collapse in">
-  	<div class="list-group" id="list1">
-    <ul style="list-style-type:disc">
-    {% include dev_ul.html items=section.dev %}
-        <ul>
-  </div>
-</div>
-    {% endfor %}
-
-    </ul>
-</div>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_includes/dev_ul.html
----------------------------------------------------------------------
diff --git a/website/_includes/dev_ul.html b/website/_includes/dev_ul.html
deleted file mode 100644
index 3c155fc..0000000
--- a/website/_includes/dev_ul.html
+++ /dev/null
@@ -1,26 +0,0 @@
-{% assign items = include.items %}
-
-
-
-{% for item in items %}
-   
-  {% assign item_url = item | prepend:"/development/" | append:".html" %}
-
-  {% if item_url == page.url %}
-    {% assign c = "current" %}
-  {% else %}
-    {% assign c = "" %}
-  {% endif %}
-
-
-
-  {% for p in site.dev %}
-    {% if p.url == item_url %}
-      <li><a href="{{ p.url }}" class="list-group-item-lay pjaxlink" id="navlist">{{p.title}}</a></li>      
-      {% break %}
-    {% endif %}
-  {% endfor %}
-
-{% endfor %}
-
-

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_includes/docs_nav.cn.html
----------------------------------------------------------------------
diff --git a/website/_includes/docs_nav.cn.html b/website/_includes/docs_nav.cn.html
deleted file mode 100755
index 434bd72..0000000
--- a/website/_includes/docs_nav.cn.html
+++ /dev/null
@@ -1,33 +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.
--->
-
-<div class="col-md-3 col-lg-3 col-xs-4 aside1 visible-md visible-lg" id="nside1" style=" padding-top: 2em">
-    <ul class="nav nav-pills nav-stacked">    
-    {% for section in site.data.docs-cn %}
-    <li><a href="#{{ section | first }}" data-toggle="collapse" id="navtitle">{{ section.title }}</a></li>
-    <div class="collapse in">
-  	<div class="list-group" id="list1">
-    <ul style="list-style-type:disc">
-    {% include docs_ul.cn.html items=section.docs %}
-        <ul>
-  </div>
-</div>
-    {% endfor %}
-
-    </ul>
-</div>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_includes/docs_nav.html
----------------------------------------------------------------------
diff --git a/website/_includes/docs_nav.html b/website/_includes/docs_nav.html
deleted file mode 100755
index bec9648..0000000
--- a/website/_includes/docs_nav.html
+++ /dev/null
@@ -1,33 +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.
--->
-
-<div class="col-md-3 col-lg-3 col-xs-4 aside1 visible-md visible-lg" id="nside1" style=" padding-top: 2em">
-    <ul class="nav nav-pills nav-stacked">
-    {% for section in site.data.docs %}
-    <li><a href="#{{ section | first }}" data-toggle="collapse" id="navtitle">{{ section.title }}</a></li>
-    <div class="collapse in">
-  	<div class="list-group" id="list1">
-    <ul style="list-style-type:disc">
-    {% include docs_ul.html items=section.docs %}
-        <ul>
-  </div>
-</div>
-    {% endfor %}
-
-    </ul>
-</div>

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_includes/docs_ul.cn.html
----------------------------------------------------------------------
diff --git a/website/_includes/docs_ul.cn.html b/website/_includes/docs_ul.cn.html
deleted file mode 100644
index b42fc84..0000000
--- a/website/_includes/docs_ul.cn.html
+++ /dev/null
@@ -1,28 +0,0 @@
-{% assign items = include.items %}
-
-
-
-{% for item in items %}
-
-  {% assign item_url = item | prepend:"/cn/docs/" | append:".html" %}
-
-  {% if item_url == page.url %}
-    {% assign c = "current" %}
-  {% else %}
-    {% assign c = "" %}
-  {% endif %}
-
-
-
-  {% for p in site.docs %}
-    {% if p.url == item_url %}
-      <li><a href="{{ p.url }}" class="list-group-item-lay pjaxlink" id="navlist">{{p.title}}</a></li>      
-      {% break %}
-    {% endif %}
-  {% endfor %}
-
-{% endfor %}
-
-
-
-

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_includes/docs_ul.html
----------------------------------------------------------------------
diff --git a/website/_includes/docs_ul.html b/website/_includes/docs_ul.html
deleted file mode 100644
index 89e0805..0000000
--- a/website/_includes/docs_ul.html
+++ /dev/null
@@ -1,29 +0,0 @@
-{% assign items = include.items %}
-
-
-
-{% for item in items %}
-
-  {% assign item_url = item | prepend:"/docs/" | append:".html" %}
-      
-
-  {% if item_url == page.url %}
-    {% assign c = "current" %}
-  {% else %}
-    {% assign c = "" %}
-  {% endif %}
-
-
-
-  {% for p in site.docs %}
-    {% if p.url == item_url %}
-      <li><a href="{{ p.url }}" class="list-group-item-lay pjaxlink" id="navlist">{{p.title}}</a></li>      
-      {% break %}
-    {% endif %}
-  {% endfor %}
-
-{% endfor %}
-
-
-
-

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_includes/footer.html
----------------------------------------------------------------------
diff --git a/website/_includes/footer.html b/website/_includes/footer.html
deleted file mode 100644
index fe51be2..0000000
--- a/website/_includes/footer.html
+++ /dev/null
@@ -1,42 +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.
--->
-
-<footer id="underfooter">
-    <div class="container">
-        <div class="row">
-            <div class="col-md-12 widget">
-                <div class="widget-body" style="text-align:center">
-                    <a href="http://www.apache.org">
-                        <img id="asf-logo" alt="Apache Software Foundation" src="/assets/images/feather-small.gif">
-                    </a>
-
-                    <div>
-                        The contents of this website are © 2015 Apache Software Foundation under the terms of the <a
-                            href="http://www.apache.org/licenses/LICENSE-2.0"> Apache License v2 </a>. Apache Kylin and
-                        its logo are trademarks of the Apache Software Foundation.
-                    </div>
-
-                </div>
-            </div>
-        </div>
-        <!-- /row of widgets -->
-
-    </div>
-    <div></div>
-
-</footer>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_includes/head.cn.html
----------------------------------------------------------------------
diff --git a/website/_includes/head.cn.html b/website/_includes/head.cn.html
deleted file mode 100644
index cd6a26b..0000000
--- a/website/_includes/head.cn.html
+++ /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.
--->
-
-<head>
-  <meta charset="utf-8">
-  <meta http-equiv="X-UA-Compatible" content="IE=edge">
-  <meta name="viewport" content="width=device-width, initial-scale=1">
-
-  <title>Apache Kylin | {% if page.title %}{{ page.title }}{% else %}{{ site.title }}{% endif %}</title>
-  <meta name="description" content="{% if page.excerpt %}{{ page.excerpt | strip_html | strip_newlines | truncate: 160 }}{% else %}{{ site.description }}{% endif %}">
-  <meta name="author"      content="Apache Kylin">
-  <link rel="shortcut icon" href="fav.png" type="image/png">
-
-
-
-<link rel="stylesheet" href="{{ "/assets/css/animate.css"| prepend: site.baseurl }}">
-<!-- Bootstrap -->
-<link rel="stylesheet" href="{{ "/assets/css/bootstrap.min.css"| prepend: site.baseurl }}">
-
-<!-- Fonts -->
-<!-- <link rel="stylesheet" href="http://fonts.googleapis.com/css?family=Alice|Open+Sans:400,300,700"> -->
-
-<!-- Icons -->
-<link rel="stylesheet" href="{{ "/assets/css/font-awesome.min.css"| prepend: site.baseurl }}">
-
-  <!-- Custom styles -->
-  <link rel="stylesheet" href="{{ "/assets/css/styles.css" | prepend: site.baseurl }}">
-  <link rel="stylesheet" href="{{ "/assets/css/docs.css" | prepend: site.baseurl }}">
-
-  <link rel="canonical" href="{{ page.url | replace:'index.html','' | prepend: site.baseurl | prepend: site.url }}">
-  <link rel="alternate" type="application/rss+xml" title="{{ site.title }}" href="{{ "/feed.xml" | prepend: site.baseurl | prepend: site.url }}" />
-
-<!--[if lt IE 9]> <script src="assets/js/html5shiv.js"></script> <![endif]-->
-<script>
-  (function(i,s,o,g,r,a,m){i['GoogleAnalyticsObject']=r;i[r]=i[r]||function(){
-  (i[r].q=i[r].q||[]).push(arguments)},i[r].l=1*new Date();a=s.createElement(o),
-  m=s.getElementsByTagName(o)[0];a.async=1;a.src=g;m.parentNode.insertBefore(a,m)
-  })(window,document,'script','//www.google-analytics.com/analytics.js','ga');
-
-
-  //oringal tracker for kylin.io
-  ga('create', 'UA-55534813-1', 'auto');
-  //new tracker for kylin.apache.org
-  ga('create', 'UA-55534813-2', 'auto', {'name':'toplevel'});
-
-  ga('send', 'pageview');
-  ga('toplevel.send', 'pageview');
-
-
-</script>
-<script type="text/javascript" src="/assets/js/jquery-1.9.1.min.js"></script>
-<script type="text/javascript" src="/assets/js/nside.js"></script> </script>
-<script type="text/javascript" src="/assets/js/nnav.js"></script> </script>
-</head>

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_includes/head.html
----------------------------------------------------------------------
diff --git a/website/_includes/head.html b/website/_includes/head.html
deleted file mode 100644
index 02ad310..0000000
--- a/website/_includes/head.html
+++ /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.
--->
-
-<head>
-  <meta charset="utf-8">
-  <meta http-equiv="X-UA-Compatible" content="IE=edge">
-  <meta name="viewport" content="width=device-width, initial-scale=1">
-
-  <title>Apache Kylin | {% if page.title %}{{ page.title }}{% else %}{{ site.title }}{% endif %}</title>
-  <meta name="description" content="{% if page.excerpt %}{{ page.excerpt | strip_html | strip_newlines | truncate: 160 }}{% else %}{{ site.description }}{% endif %}">
-  <meta name="author"      content="Apache Kylin">
-  <link rel="shortcut icon" href="fav.png" type="image/png">
-
-
-
-<link rel="stylesheet" href="{{ "/assets/css/animate.css"| prepend: site.baseurl }}">
-<!-- Bootstrap -->
-<link rel="stylesheet" href="{{ "/assets/css/bootstrap.min.css"| prepend: site.baseurl }}">
-
-<!-- Fonts -->
-<!-- <link rel="stylesheet" href="http://fonts.googleapis.com/css?family=Alice|Open+Sans:400,300,700"> -->
-
-<!-- Icons -->
-<link rel="stylesheet" href="{{ "/assets/css/font-awesome.min.css"| prepend: site.baseurl }}">
-
-  <!-- Custom styles -->
-  <link rel="stylesheet" href="{{ "/assets/css/styles.css" | prepend: site.baseurl }}">
-  <link rel="stylesheet" href="{{ "/assets/css/docs.css" | prepend: site.baseurl }}">
-  <link rel="stylesheet" href="{{ "/assets/css/pygments.css" | prepend: site.baseurl }}">
-
-  <link rel="canonical" href="{{ page.url | replace:'index.html','' | prepend: site.baseurl | prepend: site.url }}">
-  <link rel="alternate" type="application/rss+xml" title="{{ site.title }}" href="{{ "/feed.xml" | prepend: site.baseurl | prepend: site.url }}" />
-
-<!--[if lt IE 9]> <script src="assets/js/html5shiv.js"></script> <![endif]-->
-<script>
-  (function(i,s,o,g,r,a,m){i['GoogleAnalyticsObject']=r;i[r]=i[r]||function(){
-  (i[r].q=i[r].q||[]).push(arguments)},i[r].l=1*new Date();a=s.createElement(o),
-  m=s.getElementsByTagName(o)[0];a.async=1;a.src=g;m.parentNode.insertBefore(a,m)
-  })(window,document,'script','//www.google-analytics.com/analytics.js','ga');
-
-  //oringal tracker for kylin.io
-  ga('create', 'UA-55534813-1', 'auto');
-  //new tracker for kylin.apache.org
-  ga('create', 'UA-55534813-2', 'auto', {'name':'toplevel'});
-
-  ga('send', 'pageview');
-  ga('toplevel.send', 'pageview');
-
-
-</script>
-<script type="text/javascript" src="/assets/js/jquery-1.9.1.min.js"></script>
-<script type="text/javascript" src="/assets/js/nside.js"></script> </script>
-<script type="text/javascript" src="/assets/js/nnav.js"></script> </script>
-</head>

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_includes/header.cn.html
----------------------------------------------------------------------
diff --git a/website/_includes/header.cn.html b/website/_includes/header.cn.html
deleted file mode 100644
index 24627ff..0000000
--- a/website/_includes/header.cn.html
+++ /dev/null
@@ -1,58 +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.
--->
-
-<header id="header" >
-  <div id="head" class="parallax" parallax-speed="3" >
-    <div id="logo" class="text-center"> <img class="img-circle" id="circlelogo" src="{{ "/assets/images/kylin_logo.jpg"| prepend: site.baseurl }}"> <span class="title" >Apache Kylin™</span> <span class="tagline">Extreme OLAP Engine for Big Data</span> 
-    </div>
-  </div>
-
-  <!-- Main Menu -->
-  <nav class="navbar navbar-default" role="navigation" id="nav-wrapper">
-  <div class="container-fluid" id="nav">
-    <!-- Brand and toggle get grouped for better mobile display -->
-    <div class="navbar-header">
-      <button type="button" class="navbar-toggle collapsed" data-toggle="collapse" data-target="#bs-example-navbar-collapse-1">
-        <span class="sr-only">Toggle navigation</span>
-        <span class="icon-bar"></span>
-        <span class="icon-bar"></span>
-        <span class="icon-bar"></span>
-      </button>
-     
-    </div>
-
-    <!-- Collect the nav links, forms, and other content for toggling -->
-    <div class="collapse navbar-collapse" id="bs-example-navbar-collapse-1">
-      <ul class="nav navbar-nav">
-     <li><a href="/cn">首页</a></li>
-          <li><a href="/cn/docs" >文档</a></li>
-          <li><a href="/cn/download">下载</li>
-          <li><a href="/community" >社区</a></li>
-          <li><a href="/development" >开发</a></li>
-          <li><a href="/blog">博客</li>
-          <li><a href="/" >English</a></li>
-          <li><a href="https://twitter.com/apachekylin" target="_blank" class="fa fa-twitter fa-lg" title="Twitter: @ApacheKylin" ></a></li>
-          <li><a href="https://github.com/apache/kylin" target="_blank" class="fa fa-github-alt fa-lg" title="Github: apache/kylin" ></a></li>          
-          <li><a href="https://www.facebook.com/kylinio" target="_blank" class="fa fa-facebook fa-lg" title="Facebook: kylin.io" ></a></li>            
-      </ul>
-      
-
-    </div><!-- /.navbar-collapse -->
-  </div><!-- /.container-fluid -->
-</nav>
- </header>

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_includes/header.html
----------------------------------------------------------------------
diff --git a/website/_includes/header.html b/website/_includes/header.html
deleted file mode 100644
index ac44d3b..0000000
--- a/website/_includes/header.html
+++ /dev/null
@@ -1,61 +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.
--->
-
-<header id="header" >
-  
-  <div id="head" class="parallax" parallax-speed="3" >
-    <div id="logo" class="text-center"> <img class="img-circle" id="circlelogo" src="{{ "/assets/images/kylin_logo.jpg"| prepend: site.baseurl }}"> <span class="title" >Apache Kylin™</span> <span class="tagline">Extreme OLAP Engine for Big Data</span> 
-    </div>
-  </div>
-  
-
-  <!-- Main Menu -->
-  <nav class="navbar navbar-default" role="navigation" id="nav-wrapper">
-  <div class="container-fluid" id="nav">
-    <!--
-    <img class="img-circle" width="40px" height="40px" id="circlelogo" src="{{ "/assets/images/kylin_logo.jpg"| prepend: site.baseurl }}">
-    -->
-    <!-- Brand and toggle get grouped for better mobile display -->
-    <div class="navbar-header">
-      <button type="button" class="navbar-toggle collapsed" data-toggle="collapse" data-target="#bs-example-navbar-collapse-1">
-        <span class="sr-only">Toggle navigation</span>
-        <span class="icon-bar"></span>
-        <span class="icon-bar"></span>
-        <span class="icon-bar"></span>
-      </button>
-     
-    </div>
-
-    <!-- Collect the nav links, forms, and other content for toggling -->
-    <div class="collapse navbar-collapse" id="bs-example-navbar-collapse-1">
-      <ul class="nav navbar-nav">
-     <li><a href="/">Home</a></li>
-          <li><a href="/docs" >Docs</a></li>
-          <li><a href="/download">Download</li>
-          <li><a href="/community" >Community</a></li>
-          <li><a href="/development" >Development</a></li>
-          <li><a href="/blog">Blog</li>
-          <li><a href="/cn" >中文版</a></li>  
-          <li><a href="https://twitter.com/apachekylin" target="_blank" class="fa fa-twitter fa-lg" title="Twitter: @ApacheKylin" ></a></li>
-          <li><a href="https://github.com/apache/kylin" target="_blank" class="fa fa-github-alt fa-lg" title="Github: apache/kylin" ></a></li>          
-          <li><a href="https://www.facebook.com/kylinio" target="_blank" class="fa fa-facebook fa-lg" title="Facebook: kylin.io" ></a></li>   
-      </ul>      
-    </div><!-- /.navbar-collapse -->
-  </div><!-- /.container-fluid -->
-</nav>
- </header>

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_includes/navside.cn.html
----------------------------------------------------------------------
diff --git a/website/_includes/navside.cn.html b/website/_includes/navside.cn.html
deleted file mode 100755
index 0717238..0000000
--- a/website/_includes/navside.cn.html
+++ /dev/null
@@ -1,112 +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.
--->
-
-<div class="col-md-3 col-lg-3 col-xs-4 aside1 visible-md visible-lg" id="nside1" style=" padding-top: 2em">
-    <ul class="nav nav-pills nav-stacked">
-        {% for category in site.categories %}     <!-- categories -->
-        {% if category[0] == '入门' %}
-        <li><a href="#{{ category | first }}" data-toggle="collapse" id="navtitle">入门</a></li>
-        <div class="collapse in" id="{{ category[0] }}">
-            <div class="list-group" id="list1">
-                <ul style="list-style-type:disc">
-                    {% for post in category[1] %}
-                    <li><a href="{{ post.url }}" class="list-group-item-lay pjaxlink">{{post.title}}</a></li>
-                    {% endfor %}
-                    <ul>
-            </div>
-        </div>
-        {% endif %}
-        {% endfor %}
-
-        {% for category in site.categories %}
-        {% if category[0] == '安装' %}
-        <li><a href="#{{ category | first }}" data-toggle="collapse" id="navtitle">安装</a></li>
-        <div class="collapse in" id="{{ category[0] }}">
-            <div class="list-group" id="list2">
-                <ul style="list-style-type:disc">
-                    {% for post in category[1] %}
-                    <li><a href="{{ post.url }}" class="list-group-item-lay pjaxlink">{{post.title}}</a></li>
-                    {% endfor %}
-                    <ul>
-            </div>
-        </div>
-        {% endif %}
-        {% endfor %}
-
-        {% for category in site.categories %}
-        {% if category[0] == '教程' %}
-        <li><a href="#{{ category | first }}" data-toggle="collapse" id="navtitle">教程</a></li>
-        <div class="collapse in" id="{{ category[0] }}">
-            <div class="list-group" id="list3">
-                <ul style="list-style-type:disc">
-                    {% for post in category[1] %}
-                    <li><a href="{{ post.url }}" class="list-group-item-lay pjaxlink">{{post.title}}</a></li>
-                    {% endfor %}
-                    <ul>
-            </div>
-        </div>
-        {% endif %}
-        {% endfor %}
-
-        {% for category in site.categories %}
-        {% if category[0] == '开发' %}
-        <li><a href="#{{ category | first }}" data-toggle="collapse" id="navtitle">开发</a></li>
-        <div class="collapse in" id="{{ category[0] }}">
-            <div class="list-group" id="list4">
-                <ul style="list-style-type:disc">
-                    {% for post in category[1] %}
-                    <li><a href="{{ post.url }}" class="list-group-item-lay pjaxlink">{{post.title}}</a></li>
-                    {% endfor %}
-                    <ul>
-            </div>
-        </div>
-        {% endif %}
-        {% endfor %}
-
-        {% for category in site.categories %}
-        {% if category[0] == '如何' %}
-        <li><a href="#{{ category | first }}" data-toggle="collapse" id="navtitle">如何</a></li>
-        <div class="collapse in" id="{{ category[0] }}">
-            <div class="list-group" id="list5">
-                <ul style="list-style-type:disc">
-                    {% for post in category[1] %}
-                    <li><a href="{{ post.url }}" class="list-group-item-lay pjaxlink">{{post.title}}</a></li>
-                    {% endfor %}
-                    <ul>
-            </div>
-        </div>
-        {% endif %}
-        {% endfor %}
-
-
-        {% for category in site.categories %}
-        {% if category[0] == '系统api' %}
-        <li><a href="#{{ category | first }}" data-toggle="collapse" id="navtitle">系统API</a></li>
-        <div class="collapse in" id="{{ category[0] }}">
-            <div class="list-group" id="list6">
-                <ul style="list-style-type:disc">
-                    {% for post in category[1] %}
-                    <li><a href="{{ post.url }}" class="list-group-item-lay pjaxlink">{{post.title}}</a></li>
-                    {% endfor %}
-                    <ul>
-            </div>
-        </div>
-        {% endif %}
-        {% endfor %}
-    </ul>
-</div>

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_includes/navside.html
----------------------------------------------------------------------
diff --git a/website/_includes/navside.html b/website/_includes/navside.html
deleted file mode 100755
index f3cc8d4..0000000
--- a/website/_includes/navside.html
+++ /dev/null
@@ -1,90 +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.
--->
-
-
-
-<div class="col-md-3 col-lg-3 col-xs-4 aside1 visible-md visible-lg" id="nside1" style=" padding-top: 2em">
-    <ul class="nav nav-pills nav-stacked">
-
-    <li><a href="#{{ category | first }}" data-toggle="collapse" id="navtitle">Getting Started</a></li>
-        <div class="collapse in" id="{{ category[0] }}">
-            <div class="list-group" id="list1">
-                <ul style="list-style-type:disc">
-                {% for page in site.pages %}
-                    {% if page.categories contains 'gettingstarted' %}
-                    <li><a href="{{ page.url }}" class="list-group-item-lay pjaxlink">{{page.title}}</a></li>
-                    {% endif %}
-                {% endfor %}
-                <ul>
-            </div>
-        </div>
-
-    <li><a href="#{{ category | first }}" data-toggle="collapse" id="navtitle">Installation</a></li>
-    <div class="collapse in" id="{{ category[0] }}">
-        <div class="list-group" id="list1">
-            <ul style="list-style-type:disc">
-            {% for page in site.pages %}
-                {% if page.categories contains 'install' %}
-                <li><a href="{{ page.url }}" class="list-group-item-lay pjaxlink">{{page.title}}</a></li>
-                {% endif %}
-            {% endfor %}
-            <ul>
-        </div>
-        </div>
-
-        <li><a href="#{{ category | first }}" data-toggle="collapse" id="navtitle">How to</a></li>
-        <div class="collapse in" id="{{ category[0] }}">
-            <div class="list-group" id="list5">
-                <ul style="list-style-type:disc">
-                {% for page in site.pages %}
-                    {% if page.categories contains 'howto' %}
-                    <li><a href="{{ page.url }}" class="list-group-item-lay pjaxlink">{{page.title}}</a></li>
-                    {% endif %}
-                {% endfor %}
-                <ul>
-            </div>
-        </div>
-
-        <li><a href="#{{ category | first }}" data-toggle="collapse" id="navtitle">Tutorial</a></li>
-        <div class="collapse in" id="{{ category[0] }}">
-            <div class="list-group" id="list5">
-                <ul style="list-style-type:disc">
-                {% for page in site.pages %}
-                    {% if page.categories contains 'tutorial' %}
-                    <li><a href="{{ page.url }}" class="list-group-item-lay pjaxlink">{{page.title}}</a></li>
-                    {% endif %}
-                {% endfor %}
-                <ul>
-            </div>
-        </div>
-
-        <li><a href="#{{ category | first }}" data-toggle="collapse" id="navtitle">Development</a></li>
-        <div class="collapse in" id="{{ category[0] }}">
-            <div class="list-group" id="list5">
-                <ul style="list-style-type:disc">
-                {% for page in site.pages %}
-                    {% if page.categories contains 'development' %}
-                    <li><a href="{{ page.url }}" class="list-group-item-lay pjaxlink">{{page.title}}</a></li>
-                    {% endif %}
-                {% endfor %}
-                <ul>
-            </div>
-        </div>
-
-    </ul>
-</div>

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_layouts/blog.html
----------------------------------------------------------------------
diff --git a/website/_layouts/blog.html b/website/_layouts/blog.html
deleted file mode 100644
index c49b015..0000000
--- a/website/_layouts/blog.html
+++ /dev/null
@@ -1,47 +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.
--->
-<!doctype html>
-<html>
-	{% include head.html %}
-	<body>
-		{% include header.html %}
-		<div class="page-content">
-			<header style=" padding:2em 0 0 0">
-			<div class="container" >
-				<h4 class="section-title"><span>Apache Kylin™ Technical Blog</span></h4>
-			</div>
-		</div>
-
-		<div class="container">
-			<div>
-				<article class="post-content" >	
-				{{ content }}
-				</article>
-			</div>
-		</div>		
-		{% include footer.html %}
-
-	<script src="/assets/js/jquery-1.9.1.min.js"></script> 
-	<script src="/assets/js/bootstrap.min.js"></script> 
-	<script src="/assets/js/main.js"></script>
-	</body>
-</html>
-
-
-
-

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_layouts/default-cn.html
----------------------------------------------------------------------
diff --git a/website/_layouts/default-cn.html b/website/_layouts/default-cn.html
deleted file mode 100644
index f5bad69..0000000
--- a/website/_layouts/default-cn.html
+++ /dev/null
@@ -1,41 +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.
--->
-
-<!DOCTYPE html>
-<html>
-
-  {% include head.cn.html %}
-
-  <body>
-
-    {% include header.cn.html %}
-
-    <div class="page-content">
-      <div class="wrapper">
-        {{ content }}
-      </div>
-    </div>
-
-    {% include footer.html %}
-
-  <script src="/assets/js/jquery-1.9.1.min.js"></script> 
-  <script src="/assets/js/bootstrap.min.js"></script> 
-  <script src="/assets/js/main.js"></script>
-  </body>
-
-</html>

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_layouts/default.html
----------------------------------------------------------------------
diff --git a/website/_layouts/default.html b/website/_layouts/default.html
deleted file mode 100644
index de1e3ba..0000000
--- a/website/_layouts/default.html
+++ /dev/null
@@ -1,48 +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.
--->
-
-<!DOCTYPE html>
-<html>
-
-  {% include head.html %}
-
-  <body>
-
-    {% include header.html %}
-<main id="main" >
-  <div class="container" >
-    <div id="zero" class=" main" >
-
-    <div class="page-content">
-      <div class="wrapper">
-        {{ content }}
-      </div>
-    </div>
-
-    </div>
-  </div>
-</main>
-
-    {% include footer.html %}
-
-  <script src="/assets/js/jquery-1.9.1.min.js"></script> 
-  <script src="/assets/js/bootstrap.min.js"></script> 
-  <script src="/assets/js/main.js"></script>
-  </body>
-
-</html>

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_layouts/dev.html
----------------------------------------------------------------------
diff --git a/website/_layouts/dev.html b/website/_layouts/dev.html
deleted file mode 100644
index cd8c070..0000000
--- a/website/_layouts/dev.html
+++ /dev/null
@@ -1,46 +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.
--->
-
-<!doctype html>
-<html>
-	{% include head.html %}
-	<body>
-		{% include header.html %}
-		
-		<div class="container">
-			<div class="row">
-				{% include dev_nav.html %}
-				<div class="col-md-9 col-lg-9 col-xs-14 aside2">
-					<div id="container">
-						<div id="pjax">
-							<h1 class="post-title">{{ page.title }}</h1>
-							<article class="post-content" >
-							{{ content }}
-							</article>
-						</div>
-					</div>
-				</div>
-			</div>
-		</div>		
-		{% include footer.html %}
-
-	<script src="/assets/js/jquery-1.9.1.min.js"></script> 
-	<script src="/assets/js/bootstrap.min.js"></script> 
-	<script src="/assets/js/main.js"></script>
-	</body>
-</html>

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_layouts/docs-cn.html
----------------------------------------------------------------------
diff --git a/website/_layouts/docs-cn.html b/website/_layouts/docs-cn.html
deleted file mode 100755
index 5f13608..0000000
--- a/website/_layouts/docs-cn.html
+++ /dev/null
@@ -1,46 +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.
--->
-
-<!doctype html>
-<html>
-	{% include head.cn.html %}
-	<body>
-		{% include header.cn.html %}
-		
-		<div class="container">
-			<div class="row">
-				{% include docs_nav.cn.html %}
-				<div class="col-md-9 col-lg-9 col-xs-14 aside2">
-					<div id="container">
-						<div id="pjax">
-							<h1 class="post-title">{{ page.title }}</h1>
-							<article class="post-content" >
-							{{ content }}
-							</article>
-						</div>
-					</div>
-				</div>
-			</div>
-		</div>		
-		{% include footer.html %}
-
-	<script src="/assets/js/jquery-1.9.1.min.js"></script> 
-	<script src="/assets/js/bootstrap.min.js"></script> 
-	<script src="/assets/js/main.js"></script>
-	</body>
-</html>

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_layouts/docs.html
----------------------------------------------------------------------
diff --git a/website/_layouts/docs.html b/website/_layouts/docs.html
deleted file mode 100755
index 0aa9b63..0000000
--- a/website/_layouts/docs.html
+++ /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.
--->
-
-<!doctype html>
-<html>
-	{% include head.html %}
-	<body>
-		{% include header.html %}
-		
-		<div class="container">
-			<div class="row">
-				{% include docs_nav.html %}
-				<div class="col-md-9 col-lg-9 col-xs-14 aside2">
-					<div id="container">
-						<div id="pjax">
-							<h1 class="post-title">{{ page.title }}</h1>
-							{% if page.version == NULL %}
-							{% else %}							
-								<p>version: {{page.version}}, since: {{page.since}}</p>
-							{% endif %}
-							<article class="post-content" >	
-							{{ content }}
-							</article>
-						</div>
-					</div>
-				</div>
-			</div>
-		</div>		
-		{% include footer.html %}
-
-	<script src="/assets/js/jquery-1.9.1.min.js"></script> 
-	<script src="/assets/js/bootstrap.min.js"></script> 
-	<script src="/assets/js/main.js"></script>
-	</body>
-</html>

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_layouts/download-cn.html
----------------------------------------------------------------------
diff --git a/website/_layouts/download-cn.html b/website/_layouts/download-cn.html
deleted file mode 100644
index adc5068..0000000
--- a/website/_layouts/download-cn.html
+++ /dev/null
@@ -1,53 +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.
--->
-
-<!DOCTYPE html>
-<html>
-
-  {% include head.cn.html %}
-
-  <body>
-
-    {% include header.cn.html %}
-
-    <div class="page-content">
-      <div class="wrapper">
-        <main id="main" >
-          <div class="container" >
-            <div id="zero" class=" main" >
-              <header style=" padding:2em 0 0 0">
-                <div class="container" >
-                  <h4 class="section-title"><span>Apache Kylin™ 相关下载</span></h4>
-                </div>
-              </header>
-                {{ content }}
-            </div>
-          </div>
-        </main>
-
-      </div>
-    </div>
-
-    {% include footer.html %}
-
-  <script src="/assets/js/jquery-1.9.1.min.js"></script> 
-  <script src="/assets/js/bootstrap.min.js"></script> 
-  <script src="/assets/js/main.js"></script>
-  </body>
-
-</html>

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_layouts/download.html
----------------------------------------------------------------------
diff --git a/website/_layouts/download.html b/website/_layouts/download.html
deleted file mode 100644
index 93dfd0d..0000000
--- a/website/_layouts/download.html
+++ /dev/null
@@ -1,53 +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.
--->
-
-<!DOCTYPE html>
-<html>
-
-  {% include head.html %}
-
-  <body>
-
-    {% include header.html %}
-
-    <div class="page-content">
-      <div class="wrapper">
-        <main id="main" >
-          <div class="container" >
-            <div id="zero" class=" main" >
-              <header style=" padding:2em 0 0 0">
-                <div class="container" >
-                  <h4 class="section-title"><span>Apache Kylin™ Download</span></h4>
-                </div>
-              </header>
-                {{ content }}
-            </div>
-          </div>
-        </main>
-
-      </div>
-    </div>
-
-    {% include footer.html %}
-
-  <script src="/assets/js/jquery-1.9.1.min.js"></script> 
-  <script src="/assets/js/bootstrap.min.js"></script> 
-  <script src="/assets/js/main.js"></script>
-  </body>
-
-</html>

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_layouts/page.html
----------------------------------------------------------------------
diff --git a/website/_layouts/page.html b/website/_layouts/page.html
deleted file mode 100644
index 9b38fb8..0000000
--- a/website/_layouts/page.html
+++ /dev/null
@@ -1,33 +0,0 @@
----
-layout: default
----
-
-<!--
-* 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.
--->
-
-<div class="post">
-
-  <header class="post-header">
-    <h1 class="post-title">{{ page.title }}</h1>
-  </header>
-
-  <article class="post-content">
-    {{ content }}
-  </article>
-
-</div>

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_layouts/post-blog.html
----------------------------------------------------------------------
diff --git a/website/_layouts/post-blog.html b/website/_layouts/post-blog.html
deleted file mode 100644
index 6e311ec..0000000
--- a/website/_layouts/post-blog.html
+++ /dev/null
@@ -1,38 +0,0 @@
----
-layout: blog
----
-
-<!--
-* 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.
--->
-
-<div class="post" style=" padding:2em 4em 4em 4em">
-
-  <header class="post-header">
-    <h1 class="post-title">{{ page.title }}</h1>
-    <p class="post-meta" >{{ page.date | date: "%b %-d, %Y" }}{% if page.author %} • {{ page.author }}{% endif %}{% if page.meta %} • {{ page.meta }}{% endif %}</p>
-  </header>
-
-  <article class="post-content" >
-    {{ content }}
-  </article>
-
-</div>
-
-
-
-

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_layouts/post-cn.html
----------------------------------------------------------------------
diff --git a/website/_layouts/post-cn.html b/website/_layouts/post-cn.html
deleted file mode 100644
index cba1250..0000000
--- a/website/_layouts/post-cn.html
+++ /dev/null
@@ -1,38 +0,0 @@
----
-layout: docs-cn
----
-
-<!--
-* 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.
--->
-
-<div class="post" style=" padding:1.5em 4em 4em 4em">
-
-  <header class="post-header">
-    <h1 class="post-title">{{ page.title }}</h1>
-    <p class="post-meta" >{{ page.date | date: "%b %-d, %Y" }}{% if page.author %} • {{ page.author }}{% endif %}{% if page.meta %} • {{ page.meta }}{% endif %}</p>
-  </header>
-
-  <article class="post-content" >
-    {{ content }}
-  </article>
-
-</div>
-
-
-
-

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_layouts/post.html
----------------------------------------------------------------------
diff --git a/website/_layouts/post.html b/website/_layouts/post.html
deleted file mode 100644
index 19732c9..0000000
--- a/website/_layouts/post.html
+++ /dev/null
@@ -1,38 +0,0 @@
----
-layout: docs
----
-
-<!--
-* 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.
--->
-
-<div class="post" style=" padding:1.5em 4em 4em 4em">
-
-  <header class="post-header">
-    <h1 class="post-title">{{ page.title }}</h1>
-    <p class="post-meta" >{{ page.date | date: "%b %-d, %Y" }}{% if page.author %} • {{ page.author }}{% endif %}{% if page.meta %} • {{ page.meta }}{% endif %}</p>
-  </header>
-
-  <article class="post-content" >
-    {{ content }}
-  </article>
-
-</div>
-
-
-
-


[41/52] [abbrv] kylin git commit: KYLIN-1416 remove website from coding branch

Posted by li...@apache.org.
http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_docs/install/advance_settings.md
----------------------------------------------------------------------
diff --git a/website/_docs/install/advance_settings.md b/website/_docs/install/advance_settings.md
deleted file mode 100644
index 3d3cae1..0000000
--- a/website/_docs/install/advance_settings.md
+++ /dev/null
@@ -1,45 +0,0 @@
----
-layout: docs
-title:  "Advance Settings of Kylin Environment"
-categories: install
-permalink: /docs/install/advance_settings.html
-version: v0.7.2
-since: v0.7.1
----
-
-## Enable LZO compression
-
-By default Kylin leverages snappy compression to compress the output of MR jobs, as well as hbase table storage, reducing the storage overhead. We do not choose LZO compression in Kylin because hadoop venders tend to not include LZO in their distributions due to license(GPL) issues. To enable LZO in Kylin, follow these steps:
-
-#### Make sure LZO is working in your environment
-
-We have a simple tool to test whether LZO is well installed on EVERY SERVER in hbase cluster ( http://docs.hortonworks.com/HDPDocuments/HDP2/HDP-2.2.4/bk_installing_manually_book/content/ch_install_hdfs_yarn_chapter.html#install-snappy-man-install ), and restart the cluster.
-To test it on the hadoop CLI that you deployed Kylin, Just run
-
-{% highlight Groff markup %}
-hbase org.apache.hadoop.hbase.util.CompressionTest file:///PATH-TO-A-LOCAL-TMP-FILE lzo
-{% endhighlight %}
-
-If no exception is printed, you're good to go. Otherwise you'll need to first install LZO properly on this server.
-To test if the hbase cluster is ready to create LZO compressed tables, test following hbase command:
-
-{% highlight Groff markup %}
-create 'lzoTable', {NAME => 'colFam',COMPRESSION => 'LZO'}
-{% endhighlight %}
-
-#### Use LZO for HBase compression
-
-You'll need to stop Kylin first by running `./kylin.sh stop`, and then modify $KYLIN_HOME/conf/kylin_job_conf.xml by uncommenting some configuration entries related to LZO compression. 
-After this, you need to run `./kylin.sh start` to start Kylin again. Now Kylin will use LZO to compress MR outputs and hbase tables.
-
-Goto $KYLIN_HOME/conf/kylin.properties, change kylin.hbase.default.compression.codec=snappy to kylin.hbase.default.compression.codec=lzo
-
-#### Use LZO for MR jobs
-
-Modify $KYLIN_HOME/conf/kylin_job_conf.xml by changing all org.apache.hadoop.io.compress.SnappyCodec to com.hadoop.compression.lzo.LzoCodec. 
-
-Start Kylin again. Now Kylin will use LZO to compress MR outputs and HBase tables.
-
-## Enable LDAP or SSO authentication
-
-Check [How to Enable Security with LDAP and SSO](../howto/howto_ldap_and_sso.html)

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_docs/install/hadoop_evn.md
----------------------------------------------------------------------
diff --git a/website/_docs/install/hadoop_evn.md b/website/_docs/install/hadoop_evn.md
deleted file mode 100644
index c67bb50..0000000
--- a/website/_docs/install/hadoop_evn.md
+++ /dev/null
@@ -1,35 +0,0 @@
----
-layout: docs
-title:  "Hadoop Environment"
-categories: install
-permalink: /docs/install/hadoop_env.html
-version: v0.7.2
-since: v0.7.1
----
-
-## Hadoop Environment
-
-Kylin requires you having access to a hadoop CLI, where you have full permissions to hdfs, hive, hbase and map-reduce. To make things easier we strongly recommend you starting with running Kylin on a hadoop sandbox, like <http://hortonworks.com/products/hortonworks-sandbox/>. In the following tutorial we'll go with **Hortonworks Sandbox 2.1** and **Cloudera QuickStart VM 5.1**. 
-
-To avoid permission issue, we suggest you using `root` account. The password for **Hortonworks Sandbox 2.1** is `hadoop` , for **Cloudera QuickStart VM 5.1** is `cloudera`.
-
-We also suggest you using bridged mode instead of NAT mode in your virtual box settings. Bridged mode will assign your sandbox an independent IP so that you can avoid issues like https://github.com/KylinOLAP/Kylin/issues/12
-
-### Start Hadoop
-
-Please make sure Hive, HDFS and HBase are available on our CLI machine.
-If you don't know how, here's a simple tutorial for hortonworks sanbox:
-
-Use ambari helps to launch hadoop:
-
-ambari-agent start
-ambari-server start
-	
-With both command successfully run you can go to ambari homepage at <http://your_sandbox_ip:8080> (user:admin,password:admin) to check everything's status. **By default hortonworks ambari disables Hbase, you'll need manually start the `Hbase` service at ambari homepage.**
-
-![start hbase in ambari](https://raw.githubusercontent.com/KylinOLAP/kylinolap.github.io/master/docs/installation/starthbase.png)
-
-**Additonal Info for setting up HortonWorks Sandbox on Virtual Box**
-
-	Please make sure Hbase Master port [Default 60000] and Zookeeper [Default 2181] is forwarded to Host OS.
- 
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_docs/install/index.cn.md
----------------------------------------------------------------------
diff --git a/website/_docs/install/index.cn.md b/website/_docs/install/index.cn.md
deleted file mode 100644
index ca61282..0000000
--- a/website/_docs/install/index.cn.md
+++ /dev/null
@@ -1,46 +0,0 @@
----
-layout: docs
-title:  "Installation Guide"
-categories: install
-permalink: /docs/install/index.html
-version: v0.7.2
-since: v0.7.1
----
-
-### Environment
-
-Kylin requires a properly setup hadoop environment to run. Following are the minimal request to run Kylin, for more detial, please check this reference: [Hadoop Environment](hadoop_env.html).
-
-## Prerequisites on Hadoop
-
-* Hadoop: 2.4+
-* Hive: 0.13+
-* HBase: 0.98+
-* JDK: 1.7+  
-_Tested with Hortonworks HDP 2.2 and Cloudera Quickstart VM 5.1_
-
-
-It is most common to install Kylin on a Hadoop client machine. It can be used for demo use, or for those who want to host their own web site to provide Kylin service. The scenario is depicted as:
-
-![On-Hadoop-CLI-installation](/images/install/on_cli_install_scene.png)
-
-For normal use cases, the application in the above picture means Kylin Web, which contains a web interface for cube building, querying and all sorts of management. Kylin Web launches a query engine for querying and a cube build engine for building cubes. These two engines interact with the Hadoop components, like hive and hbase.
-
-Except for some prerequisite software installations, the core of Kylin installation is accomplished by running a single script. After running the script, you will be able to build sample cube and query the tables behind the cubes via a unified web interface.
-
-### Install Kylin
-
-1. Download latest Kylin binaries at [http://kylin.apache.org/download](http://kylin.apache.org/download)
-2. Export KYLIN_HOME pointing to the extracted Kylin folder
-3. Make sure the user has the privilege to run hadoop, hive and hbase cmd in shell. If you are not so sure, you can run **bin/check-env.sh**, it will print out the detail information if you have some environment issues.
-4. To start Kylin, simply run **bin/kylin.sh start**
-5. To stop Kylin, simply run **bin/kylin.sh stop**
-
-> If you want to have multiple Kylin nodes please refer to [this](kylin_cluster.html)
-
-After Kylin started you can visit <http://your_hostname:7070/kylin>. The username/password is ADMIN/KYLIN. It's a clean Kylin homepage with nothing in there. To start with you can:
-
-1. [Quick play with a sample cube](../tutorial/kylin_sample.html)
-2. [Create and Build your own cube](../tutorial/create_cube.html)
-3. [Kylin Web Tutorial](../tutorial/web.html)
-

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_docs/install/index.md
----------------------------------------------------------------------
diff --git a/website/_docs/install/index.md b/website/_docs/install/index.md
deleted file mode 100644
index 1ee0af8..0000000
--- a/website/_docs/install/index.md
+++ /dev/null
@@ -1,47 +0,0 @@
----
-layout: docs
-title:  "Installation Guide"
-categories: install
-permalink: /docs/install/index.html
-version: v0.7.2
-since: v0.7.1
----
-
-### Environment
-
-Kylin requires a properly setup hadoop environment to run. Following are the minimal request to run Kylin, for more detial, please check this reference: [Hadoop Environment](hadoop_env.html).
-
-## Recommended Hadoop Versions
-
-* Hadoop: 2.4 - 2.7
-* Hive: 0.13 - 0.14
-* HBase: 0.98 - 0.99
-* JDK: 1.7+
-
-_Tested with Hortonworks HDP 2.2 and Cloudera Quickstart VM 5.1_
-
-
-It is most common to install Kylin on a Hadoop client machine. It can be used for demo use, or for those who want to host their own web site to provide Kylin service. The scenario is depicted as:
-
-![On-Hadoop-CLI-installation](/images/install/on_cli_install_scene.png)
-
-For normal use cases, the application in the above picture means Kylin Web, which contains a web interface for cube building, querying and all sorts of management. Kylin Web launches a query engine for querying and a cube build engine for building cubes. These two engines interact with the Hadoop components, like hive and hbase.
-
-Except for some prerequisite software installations, the core of Kylin installation is accomplished by running a single script. After running the script, you will be able to build sample cube and query the tables behind the cubes via a unified web interface.
-
-### Install Kylin
-
-1. Download latest Kylin binaries at [http://kylin.apache.org/download](http://kylin.apache.org/download)
-2. Export KYLIN_HOME pointing to the extracted Kylin folder
-3. Make sure the user has the privilege to run hadoop, hive and hbase cmd in shell. If you are not so sure, you can run **bin/check-env.sh**, it will print out the detail information if you have some environment issues.
-4. To start Kylin, simply run **bin/kylin.sh start**
-5. To stop Kylin, simply run **bin/kylin.sh stop**
-
-> If you want to have multiple Kylin nodes please refer to [this](kylin_cluster.html)
-
-After Kylin started you can visit <http://your_hostname:7070/kylin>. The username/password is ADMIN/KYLIN. It's a clean Kylin homepage with nothing in there. To start with you can:
-
-1. [Quick play with a sample cube](../tutorial/kylin_sample.html)
-2. [Create and Build your own cube](../tutorial/create_cube.html)
-3. [Kylin Web Tutorial](../tutorial/web.html)
-

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_docs/install/kylin_cluster.md
----------------------------------------------------------------------
diff --git a/website/_docs/install/kylin_cluster.md b/website/_docs/install/kylin_cluster.md
deleted file mode 100644
index 8dfd094..0000000
--- a/website/_docs/install/kylin_cluster.md
+++ /dev/null
@@ -1,30 +0,0 @@
----
-layout: docs
-title:  "Multiple Kylin REST servers"
-categories: install
-permalink: /docs/install/kylin_cluster.html
-version: v0.7.2
-since: v0.7.1
----
-
-
-### Kylin Server modes
-
-Kylin instances are stateless,  the runtime state is saved in its "Metadata Store" in hbase (kylin.metadata.url config in conf/kylin.properties). For load balance considerations it is possible to start multiple Kylin instances sharing the same metadata store (thus sharing the same state on table schemas, job status, cube status, etc.)
-
-Each of the kylin instances has a kylin.server.mode entry in conf/kylin.properties specifying the runtime mode, it has three options: 1. "job" for running job engine only 2. "query" for running query engine only and 3 "all" for running both. Notice that only one server can run the job engine("all" mode or "job" mode), the others must all be "query" mode.
-
-A typical scenario is depicted in the following chart:
-
-![]( /images/install/kylin_server_modes.png)
-
-### Setting up Multiple Kylin REST servers
-
-If you are running Kylin in a cluster or you have multiple Kylin REST server instances, please make sure you have the following property correctly configured in ${KYLIN_HOME}/conf/kylin.properties
-
-1. kylin.rest.servers 
-	List of web servers in use, this enables one web server instance to sync up with other servers. For example: kylin.rest.servers=sandbox1:7070,sandbox2:7070
-  
-2. kylin.server.mode
-	Make sure there is only one instance whose "kylin.server.mode" is set to "all" if there are multiple instances.
-	
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_docs/install/kylin_docker.md
----------------------------------------------------------------------
diff --git a/website/_docs/install/kylin_docker.md b/website/_docs/install/kylin_docker.md
deleted file mode 100644
index 1861d08..0000000
--- a/website/_docs/install/kylin_docker.md
+++ /dev/null
@@ -1,46 +0,0 @@
----
-layout: docs
-title:  "On Hadoop Kylin installation using Docker"
-categories: install
-permalink: /docs/install/kylin_docker.html
-version: v0.6
-since: v0.6
----
-
-With help of SequenceIQ, we have put together a fully automated method of creating a Kylin cluster (along with Hadoop, HBase and Hive). The only thing you will need to do is to pull the container from the official Docker repository by using the commands listed below:
-
-### Pre-Requisite
-
-1. Docker (If you don't have Docker installed, follow this [link](https://docs.docker.com/installation/#installation))
-2. Minimum RAM - 4Gb (We'll be running Kylin, Hadoop, HBase & Hive)
-
-### Installation
-{% highlight Groff markup %}
-docker pull sequenceiq/kylin:0.7.2
-{% endhighlight %}
-
-Once the container is pulled you are ready to start playing with Kylin. Get the following helper functions from our Kylin GitHub [repository](https://github.com/sequenceiq/docker-kylin/blob/master/ambari-functions) - _(make sure you source it)._
-
-{% highlight Groff markup %}
- $ wget https://raw.githubusercontent.com/sequenceiq/docker-kylin/master/ambari-functions
- $ source ambari-functions
-{% endhighlight %}
-{% highlight Groff markup %}
- $ kylin-deploy-cluster 1
-{% endhighlight %}
-
-You can specify the number of nodes you'd like to have in your cluster (1 in this case). Once we installed all the necessary Hadoop
-services we'll build Kylin on top of it and then you can reach the UI on: 
-{% highlight Groff markup %}
-#Ambari Dashboard
-http://<container_ip>:8080
-{% endhighlight %}
-Use `admin/admin` to login. Make sure HBase is running. 
-
-{% highlight Groff markup %}
-#Kylin Dashboard
-http://<container_ip>:7070/kylin
-{% endhighlight %}
-The default credentials to login are: `ADMIN:KYLIN`. 
-The cluster is pre-populated with sample data and is ready to build cubes as shown [here](../tutorial/create_cube.html).
-  

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_docs/install/manual_install_guide.cn.md
----------------------------------------------------------------------
diff --git a/website/_docs/install/manual_install_guide.cn.md b/website/_docs/install/manual_install_guide.cn.md
deleted file mode 100644
index a6e6c6b..0000000
--- a/website/_docs/install/manual_install_guide.cn.md
+++ /dev/null
@@ -1,48 +0,0 @@
----
-layout: docs-cn
-title:  "手动安装指南"
-categories: 安装
-permalink: /cn/docs/install/manual_install_guide.html
-version: v0.7.2
-since: v0.7.1
----
-
-## 引言
-
-在大多数情况下,我们的自动脚本[Installation Guide](./index.html)可以帮助你在你的hadoop sandbox甚至你的hadoop cluster中启动Kylin。但是,为防部署脚本出错,我们撰写本文作为参考指南来解决你的问题。
-
-基本上本文解释了自动脚本中的每一步骤。我们假设你已经对Linux上的Hadoop操作非常熟悉。
-
-## 前提条件
-* 已安装Tomcat,输出到CATALINA_HOME(with CATALINA_HOME exported). 
-* Kylin 二进制文件拷贝至本地并解压,之后使用$KYLIN_HOME引用
-
-## 步骤
-
-### 准备Jars
-
-Kylin会需要使用两个jar包,两个jar包和配置在默认kylin.properties:(there two jars and configured in the default kylin.properties)
-
-```
-kylin.job.jar=/tmp/kylin/kylin-job-latest.jar
-
-```
-
-这是Kylin用于MR jobs的job jar包。你需要复制 $KYLIN_HOME/job/target/kylin-job-latest.jar 到 /tmp/kylin/
-
-```
-kylin.coprocessor.local.jar=/tmp/kylin/kylin-coprocessor-latest.jar
-
-```
-
-这是一个Kylin会放在hbase上的hbase协处理jar包。它用于提高性能。你需要复制 $KYLIN_HOME/storage/target/kylin-coprocessor-latest.jar 到 /tmp/kylin/
-
-### 启动Kylin
-
-以`./kylin.sh start`
-
-启动Kylin
-
-并以`./Kylin.sh stop`
-
-停止Kylin

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_docs/install/manual_install_guide.md
----------------------------------------------------------------------
diff --git a/website/_docs/install/manual_install_guide.md b/website/_docs/install/manual_install_guide.md
deleted file mode 100644
index f75ce7e..0000000
--- a/website/_docs/install/manual_install_guide.md
+++ /dev/null
@@ -1,48 +0,0 @@
----
-layout: docs
-title:  Manual Installation Guide
-categories: install
-permalink: /docs/install/manual_install_guide.html
-version: v0.7.2
-since: v0.7.1
----
-
-## INTRODUCTION
-
-In most cases our automated script [Installation Guide](index.html) can help you launch Kylin in your hadoop sandbox and even your hadoop cluster. However, in case something went wrong in the deploy script, this article comes as an reference guide to fix your issues.
-
-Basically this article explains every step in the automatic script. We assume that you are already very familiar with Hadoop operations on Linux. 
-
-## PREREQUISITES
-* Tomcat installed, with CATALINA_HOME exported. 
-* Kylin binary pacakge copied to local and setup $KYLIN_HOME correctly
-
-## STEPS
-
-### 4. Prepare Jars
-
-There are two jars that Kylin will need to use, there two jars and configured in the default kylin.properties:
-
-```
-kylin.job.jar=/tmp/kylin/kylin-job-latest.jar
-
-```
-
-This is job jar that Kylin uses for MR jobs. You need to copy $KYLIN_HOME/job/target/kylin-job-latest.jar to /tmp/kylin/
-
-```
-kylin.coprocessor.local.jar=/tmp/kylin/kylin-coprocessor-latest.jar
-
-```
-
-This is a hbase coprocessor jar that Kylin will put on hbase. It is used for performance boosting. You need to copy $KYLIN_HOME/storage/target/kylin-coprocessor-latest.jar to /tmp/kylin/
-
-### 5. Start Kylin
-
-Start Kylin with
-
-`./kylin.sh start`
-
-and stop Kylin with
-
-`./Kylin.sh stop`

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_docs/release_notes.md
----------------------------------------------------------------------
diff --git a/website/_docs/release_notes.md b/website/_docs/release_notes.md
deleted file mode 100644
index 3adea81..0000000
--- a/website/_docs/release_notes.md
+++ /dev/null
@@ -1,706 +0,0 @@
----
-layout: docs
-title:  Apache Kylin™ Release Notes
-categories: gettingstarted
-permalink: /docs/release_notes.html
-version: v2.0
-since: v0.7.1
----
-
-To download latest release, please visit: [http://kylin.apache.org/download/](http://kylin.apache.org/download/), 
-there are source code package, binary package, ODBC driver and installation guide avaliable.
-
-Any problem or issue, please report to Apache Kylin JIRA project: [https://issues.apache.org/jira/browse/KYLIN](https://issues.apache.org/jira/browse/KYLIN)
-
-or send to Apache Kylin mailing list:   
-* User relative: [user@kylin.apache.org](mailto:user@kylin.apache.org)
-* Development relative: [dev@kylin.apache.org](mailto:dev@kylin.apache.org)
-
-
-## v2.0-alpha - 2016-02-09
-_Tag:_ [kylin-2.0-alpha](https://github.com/apache/kylin/tree/kylin-2.0-alpha)
-
-__Highlights__
-
-    * [KYLIN-875] - A plugin-able architecture, to allow alternative cube engine / storage engine / data source.
-    * [KYLIN-1245] - A better MR cubing algorithm, about 1.5 times faster than 1.x by comparing hundreds of jobs.
-    * [KYLIN-942] - A better storage engine, makes query roughly 2 times faster (especially for slow queries) than 1.x by comparing tens of thousands sqls.
-    * [KYLIN-738] - Streaming cubing EXPERIMENTAL support, source from kafka, build cube in-mem at minutes interval
-    * [KYLIN-943] - TopN pre-calculation (more UDFs coming)
-    * [KYLIN-1065] - ODBC compatible with Tableau 9.1, MS Excel, MS PowerBI
-    * [KYLIN-1219] - Kylin support SSO with Spring SAML
-
-__Below generated from JIRA system, pending manual revision.__
-
-__New Feature__
-
-    * [KYLIN-196] - Support Job Priority
-    * [KYLIN-528] - Build job flow for Inverted Index building
-    * [KYLIN-596] - Support Excel and Power BI
-    * [KYLIN-599] - Near real-time support
-    * [KYLIN-603] - Add mem store for seconds data latency
-    * [KYLIN-606] - Block level index for Inverted-Index
-    * [KYLIN-607] - More efficient cube building
-    * [KYLIN-609] - Add Hybrid as a federation of Cube and Inverted-index realization
-    * [KYLIN-625] - Create GridTable, a data structure that abstracts vertical and horizontal partition of a table
-    * [KYLIN-728] - IGTStore implementation which use disk when memory runs short
-    * [KYLIN-738] - StreamingOLAP
-    * [KYLIN-749] - support timestamp type in II and cube
-    * [KYLIN-774] - Automatically merge cube segments
-    * [KYLIN-868] - add a metadata backup/restore script in bin folder
-    * [KYLIN-886] - Data Retention for streaming data
-    * [KYLIN-906] - cube retention
-    * [KYLIN-943] - Approximate TopN supported by Cube
-    * [KYLIN-986] - Generalize Streaming scripts and put them into code repository 
-    * [KYLIN-1219] - Kylin support SSO with Spring SAML
-    * [KYLIN-1277] - Upgrade tool to put old-version cube and new-version cube into a hybrid model 
-
-__Improvement__
-
-    * [KYLIN-225] - Support edit "cost" of cube
-    * [KYLIN-589] - Cleanup Intermediate hive table after cube build
-    * [KYLIN-623] - update Kylin UI Style to latest AdminLTE
-    * [KYLIN-633] - Support Timestamp for cube partition
-    * [KYLIN-649] -  move the cache layer from service tier back to storage tier
-    * [KYLIN-655] - Migrate cube storage (query side) to use GridTable API
-    * [KYLIN-663] - Push time condition down to ii endpoint
-    * [KYLIN-668] - Out of memory in mapper when building cube in mem
-    * [KYLIN-671] - Implement fine grained cache for cube and ii
-    * [KYLIN-673] - Performance tuning for In-Mem cubing
-    * [KYLIN-674] - IIEndpoint return metrics as well
-    * [KYLIN-675] - cube&model designer refactor
-    * [KYLIN-678] - optimize RowKeyColumnIO
-    * [KYLIN-697] - Reorganize all test cases to unit test and integration tests
-    * [KYLIN-702] - When Kylin create the flat hive table, it generates large number of small files in HDFS 
-    * [KYLIN-708] - replace BitSet for AggrKey
-    * [KYLIN-712] - some enhancement after code review
-    * [KYLIN-717] - optimize OLAPEnumerator.convertCurrentRow()
-    * [KYLIN-718] - replace aliasMap in storage context with a clear specified return column list
-    * [KYLIN-719] - bundle statistics info in endpoint response
-    * [KYLIN-720] - Optimize endpoint's response structure to suit with no-dictionary data
-    * [KYLIN-721] - streaming cli support third-party streammessage parser
-    * [KYLIN-726] - add remote cli port configuration for KylinConfig
-    * [KYLIN-729] - IIEndpoint eliminate the non-aggregate routine
-    * [KYLIN-734] - Push cache layer to each storage engine
-    * [KYLIN-752] - Improved IN clause performance
-    * [KYLIN-753] - Make the dependency on hbase-common to "provided"
-    * [KYLIN-755] - extract copying libs from prepare.sh so that it can be reused
-    * [KYLIN-760] - Improve the hasing performance in Sampling cuboid size
-    * [KYLIN-772] - Continue cube job when hive query return empty resultset
-    * [KYLIN-773] - performance is slow list jobs
-    * [KYLIN-783] - update hdp version in test cases to 2.2.4
-    * [KYLIN-796] - Add REST API to trigger storage cleanup/GC
-    * [KYLIN-809] - Streaming cubing allow multiple kafka clusters/topics
-    * [KYLIN-816] - Allow gap in cube segments, for streaming case
-    * [KYLIN-822] - list cube overview in one page
-    * [KYLIN-823] - replace fk on fact table on rowkey & aggregation group generate
-    * [KYLIN-838] - improve performance of job query
-    * [KYLIN-844] - add backdoor toggles to control query behavior 
-    * [KYLIN-845] - Enable coprocessor even when there is memory hungry distinct count
-    * [KYLIN-858] - add snappy compression support
-    * [KYLIN-866] - Confirm with user when he selects empty segments to merge
-    * [KYLIN-869] - Enhance mail notification
-    * [KYLIN-870] - Speed up hbase segments info by caching
-    * [KYLIN-871] - growing dictionary for streaming case
-    * [KYLIN-874] - script for fill streaming gap automatically
-    * [KYLIN-875] - Decouple with Hadoop to allow alternative Input / Build Engine / Storage
-    * [KYLIN-879] - add a tool to collect orphan hbases 
-    * [KYLIN-880] -  Kylin should change the default folder from /tmp to user configurable destination
-    * [KYLIN-881] - Upgrade Calcite to 1.3.0
-    * [KYLIN-882] - check access to kylin.hdfs.working.dir
-    * [KYLIN-883] - Using configurable option for Hive intermediate tables created by Kylin job
-    * [KYLIN-893] - Remove the dependency on quartz and metrics
-    * [KYLIN-895] - Add "retention_range" attribute for cube instance, and automatically drop the oldest segment when exceeds retention
-    * [KYLIN-896] - Clean ODBC code, add them into main repository and write docs to help compiling
-    * [KYLIN-901] - Add tool for cleanup Kylin metadata storage
-    * [KYLIN-902] - move streaming related parameters into StreamingConfig
-    * [KYLIN-903] - automate metadata cleanup job
-    * [KYLIN-909] - Adapt GTStore to hbase endpoint
-    * [KYLIN-919] - more friendly UI for 0.8
-    * [KYLIN-922] - Enforce same code style for both intellij and eclipse user
-    * [KYLIN-926] - Make sure Kylin leaves no garbage files in local OS and HDFS/HBASE
-    * [KYLIN-927] - Real time cubes merging skipping gaps
-    * [KYLIN-933] - friendly UI to use data model
-    * [KYLIN-938] - add friendly tip to page when rest request failed
-    * [KYLIN-942] - Cube parallel scan on Hbase
-    * [KYLIN-956] - Allow users to configure hbase compression algorithm in kylin.properties
-    * [KYLIN-957] - Support HBase in a separate cluster
-    * [KYLIN-960] - Split storage module to core-storage and storage-hbase
-    * [KYLIN-973] - add a tool to analyse streaming output logs
-    * [KYLIN-984] - Behavior change in streaming data consuming
-    * [KYLIN-987] - Rename 0.7-staging and 0.8 branch
-    * [KYLIN-1014] - Support kerberos authentication while getting status from RM
-    * [KYLIN-1018] - make TimedJsonStreamParser default parser 
-    * [KYLIN-1019] - Remove v1 cube model classes from code repository
-    * [KYLIN-1021] - upload dependent jars of kylin to HDFS and set tmpjars
-    * [KYLIN-1025] - Save cube change is very slow
-    * [KYLIN-1036] - Code Clean, remove code which never used at front end
-    * [KYLIN-1041] - ADD Streaming UI 
-    * [KYLIN-1048] - CPU and memory killer in Cuboid.findById()
-    * [KYLIN-1058] - Remove "right join" during model creation
-    * [KYLIN-1061] - "kylin.sh start" should check whether kylin has already been running
-    * [KYLIN-1064] - restore disabled queries in KylinQueryTest.testVerifyQuery
-    * [KYLIN-1065] - ODBC driver support tableau 9.1
-    * [KYLIN-1068] - Optimize the memory footprint for TopN counter
-    * [KYLIN-1069] - update tip for 'Partition Column' on UI
-    * [KYLIN-1095] - Update AdminLTE to latest version
-    * [KYLIN-1096] - Deprecate minicluster in 2.x staging
-    * [KYLIN-1099] - Support dictionary of cardinality over 10 millions
-    * [KYLIN-1101] - Allow "YYYYMMDD" as a date partition column
-    * [KYLIN-1105] - Cache in AbstractRowKeyEncoder.createInstance() is useless
-    * [KYLIN-1116] - Use local dictionary for InvertedIndex batch building
-    * [KYLIN-1119] - refine find-hive-dependency.sh to correctly get hcatalog path
-    * [KYLIN-1126] - v2 storage(for parallel scan) backward compatibility with v1 storage
-    * [KYLIN-1135] - Pscan use share thread pool
-    * [KYLIN-1136] - Distinguish fast build mode and complete build mode
-    * [KYLIN-1139] - Hive job not starting due to error "conflicting lock present for default mode EXCLUSIVE "
-    * [KYLIN-1149] - When yarn return an incomplete job tracking URL, Kylin will fail to get job status
-    * [KYLIN-1154] - Load job page is very slow when there are a lot of history job
-    * [KYLIN-1157] - CubeMigrationCLI doesn't copy ACL
-    * [KYLIN-1160] - Set default logger appender of log4j for JDBC
-    * [KYLIN-1161] - Rest API /api/cubes?cubeName=  is doing fuzzy match instead of exact match
-    * [KYLIN-1162] - Enhance HadoopStatusGetter to be compatible with YARN-2605
-    * [KYLIN-1190] - Make memory budget per query configurable
-    * [KYLIN-1234] - Cube ACL does not work
-    * [KYLIN-1235] - allow user to select dimension column as options when edit COUNT_DISTINCT measure
-    * [KYLIN-1237] - Revisit on cube size estimation
-    * [KYLIN-1239] - attribute each htable with team contact and owner name
-    * [KYLIN-1244] - In query window, enable fast copy&paste by double clicking tables/columns' names.
-    * [KYLIN-1245] - Switch between layer cubing and in-mem cubing according to stats
-    * [KYLIN-1246] - get cubes API update - offset,limit not required
-    * [KYLIN-1251] - add toggle event for tree label
-    * [KYLIN-1259] - Change font/background color of job progress
-    * [KYLIN-1265] - Make sure 2.0 query is no slower than 1.0
-    * [KYLIN-1266] - Tune 2.0 release package size
-    * [KYLIN-1267] - Check Kryo performance when spilling aggregation cache
-    * [KYLIN-1268] - Fix 2 kylin logs
-    * [KYLIN-1270] - improve TimedJsonStreamParser to support month_start,quarter_start,year_start
-    * [KYLIN-1281] - Add "partition_date_end", and move "partition_date_start" into cube descriptor
-    * [KYLIN-1283] - Replace GTScanRequest's SerDer form Kryo to manual 
-    * [KYLIN-1287] - UI update for streaming build action
-    * [KYLIN-1297] - Diagnose query performance issues in 2.x versions
-    * [KYLIN-1301] - fix segment pruning failure in 2.x versions
-    * [KYLIN-1308] - query storage v2 enable parallel cube visiting
-    * [KYLIN-1312] - Enhance DeployCoprocessorCLI to support Cube level filter
-    * [KYLIN-1318] - enable gc log for kylin server instance
-    * [KYLIN-1323] - Improve performance of converting data to hfile
-    * [KYLIN-1327] - Tool for batch updating host information of htables
-    * [KYLIN-1334] - allow truncating string for fixed length dimensions
-    * [KYLIN-1341] - Display JSON of Data Model in the dialog
-    * [KYLIN-1350] - hbase Result.binarySearch is found to be problematic in concurrent environments
-    * [KYLIN-1368] - JDBC Driver is not generic to restAPI json result
-
-__Bug__
-
-    * [KYLIN-404] - Can't get cube source record size.
-    * [KYLIN-457] - log4j error and dup lines in kylin.log
-    * [KYLIN-521] - No verification even if join condition is invalid
-    * [KYLIN-632] - "kylin.sh stop" doesn't check whether KYLIN_HOME was set
-    * [KYLIN-635] - IN clause within CASE when is not working
-    * [KYLIN-656] - REST API get cube desc NullPointerException when cube is not exists
-    * [KYLIN-660] - Make configurable of dictionary cardinality cap
-    * [KYLIN-665] - buffer error while in mem cubing
-    * [KYLIN-688] - possible memory leak for segmentIterator
-    * [KYLIN-731] - Parallel stream build will throw OOM
-    * [KYLIN-740] - Slowness with many IN() values
-    * [KYLIN-747] - bad query performance when IN clause contains a value doesn't exist in the dictionary
-    * [KYLIN-748] - II returned result not correct when decimal omits precision and scal
-    * [KYLIN-751] - Max on negative double values is not working
-    * [KYLIN-766] - round BigDecimal according to the DataType scale
-    * [KYLIN-769] - empty segment build fail due to no dictionary 
-    * [KYLIN-771] - query cache is not evicted when metadata changes
-    * [KYLIN-778] - can't build cube after package to binary 
-    * [KYLIN-780] - Upgrade Calcite to 1.0
-    * [KYLIN-797] - Cuboid cache will cache massive invalid cuboid if existed many cubes which already be deleted 
-    * [KYLIN-801] - fix remaining issues on query cache and storage cache
-    * [KYLIN-805] - Drop useless Hive intermediate table and HBase tables in the last step of cube build/merge
-    * [KYLIN-807] - Avoid write conflict between job engine and stream cube builder
-    * [KYLIN-817] - Support Extract() on timestamp column
-    * [KYLIN-824] - Cube Build fails if lookup table doesn't have any files under HDFS location
-    * [KYLIN-828] - kylin still use ldap profile when comment the line "kylin.sandbox=false" in kylin.properties
-    * [KYLIN-834] - optimize StreamingUtil binary search perf
-    * [KYLIN-837] - fix submit build type when refresh cube
-    * [KYLIN-873] - cancel button does not work when [resume][discard] job
-    * [KYLIN-889] - Support more than one HDFS files of lookup table
-    * [KYLIN-897] - Update CubeMigrationCLI to copy data model info
-    * [KYLIN-898] - "CUBOID_CACHE" in Cuboid.java never flushes
-    * [KYLIN-905] - Boolean type not supported
-    * [KYLIN-911] - NEW segments not DELETED when cancel BuildAndMerge Job
-    * [KYLIN-912] - $KYLIN_HOME/tomcat/temp folder takes much disk space after long run
-    * [KYLIN-913] - Cannot find rowkey column XXX in cube CubeDesc
-    * [KYLIN-914] - Scripts shebang should use /bin/bash
-    * [KYLIN-918] - Calcite throws "java.lang.Float cannot be cast to java.lang.Double" error while executing SQL
-    * [KYLIN-929] - can not sort cubes by [Source Records] at cubes list page
-    * [KYLIN-930] - can't see realizations under each project at project list page
-    * [KYLIN-934] - Negative number in SUM result and Kylin results not matching exactly Hive results
-    * [KYLIN-935] - always loading when try to view the log of the sub-step of cube build job
-    * [KYLIN-936] - can not see job step log 
-    * [KYLIN-944] - update doc about how to consume kylin API in javascript
-    * [KYLIN-946] - [UI] refresh page show no results when Project selected as [--Select All--]
-    * [KYLIN-950] - Web UI "Jobs" tab view the job reduplicated
-    * [KYLIN-951] - Drop RowBlock concept from GridTable general API
-    * [KYLIN-952] - User can trigger a Refresh job on an non-existing cube segment via REST API
-    * [KYLIN-967] - Dump running queries on memory shortage
-    * [KYLIN-975] - change kylin.job.hive.database.for.intermediatetable cause job to fail
-    * [KYLIN-978] - GarbageCollectionStep dropped Hive Intermediate Table but didn't drop external hdfs path
-    * [KYLIN-982] - package.sh should grep out "Download*" messages when determining version
-    * [KYLIN-983] - Query sql offset keyword bug
-    * [KYLIN-985] - Don't suppoprt aggregation AVG while executing SQL
-    * [KYLIN-991] - StorageCleanupJob may clean a newly created HTable in streaming cube building
-    * [KYLIN-992] - ConcurrentModificationException when initializing ResourceStore
-    * [KYLIN-1001] - Kylin generates wrong HDFS path in creating intermediate table
-    * [KYLIN-1004] - Dictionary with '' value cause cube merge to fail
-    * [KYLIN-1020] - Although "kylin.query.scan.threshold" is set, it still be restricted to less than 4 million 
-    * [KYLIN-1026] - Error message for git check is not correct in package.sh
-    * [KYLIN-1027] - HBase Token not added after KYLIN-1007
-    * [KYLIN-1033] - Error when joining two sub-queries
-    * [KYLIN-1039] - Filter like (A or false) yields wrong result
-    * [KYLIN-1047] - Upgrade to Calcite 1.4
-    * [KYLIN-1066] - Only 1 reducer is started in the "Build cube" step of MR_Engine_V2
-    * [KYLIN-1067] - Support get MapReduce Job status for ResourceManager HA Env
-    * [KYLIN-1075] - select [MeasureCol] from [FactTbl] is not supported
-    * [KYLIN-1078] - UI - Cannot have comments in the end of New Query textbox
-    * [KYLIN-1093] - Consolidate getCurrentHBaseConfiguration() and newHBaseConfiguration() in HadoopUtil
-    * [KYLIN-1106] - Can not send email caused by Build Base Cuboid Data step failed
-    * [KYLIN-1108] - Return Type Empty When Measure-> Count In Cube Design
-    * [KYLIN-1113] - Support TopN query in v2/CubeStorageQuery.java
-    * [KYLIN-1115] - Clean up ODBC driver code
-    * [KYLIN-1121] - ResourceTool download/upload does not work in binary package
-    * [KYLIN-1127] - Refactor CacheService
-    * [KYLIN-1137] - TopN measure need support dictionary merge
-    * [KYLIN-1138] - Bad CubeDesc signature cause segment be delete when enable a cube
-    * [KYLIN-1140] - Kylin's sample cube "kylin_sales_cube" couldn't be saved.
-    * [KYLIN-1151] - Menu items should be aligned when create new model
-    * [KYLIN-1152] - ResourceStore should read content and timestamp in one go
-    * [KYLIN-1153] - Upgrade is needed for cubedesc metadata from 1.x to 2.0
-    * [KYLIN-1171] - KylinConfig truncate bug
-    * [KYLIN-1179] - Cannot use String as partition column
-    * [KYLIN-1180] - Some NPE in Dictionary
-    * [KYLIN-1181] - Split metadata size exceeded when data got huge in one segment
-    * [KYLIN-1192] - Cannot edit data model desc without name change
-    * [KYLIN-1205] - hbase RpcClient java.io.IOException: Unexpected closed connection
-    * [KYLIN-1211] - Add 'Enable Cache' button in System page
-    * [KYLIN-1216] - Can't parse DateFormat like 'YYYYMMDD' correctly in query
-    * [KYLIN-1218] - java.lang.NullPointerException in MeasureTypeFactory when sync hive table
-    * [KYLIN-1220] - JsonMappingException: Can not deserialize instance of java.lang.String out of START_ARRAY
-    * [KYLIN-1225] - Only 15 cubes listed in the /models page
-    * [KYLIN-1226] - InMemCubeBuilder throw OOM for multiple HLLC measures
-    * [KYLIN-1230] - When CubeMigrationCLI copied ACL from one env to another, it may not work
-    * [KYLIN-1236] - redirect to home page when input invalid url
-    * [KYLIN-1250] - Got NPE when discarding a job
-    * [KYLIN-1260] - Job status labels are not in same style
-    * [KYLIN-1269] - Can not get last error message in email
-    * [KYLIN-1271] - Create streaming table layer will disappear if click on outside
-    * [KYLIN-1274] - Query from JDBC is partial results by default
-    * [KYLIN-1282] - Comparison filter on Date/Time column not work for query
-    * [KYLIN-1289] - Click on subsequent wizard steps doesn't work when editing existing cube or model
-    * [KYLIN-1303] - Error when in-mem cubing on empty data source which has boolean columns
-    * [KYLIN-1306] - Null strings are not applied during fast cubing
-    * [KYLIN-1314] - Display issue for aggression groups 
-    * [KYLIN-1315] - UI: Cannot add normal dimension when creating new cube 
-    * [KYLIN-1316] - Wrong label in Dialog CUBE REFRESH CONFIRM
-    * [KYLIN-1317] - Kill underlying running hadoop job while discard a job
-    * [KYLIN-1328] - "UnsupportedOperationException" is thrown when remove a data model
-    * [KYLIN-1330] - UI create model: Press enter will go back to pre step
-    * [KYLIN-1336] - 404 errors of model page and api 'access/DataModelDesc' in console
-    * [KYLIN-1337] - Sort cube name doesn't work well 
-    * [KYLIN-1346] - IllegalStateException happens in SparkCubing
-    * [KYLIN-1347] - UI: cannot place cursor in front of the last dimension
-    * [KYLIN-1349] - 'undefined' is logged in console when adding lookup table
-    * [KYLIN-1352] - 'Cache already exists' exception in high-concurrency query situation
-    * [KYLIN-1356] - use exec-maven-plugin for IT environment provision
-    * [KYLIN-1357] - Cloned cube has build time information
-    * [KYLIN-1372] - Query using PrepareStatement failed with multi OR clause
-    * [KYLIN-1382] - CubeMigrationCLI reports error when migrate cube
-    * [KYLIN-1396] - minor bug in BigDecimalSerializer - avoidVerbose should be incremented each time when input scale is larger than given scale 
-    * [KYLIN-1400] - kylin.metadata.url with hbase namespace problem
-    * [KYLIN-1402] - StringIndexOutOfBoundsException in Kylin Hive Column Cardinality Job
-    * [KYLIN-1414] - Couldn't drag and drop rowkey, js error is thrown in browser console
-
-
-## v1.2 - 2015-12-15
-_Tag:_ [kylin-1.2](https://github.com/apache/kylin/tree/kylin-1.2)
-
-__New Feature__
-
-    * [KYLIN-596] - Support Excel and Power BI
-    
-__Improvement__
-
-    * [KYLIN-389] - Can't edit cube name for existing cubes
-    * [KYLIN-702] - When Kylin create the flat hive table, it generates large number of small files in HDFS 
-    * [KYLIN-1021] - upload dependent jars of kylin to HDFS and set tmpjars
-    * [KYLIN-1058] - Remove "right join" during model creation
-    * [KYLIN-1064] - restore disabled queries in KylinQueryTest.testVerifyQuery
-    * [KYLIN-1065] - ODBC driver support tableau 9.1
-    * [KYLIN-1069] - update tip for 'Partition Column' on UI
-    * [KYLIN-1081] - ./bin/find-hive-dependency.sh may not find hive-hcatalog-core.jar
-    * [KYLIN-1095] - Update AdminLTE to latest version
-    * [KYLIN-1099] - Support dictionary of cardinality over 10 millions
-    * [KYLIN-1101] - Allow "YYYYMMDD" as a date partition column
-    * [KYLIN-1105] - Cache in AbstractRowKeyEncoder.createInstance() is useless
-    * [KYLIN-1119] - refine find-hive-dependency.sh to correctly get hcatalog path
-    * [KYLIN-1139] - Hive job not starting due to error "conflicting lock present for default mode EXCLUSIVE "
-    * [KYLIN-1149] - When yarn return an incomplete job tracking URL, Kylin will fail to get job status
-    * [KYLIN-1154] - Load job page is very slow when there are a lot of history job
-    * [KYLIN-1157] - CubeMigrationCLI doesn't copy ACL
-    * [KYLIN-1160] - Set default logger appender of log4j for JDBC
-    * [KYLIN-1161] - Rest API /api/cubes?cubeName=  is doing fuzzy match instead of exact match
-    * [KYLIN-1162] - Enhance HadoopStatusGetter to be compatible with YARN-2605
-    * [KYLIN-1166] - CubeMigrationCLI should disable and purge the cube in source store after be migrated
-    * [KYLIN-1168] - Couldn't save cube after doing some modification, get "Update data model is not allowed! Please create a new cube if needed" error
-    * [KYLIN-1190] - Make memory budget per query configurable
-
-__Bug__
-
-    * [KYLIN-693] - Couldn't change a cube's name after it be created
-    * [KYLIN-930] - can't see realizations under each project at project list page
-    * [KYLIN-966] - When user creates a cube, if enter a name which already exists, Kylin will thrown expection on last step
-    * [KYLIN-1033] - Error when joining two sub-queries
-    * [KYLIN-1039] - Filter like (A or false) yields wrong result
-    * [KYLIN-1067] - Support get MapReduce Job status for ResourceManager HA Env
-    * [KYLIN-1070] - changing  case in table name in  model desc
-    * [KYLIN-1093] - Consolidate getCurrentHBaseConfiguration() and newHBaseConfiguration() in HadoopUtil
-    * [KYLIN-1098] - two "kylin.hbase.region.count.min" in conf/kylin.properties
-    * [KYLIN-1106] - Can not send email caused by Build Base Cuboid Data step failed
-    * [KYLIN-1108] - Return Type Empty When Measure-> Count In Cube Design
-    * [KYLIN-1120] - MapReduce job read local meta issue
-    * [KYLIN-1121] - ResourceTool download/upload does not work in binary package
-    * [KYLIN-1140] - Kylin's sample cube "kylin_sales_cube" couldn't be saved.
-    * [KYLIN-1148] - Edit project's name and cancel edit, project's name still modified
-    * [KYLIN-1152] - ResourceStore should read content and timestamp in one go
-    * [KYLIN-1155] - unit test with minicluster doesn't work on 1.x
-    * [KYLIN-1203] - Cannot save cube after correcting the configuration mistake
-    * [KYLIN-1205] - hbase RpcClient java.io.IOException: Unexpected closed connection
-    * [KYLIN-1216] - Can't parse DateFormat like 'YYYYMMDD' correctly in query
-
-__Task__
-
-    * [KYLIN-1170] - Update website and status files to TLP
-
-
-## v1.1.1-incubating - 2015-11-04
-_Tag:_ [kylin-1.1.1-incubating](https://github.com/apache/kylin/tree/kylin-1.1.1-incubating)
-
-__Improvement__
-
-    * [KYLIN-999] - License check and cleanup for release
-
-## v1.1-incubating - 2015-10-25
-_Tag:_ [kylin-1.1-incubating](https://github.com/apache/kylin/tree/kylin-1.1-incubating)
-
-__New Feature__
-
-    * [KYLIN-222] - Web UI to Display CubeInstance Information
-    * [KYLIN-906] - cube retention
-    * [KYLIN-910] - Allow user to enter "retention range" in days on Cube UI
-
-__Bug__
-
-    * [KYLIN-457] - log4j error and dup lines in kylin.log
-    * [KYLIN-632] - "kylin.sh stop" doesn't check whether KYLIN_HOME was set
-    * [KYLIN-740] - Slowness with many IN() values
-    * [KYLIN-747] - bad query performance when IN clause contains a value doesn't exist in the dictionary
-    * [KYLIN-771] - query cache is not evicted when metadata changes
-    * [KYLIN-797] - Cuboid cache will cache massive invalid cuboid if existed many cubes which already be deleted 
-    * [KYLIN-847] - "select * from fact" does not work on 0.7 branch
-    * [KYLIN-913] - Cannot find rowkey column XXX in cube CubeDesc
-    * [KYLIN-918] - Calcite throws "java.lang.Float cannot be cast to java.lang.Double" error while executing SQL
-    * [KYLIN-944] - update doc about how to consume kylin API in javascript
-    * [KYLIN-950] - Web UI "Jobs" tab view the job reduplicated
-    * [KYLIN-952] - User can trigger a Refresh job on an non-existing cube segment via REST API
-    * [KYLIN-958] - update cube data model may fail and leave metadata in inconsistent state
-    * [KYLIN-961] - Can't get cube  source record count.
-    * [KYLIN-967] - Dump running queries on memory shortage
-    * [KYLIN-968] - CubeSegment.lastBuildJobID is null in new instance but used for rowkey_stats path
-    * [KYLIN-975] - change kylin.job.hive.database.for.intermediatetable cause job to fail
-    * [KYLIN-978] - GarbageCollectionStep dropped Hive Intermediate Table but didn't drop external hdfs path
-    * [KYLIN-982] - package.sh should grep out "Download*" messages when determining version
-    * [KYLIN-983] - Query sql offset keyword bug
-    * [KYLIN-985] - Don't suppoprt aggregation AVG while executing SQL
-    * [KYLIN-1001] - Kylin generates wrong HDFS path in creating intermediate table
-    * [KYLIN-1004] - Dictionary with '' value cause cube merge to fail
-    * [KYLIN-1005] - fail to acquire ZookeeperJobLock when hbase.zookeeper.property.clientPort is configured other than 2181
-    * [KYLIN-1015] - Hive dependency jars appeared twice on job configuration
-    * [KYLIN-1020] - Although "kylin.query.scan.threshold" is set, it still be restricted to less than 4 million 
-    * [KYLIN-1026] - Error message for git check is not correct in package.sh
-
-__Improvement__
-
-    * [KYLIN-343] - Enable timeout on query 
-    * [KYLIN-367] - automatically backup metadata everyday
-    * [KYLIN-589] - Cleanup Intermediate hive table after cube build
-    * [KYLIN-772] - Continue cube job when hive query return empty resultset
-    * [KYLIN-858] - add snappy compression support
-    * [KYLIN-882] - check access to kylin.hdfs.working.dir
-    * [KYLIN-895] - Add "retention_range" attribute for cube instance, and automatically drop the oldest segment when exceeds retention
-    * [KYLIN-901] - Add tool for cleanup Kylin metadata storage
-    * [KYLIN-956] - Allow users to configure hbase compression algorithm in kylin.properties
-    * [KYLIN-957] - Support HBase in a separate cluster
-    * [KYLIN-965] - Allow user to configure the region split size for cube
-    * [KYLIN-971] - kylin display timezone on UI
-    * [KYLIN-987] - Rename 0.7-staging and 0.8 branch
-    * [KYLIN-998] - Finish the hive intermediate table clean up job in org.apache.kylin.job.hadoop.cube.StorageCleanupJob
-    * [KYLIN-999] - License check and cleanup for release
-    * [KYLIN-1013] - Make hbase client configurations like timeout configurable
-    * [KYLIN-1025] - Save cube change is very slow
-    * [KYLIN-1034] - Faster bitmap indexes with Roaring bitmaps
-    * [KYLIN-1035] - Validate [Project] before create Cube on UI
-    * [KYLIN-1037] - Remove hardcoded "hdp.version" from regression tests
-    * [KYLIN-1047] - Upgrade to Calcite 1.4
-    * [KYLIN-1048] - CPU and memory killer in Cuboid.findById()
-    * [KYLIN-1061] - "kylin.sh start" should check whether kylin has already been running
-    * [KYLIN-1048] - CPU and memory killer in Cuboid.findById()
-    * [KYLIN-1061] - "kylin.sh start" should check whether kylin has already been running
-
-
-## v1.0-incubating - 2015-09-06
-_Tag:_ [kylin-1.0-incubating](https://github.com/apache/kylin/tree/kylin-1.0-incubating)
-
-__New Feature__
-
-    * [KYLIN-591] - Leverage Zeppelin to interactive with Kylin
-
-__Bug__
-
-    * [KYLIN-404] - Can't get cube source record size.
-    * [KYLIN-626] - JDBC error for float and double values
-    * [KYLIN-751] - Max on negative double values is not working
-    * [KYLIN-757] - Cache wasn't flushed in cluster mode
-    * [KYLIN-780] - Upgrade Calcite to 1.0
-    * [KYLIN-805] - Drop useless Hive intermediate table and HBase tables in the last step of cube build/merge
-    * [KYLIN-889] - Support more than one HDFS files of lookup table
-    * [KYLIN-897] - Update CubeMigrationCLI to copy data model info
-    * [KYLIN-898] - "CUBOID_CACHE" in Cuboid.java never flushes
-    * [KYLIN-911] - NEW segments not DELETED when cancel BuildAndMerge Job
-    * [KYLIN-912] - $KYLIN_HOME/tomcat/temp folder takes much disk space after long run
-    * [KYLIN-914] - Scripts shebang should use /bin/bash
-    * [KYLIN-915] - appendDBName in CubeMetadataUpgrade will return null
-    * [KYLIN-921] - Dimension with all nulls cause BuildDimensionDictionary failed due to FileNotFoundException
-    * [KYLIN-923] - FetcherRunner will never run again if encountered exception during running
-    * [KYLIN-929] - can not sort cubes by [Source Records] at cubes list page
-    * [KYLIN-934] - Negative number in SUM result and Kylin results not matching exactly Hive results
-    * [KYLIN-935] - always loading when try to view the log of the sub-step of cube build job
-    * [KYLIN-936] - can not see job step log 
-    * [KYLIN-940] - NPE when close the null resouce
-    * [KYLIN-945] - Kylin JDBC - Get Connection from DataSource results in NullPointerException
-    * [KYLIN-946] - [UI] refresh page show no results when Project selected as [--Select All--]
-    * [KYLIN-949] - Query cache doesn't work properly for prepareStatement queries
-
-__Improvement__
-
-    * [KYLIN-568] - job support stop/suspend function so that users can manually resume a job
-    * [KYLIN-717] - optimize OLAPEnumerator.convertCurrentRow()
-    * [KYLIN-792] - kylin performance insight [dashboard]
-    * [KYLIN-838] - improve performance of job query
-    * [KYLIN-842] - Add version and commit id into binary package
-    * [KYLIN-844] - add backdoor toggles to control query behavior 
-    * [KYLIN-857] - backport coprocessor improvement in 0.8 to 0.7
-    * [KYLIN-866] - Confirm with user when he selects empty segments to merge
-    * [KYLIN-867] - Hybrid model for multiple realizations/cubes
-    * [KYLIN-880] -  Kylin should change the default folder from /tmp to user configurable destination
-    * [KYLIN-881] - Upgrade Calcite to 1.3.0
-    * [KYLIN-883] - Using configurable option for Hive intermediate tables created by Kylin job
-    * [KYLIN-893] - Remove the dependency on quartz and metrics
-    * [KYLIN-922] - Enforce same code style for both intellij and eclipse user
-    * [KYLIN-926] - Make sure Kylin leaves no garbage files in local OS and HDFS/HBASE
-    * [KYLIN-933] - friendly UI to use data model
-    * [KYLIN-938] - add friendly tip to page when rest request failed
-
-__Task__
-
-    * [KYLIN-884] - Restructure docs and website
-    * [KYLIN-907] - Improve Kylin community development experience
-    * [KYLIN-954] - Release v1.0 (formerly v0.7.3)
-    * [KYLIN-863] - create empty segment when there is no data in one single streaming batch
-    * [KYLIN-908] - Help community developer to setup develop/debug environment
-    * [KYLIN-931] - Port KYLIN-921 to 0.8 branch
-
-## v0.7.2-incubating - 2015-07-21
-_Tag:_ [kylin-0.7.2-incubating](https://github.com/apache/kylin/tree/kylin-0.7.2-incubating)
-
-__Main Changes:__  
-Critical bug fixes after v0.7.1 release, please go with this version directly for new case and upgrade to this version for existing deployment.
-
-__Bug__  
-
-    * [KYLIN-514] - Error message is not helpful to user when doing something in Jason Editor window
-    * [KYLIN-598] - Kylin detecting hive table delim failure
-    * [KYLIN-660] - Make configurable of dictionary cardinality cap
-    * [KYLIN-765] - When a cube job is failed, still be possible to submit a new job
-    * [KYLIN-814] - Duplicate columns error for subqueries on fact table
-    * [KYLIN-819] - Fix necessary ColumnMetaData order for Calcite (Optic)
-    * [KYLIN-824] - Cube Build fails if lookup table doesn't have any files under HDFS location
-    * [KYLIN-829] - Cube "Actions" shows "NA"; but after expand the "access" tab, the button shows up
-    * [KYLIN-830] - Cube merge failed after migrating from v0.6 to v0.7
-    * [KYLIN-831] - Kylin report "Column 'ABC' not found in table 'TABLE' while executing SQL", when that column is FK but not define as a dimension
-    * [KYLIN-840] - HBase table compress not enabled even LZO is installed
-    * [KYLIN-848] - Couldn't resume or discard a cube job
-    * [KYLIN-849] - Couldn't query metrics on lookup table PK
-    * [KYLIN-865] - Cube has been built but couldn't query; In log it said "Realization 'CUBE.CUBE_NAME' defined under project PROJECT_NAME is not found
-    * [KYLIN-873] - cancel button does not work when [resume][discard] job
-    * [KYLIN-888] - "Jobs" page only shows 15 job at max, the "Load more" button was disappeared
-
-__Improvement__
-
-    * [KYLIN-159] - Metadata migrate tool 
-    * [KYLIN-199] - Validation Rule: Unique value of Lookup table's key columns
-    * [KYLIN-207] - Support SQL pagination
-    * [KYLIN-209] - Merge tail small MR jobs into one
-    * [KYLIN-210] - Split heavy MR job to more small jobs
-    * [KYLIN-221] - Convert cleanup and GC to job 
-    * [KYLIN-284] - add log for all Rest API Request
-    * [KYLIN-488] - Increase HDFS block size 1GB
-    * [KYLIN-600] - measure return type update
-    * [KYLIN-611] - Allow Implicit Joins
-    * [KYLIN-623] - update Kylin UI Style to latest AdminLTE
-    * [KYLIN-727] - Cube build in BuildCubeWithEngine does not cover incremental build/cube merge
-    * [KYLIN-752] - Improved IN clause performance
-    * [KYLIN-773] - performance is slow list jobs
-    * [KYLIN-839] - Optimize Snapshot table memory usage 
-
-__New Feature__
-
-    * [KYLIN-211] - Bitmap Inverted Index
-    * [KYLIN-285] - Enhance alert program for whole system
-    * [KYLIN-467] - Validataion Rule: Check duplicate rows in lookup table
-    * [KYLIN-471] - Support "Copy" on grid result
-
-__Task__
-
-    * [KYLIN-7] - Enable maven checkstyle plugin
-    * [KYLIN-885] - Release v0.7.2
-    * [KYLIN-812] - Upgrade to Calcite 0.9.2
-
-## v0.7.1-incubating (First Apache Release) - 2015-06-10  
-_Tag:_ [kylin-0.7.1-incubating](https://github.com/apache/kylin/tree/kylin-0.7.1-incubating)
-
-Apache Kylin v0.7.1-incubating has rolled out on June 10, 2015. This is also the first Apache release after join incubating. 
-
-__Main Changes:__
-
-* Package renamed from com.kylinolap to org.apache.kylin
-* Code cleaned up to apply Apache License policy
-* Easy install and setup with bunch of scripts and automation
-* Job engine refactor to be generic job manager for all jobs, and improved efficiency
-* Support Hive database other than 'default'
-* JDBC driver avaliable for client to interactive with Kylin server
-* Binary pacakge avaliable download 
-
-__New Feature__
-
-    * [KYLIN-327] - Binary distribution 
-    * [KYLIN-368] - Move MailService to Common module
-    * [KYLIN-540] - Data model upgrade for legacy cube descs
-    * [KYLIN-576] - Refactor expansion rate expression
-
-__Task__
-
-    * [KYLIN-361] - Rename package name with Apache Kylin
-    * [KYLIN-531] - Rename package name to org.apache.kylin
-    * [KYLIN-533] - Job Engine Refactoring
-    * [KYLIN-585] - Simplify deployment
-    * [KYLIN-586] - Add Apache License header in each source file
-    * [KYLIN-587] - Remove hard copy of javascript libraries
-    * [KYLIN-624] - Add dimension and metric info into DataModel
-    * [KYLIN-650] - Move all document from github wiki to code repository (using md file)
-    * [KYLIN-669] - Release v0.7.1 as first apache release
-    * [KYLIN-670] - Update pom with "incubating" in version number
-    * [KYLIN-737] - Generate and sign release package for review and vote
-    * [KYLIN-795] - Release after success vote
-
-__Bug__
-
-    * [KYLIN-132] - Job framework
-    * [KYLIN-194] - Dict & ColumnValueContainer does not support number comparison, they do string comparison right now
-    * [KYLIN-220] - Enable swap column of Rowkeys in Cube Designer
-    * [KYLIN-230] - Error when create HTable
-    * [KYLIN-255] - Error when a aggregated function appear twice in select clause
-    * [KYLIN-383] - Sample Hive EDW database name should be replaced by "default" in the sample
-    * [KYLIN-399] - refreshed segment not correctly published to cube
-    * [KYLIN-412] - No exception or message when sync up table which can't access
-    * [KYLIN-421] - Hive table metadata issue
-    * [KYLIN-436] - Can't sync Hive table metadata from other database rather than "default"
-    * [KYLIN-508] - Too high cardinality is not suitable for dictionary!
-    * [KYLIN-509] - Order by on fact table not works correctly
-    * [KYLIN-517] - Always delete the last one of Add Lookup page buttom even if deleting the first join condition
-    * [KYLIN-524] - Exception will throw out if dimension is created on a lookup table, then deleting the lookup table.
-    * [KYLIN-547] - Create cube failed if column dictionary sets false and column length value greater than 0
-    * [KYLIN-556] - error tip enhance when cube detail return empty
-    * [KYLIN-570] - Need not to call API before sending login request
-    * [KYLIN-571] - Dimensions lost when creating cube though Joson Editor
-    * [KYLIN-572] - HTable size is wrong
-    * [KYLIN-581] - unable to build cube
-    * [KYLIN-583] - Dependency of Hive conf/jar in II branch will affect auto deploy
-    * [KYLIN-588] - Error when run package.sh
-    * [KYLIN-593] - angular.min.js.map and angular-resource.min.js.map are missing in kylin.war
-    * [KYLIN-594] - Making changes in build and packaging with respect to apache release process
-    * [KYLIN-595] - Kylin JDBC driver should not assume Kylin server listen on either 80 or 443
-    * [KYLIN-605] - Issue when install Kylin on a CLI which does not have yarn Resource Manager
-    * [KYLIN-614] - find hive dependency shell fine is unable to set the hive dependency correctly
-    * [KYLIN-615] - Unable add measures in Kylin web UI
-    * [KYLIN-619] - Cube build fails with hive+tez
-    * [KYLIN-620] - Wrong duration number
-    * [KYLIN-621] - SecurityException when running MR job
-    * [KYLIN-627] - Hive tables' partition column was not sync into Kylin
-    * [KYLIN-628] - Couldn't build a new created cube
-    * [KYLIN-629] - Kylin failed to run mapreduce job if there is no mapreduce.application.classpath in mapred-site.xml
-    * [KYLIN-630] - ArrayIndexOutOfBoundsException when merge cube segments 
-    * [KYLIN-638] - kylin.sh stop not working
-    * [KYLIN-639] - Get "Table 'xxxx' not found while executing SQL" error after a cube be successfully built
-    * [KYLIN-640] - sum of float not working
-    * [KYLIN-642] - Couldn't refresh cube segment
-    * [KYLIN-643] - JDBC couldn't connect to Kylin: "java.sql.SQLException: Authentication Failed"
-    * [KYLIN-644] - join table as null error when build the cube
-    * [KYLIN-652] - Lookup table alias will be set to null
-    * [KYLIN-657] - JDBC Driver not register into DriverManager
-    * [KYLIN-658] - java.lang.IllegalArgumentException: Cannot find rowkey column XXX in cube CubeDesc
-    * [KYLIN-659] - Couldn't adjust the rowkey sequence when create cube
-    * [KYLIN-666] - Select float type column got class cast exception
-    * [KYLIN-681] - Failed to build dictionary if the rowkey's dictionary property is "date(yyyy-mm-dd)"
-    * [KYLIN-682] - Got "No aggregator for func 'MIN' and return type 'decimal(19,4)'" error when build cube
-    * [KYLIN-684] - Remove holistic distinct count and multiple column distinct count from sample cube
-    * [KYLIN-691] - update tomcat download address in download-tomcat.sh
-    * [KYLIN-696] - Dictionary couldn't recognize a value and throw IllegalArgumentException: "Not a valid value"
-    * [KYLIN-703] - UT failed due to unknown host issue
-    * [KYLIN-711] - UT failure in REST module
-    * [KYLIN-739] - Dimension as metrics does not work with PK-FK derived column
-    * [KYLIN-761] - Tables are not shown in the "Query" tab, and couldn't run SQL query after cube be built
-
-__Improvement__
-
-    * [KYLIN-168] - Installation fails if multiple ZK
-    * [KYLIN-182] - Validation Rule: columns used in Join condition should have same datatype
-    * [KYLIN-204] - Kylin web not works properly in IE
-    * [KYLIN-217] - Enhance coprocessor with endpoints 
-    * [KYLIN-251] - job engine refactoring
-    * [KYLIN-261] - derived column validate when create cube
-    * [KYLIN-317] - note: grunt.json need to be configured when add new javascript or css file
-    * [KYLIN-324] - Refactor metadata to support InvertedIndex
-    * [KYLIN-407] - Validation: There's should no Hive table column using "binary" data type
-    * [KYLIN-445] - Rename cube_desc/cube folder
-    * [KYLIN-452] - Automatically create local cluster for running tests
-    * [KYLIN-498] - Merge metadata tables 
-    * [KYLIN-532] - Refactor data model in kylin front end
-    * [KYLIN-539] - use hbase command to launch tomcat
-    * [KYLIN-542] - add project property feature for cube
-    * [KYLIN-553] - From cube instance, couldn't easily find the project instance that it belongs to
-    * [KYLIN-563] - Wrap kylin start and stop with a script 
-    * [KYLIN-567] - More flexible validation of new segments
-    * [KYLIN-569] - Support increment+merge job
-    * [KYLIN-578] - add more generic configuration for ssh
-    * [KYLIN-601] - Extract content from kylin.tgz to "kylin" folder
-    * [KYLIN-616] - Validation Rule: partition date column should be in dimension columns
-    * [KYLIN-634] - Script to import sample data and cube metadata
-    * [KYLIN-636] - wiki/On-Hadoop-CLI-installation is not up to date
-    * [KYLIN-637] - add start&end date for hbase info in cubeDesigner
-    * [KYLIN-714] - Add Apache RAT to pom.xml
-    * [KYLIN-753] - Make the dependency on hbase-common to "provided"
-    * [KYLIN-758] - Updating port forwarding issue Hadoop Installation on Hortonworks Sandbox.
-    * [KYLIN-779] - [UI] jump to cube list after create cube
-    * [KYLIN-796] - Add REST API to trigger storage cleanup/GC
-
-__Wish__
-
-    * [KYLIN-608] - Distinct count for ii storage
-

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_docs/tutorial/acl.cn.md
----------------------------------------------------------------------
diff --git a/website/_docs/tutorial/acl.cn.md b/website/_docs/tutorial/acl.cn.md
deleted file mode 100644
index 8145371..0000000
--- a/website/_docs/tutorial/acl.cn.md
+++ /dev/null
@@ -1,35 +0,0 @@
----
-layout: docs-cn
-title:  Kylin Cube 权限授予教程
-categories: 教程
-permalink: /cn/docs/tutorial/acl.html
-version: v1.2
-since: v0.7.1
----
-
-  
-
-在`Cubes`页面,双击cube行查看详细信息。在这里我们关注`Access`标签。
-点击`+Grant`按钮进行授权。
-
-![]( /images/Kylin-Cube-Permission-Grant-Tutorial/14 +grant.png)
-
-一个cube有四种不同的权限。将你的鼠标移动到`?`图标查看详细信息。
-
-![]( /images/Kylin-Cube-Permission-Grant-Tutorial/15 grantInfo.png)
-
-授权对象也有两种:`User`和`Role`。`Role`是指一组拥有同样权限的用户。
-
-### 1. 授予用户权限
-* 选择`User`类型,输入你想要授权的用户的用户名并选择相应的权限。
-
-     ![]( /images/Kylin-Cube-Permission-Grant-Tutorial/16 grant-user.png)
-
-* 然后点击`Grant`按钮提交请求。在这一操作成功后,你会在表中看到一个新的表项。你可以选择不同的访问权限来修改用户权限。点击`Revoke`按钮可以删除一个拥有权限的用户。
-
-     ![]( /images/Kylin-Cube-Permission-Grant-Tutorial/16 user-update.png)
-
-### 2. 授予角色权限
-* 选择`Role`类型,通过点击下拉按钮选择你想要授权的一组用户并选择一个权限。
-
-* 然后点击`Grant`按钮提交请求。在这一操作成功后,你会在表中看到一个新的表项。你可以选择不同的访问权限来修改组权限。点击`Revoke`按钮可以删除一个拥有权限的组。

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_docs/tutorial/acl.md
----------------------------------------------------------------------
diff --git a/website/_docs/tutorial/acl.md b/website/_docs/tutorial/acl.md
deleted file mode 100644
index abe923b..0000000
--- a/website/_docs/tutorial/acl.md
+++ /dev/null
@@ -1,35 +0,0 @@
----
-layout: docs
-title:  Kylin Cube Permission Grant Tutorial
-categories: tutorial
-permalink: /docs/tutorial/acl.html
-version: v1.2
-since: v0.7.1
----
-
-   
-
-In `Cubes` page, double click the cube row to see the detail information. Here we focus on the `Access` tab.
-Click the `+Grant` button to grant permission. 
-
-![](/images/Kylin-Cube-Permission-Grant-Tutorial/14 +grant.png)
-
-There are four different kinds of permissions for a cube. Move your mouse over the `?` icon to see detail information. 
-
-![](/images/Kylin-Cube-Permission-Grant-Tutorial/15 grantInfo.png)
-
-There are also two types of user that a permission can be granted: `User` and `Role`. `Role` means a group of users who have the same role.
-
-### 1. Grant User Permission
-* Select `User` type, enter the username of the user you want to grant and select the related permission. 
-
-     ![](/images/Kylin-Cube-Permission-Grant-Tutorial/16 grant-user.png)
-
-* Then click the `Grant` button to send a request. After the success of this operation, you will see a new table entry show in the table. You can select various permission of access to change the permission of a user. To delete a user with permission, just click the `Revoke` button.
-
-     ![](/images/Kylin-Cube-Permission-Grant-Tutorial/16 user-update.png)
-
-### 2. Grant Role Permission
-* Select `Role` type, choose a group of users that you want to grant by click the drop down button and select a permission.
-
-* Then click the `Grant` button to send a request. After the success of this operation, you will see a new table entry show in the table. You can select various permission of access to change the permission of a group. To delete a group with permission, just click the `Revoke` button.

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_docs/tutorial/create_cube.cn.md
----------------------------------------------------------------------
diff --git a/website/_docs/tutorial/create_cube.cn.md b/website/_docs/tutorial/create_cube.cn.md
deleted file mode 100644
index 3b44a3d..0000000
--- a/website/_docs/tutorial/create_cube.cn.md
+++ /dev/null
@@ -1,129 +0,0 @@
----
-layout: docs-cn
-title:  Kylin Cube 创建教程
-categories: 教程
-permalink: /cn/docs/tutorial/create_cube.html
-version: v1.2
-since: v0.7.1
----
-  
-  
-### I. 新建一个项目
-1. 由顶部菜单栏进入`Query`页面,然后点击`Manage Projects`。
-
-   ![](/images/Kylin-Cube-Creation-Tutorial/1 manage-prject.png)
-
-2. 点击`+ Project`按钮添加一个新的项目。
-
-   ![](/images/Kylin-Cube-Creation-Tutorial/2 %2Bproject.png)
-
-3. 填写下列表单并点击`submit`按钮提交请求。
-
-   ![](/images/Kylin-Cube-Creation-Tutorial/3 new-project.png)
-
-4. 成功后,底部会显示通知。
-
-   ![](/images/Kylin-Cube-Creation-Tutorial/3.1 pj-created.png)
-
-### II. 同步一张表
-1. 在顶部菜单栏点击`Tables`,然后点击`+ Sync`按钮加载hive表元数据。
-
-   ![](/images/Kylin-Cube-Creation-Tutorial/4 %2Btable.png)
-
-2. 输入表名并点击`Sync`按钮提交请求。
-
-   ![](/images/Kylin-Cube-Creation-Tutorial/5 hive-table.png)
-
-### III. 新建一个cube
-首先,在顶部菜单栏点击`Cubes`。然后点击`+Cube`按钮进入cube designer页面。
-
-![](/images/Kylin-Cube-Creation-Tutorial/6 %2Bcube.png)
-
-**步骤1. Cube信息**
-
-填写cube基本信息。点击`Next`进入下一步。
-
-你可以使用字母、数字和“_”来为你的cube命名(注意名字中不能使用空格)。
-
-![](/images/Kylin-Cube-Creation-Tutorial/7 cube-info.png)
-
-**步骤2. 维度**
-
-1. 建立事实表。
-
-    ![](/images/Kylin-Cube-Creation-Tutorial/8 dim-factable.png)
-
-2. 点击`+Dimension`按钮添加一个新的维度。
-
-    ![](/images/Kylin-Cube-Creation-Tutorial/8 dim-%2Bdim.png)
-
-3. 可以选择不同类型的维度加入一个cube。我们在这里列出其中一部分供你参考。
-
-    * 从事实表获取维度。
-          ![](/images/Kylin-Cube-Creation-Tutorial/8 dim-typeA.png)
-
-    * 从查找表获取维度。
-        ![]( /images/Kylin-Cube-Creation-Tutorial/8 dim-typeB-1.png)
-
-        ![]( /images/Kylin-Cube-Creation-Tutorial/8 dim-typeB-2.png)
-   
-    * 从有分级结构的查找表获取维度。
-          ![](/images/Kylin-Cube-Creation-Tutorial/8 dim-typeC.png)
-
-    * 从有衍生维度(derived dimensions)的查找表获取维度。
-          ![](/images/Kylin-Cube-Creation-Tutorial/8 dim-typeD.png)
-
-4. 用户可以在保存维度后进行编辑。
-   ![](/images/Kylin-Cube-Creation-Tutorial/8 dim-edit.png)
-
-**步骤3. 度量**
-
-1. 点击`+Measure`按钮添加一个新的度量。
-   ![](/images/Kylin-Cube-Creation-Tutorial/9 meas-%2Bmeas.png)
-
-2. 根据它的表达式共有5种不同类型的度量:`SUM`、`MAX`、`MIN`、`COUNT`和`COUNT_DISTINCT`。请谨慎选择返回类型,它与`COUNT(DISTINCT)`的误差率相关。
-   * SUM
-
-     ![](/images/Kylin-Cube-Creation-Tutorial/9 meas-sum.png)
-
-   * MIN
-
-     ![](/images/Kylin-Cube-Creation-Tutorial/9 meas-min.png)
-
-   * MAX
-
-     ![](/images/Kylin-Cube-Creation-Tutorial/9 meas-max.png)
-
-   * COUNT
-
-     ![](/images/Kylin-Cube-Creation-Tutorial/9 meas-count.png)
-
-   * DISTINCT_COUNT
-
-     ![](/images/Kylin-Cube-Creation-Tutorial/9 meas-distinct.png)
-
-**步骤4. 过滤器**
-
-这一步骤是可选的。你可以使用`SQL`格式添加一些条件过滤器。
-
-![](/images/Kylin-Cube-Creation-Tutorial/10 filter.png)
-
-**步骤5. 更新设置**
-
-这一步骤是为增量构建cube而设计的。
-
-![](/images/Kylin-Cube-Creation-Tutorial/11 refresh-setting1.png)
-
-选择分区类型、分区列和开始日期。
-
-![](/images/Kylin-Cube-Creation-Tutorial/11 refresh-setting2.png)
-
-**步骤6. 高级设置**
-
-![](/images/Kylin-Cube-Creation-Tutorial/12 advanced.png)
-
-**步骤7. 概览 & 保存**
-
-你可以概览你的cube并返回之前的步骤进行修改。点击`Save`按钮完成cube创建。
-
-![](/images/Kylin-Cube-Creation-Tutorial/13 overview.png)

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_docs/tutorial/create_cube.md
----------------------------------------------------------------------
diff --git a/website/_docs/tutorial/create_cube.md b/website/_docs/tutorial/create_cube.md
deleted file mode 100644
index 966cdd5..0000000
--- a/website/_docs/tutorial/create_cube.md
+++ /dev/null
@@ -1,129 +0,0 @@
----
-layout: docs
-title:  Kylin Cube Creation Tutorial
-categories: tutorial
-permalink: /docs/tutorial/create_cube.html
-version: v1.2
-since: v0.7.1
----
-  
-  
-### I. Create a Project
-1. Go to `Query` page in top menu bar, then click `Manage Projects`.
-
-   ![]( /images/Kylin-Cube-Creation-Tutorial/1 manage-prject.png)
-
-2. Click the `+ Project` button to add a new project.
-
-   ![]( /images/Kylin-Cube-Creation-Tutorial/2 +project.png)
-
-3. Fulfill the following form and click `submit` button to send a request.
-
-   ![]( /images/Kylin-Cube-Creation-Tutorial/3 new-project.png)
-
-4. After success, there will be a notification show in the bottom.
-
-   ![]( /images/Kylin-Cube-Creation-Tutorial/3.1 pj-created.png)
-
-### II. Sync up a Table
-1. Click `Tables` in top bar and then click the `+ Sync` button to load hive table metadata.
-
-   ![]( /images/Kylin-Cube-Creation-Tutorial/4 +table.png)
-
-2. Enter the table names and click `Sync` to send a request.
-
-   ![]( /images/Kylin-Cube-Creation-Tutorial/5 hive-table.png)
-
-### III. Create a Cube
-To start with, click `Cubes` in top bar.Then click `+Cube` button to enter the cube designer page.
-
-![]( /images/Kylin-Cube-Creation-Tutorial/6 +cube.png)
-
-**Step 1. Cube Info**
-
-Fill up the basic information of the cube. Click `Next` to enter the next step.
-
-You can use letters, numbers and '_' to name your cube (Notice that space in name is not allowed).
-
-![]( /images/Kylin-Cube-Creation-Tutorial/7 cube-info.png)
-
-**Step 2. Dimensions**
-
-1. Set up the fact table.
-
-    ![]( /images/Kylin-Cube-Creation-Tutorial/8 dim-factable.png)
-
-2. Click `+Dimension` to add a new dimension.
-
-    ![]( /images/Kylin-Cube-Creation-Tutorial/8 dim-+dim.png)
-
-3. There are different types of dimensions that might be added to a cube. Here we list some of them for your reference.
-
-    * Dimensions from fact table.
-        ![]( /images/Kylin-Cube-Creation-Tutorial/8 dim-typeA.png)
-
-    * Dimensions from look up table.
-        ![]( /images/Kylin-Cube-Creation-Tutorial/8 dim-typeB-1.png)
-
-        ![]( /images/Kylin-Cube-Creation-Tutorial/8 dim-typeB-2.png)
-   
-    * Dimensions from look up table with hierarchy.
-        ![]( /images/Kylin-Cube-Creation-Tutorial/8 dim-typeC.png)
-
-    * Dimensions from look up table with derived dimensions.
-        ![]( /images/Kylin-Cube-Creation-Tutorial/8 dim-typeD.png)
-
-4. User can edit the dimension after saving it.
-   ![]( /images/Kylin-Cube-Creation-Tutorial/8 dim-edit.png)
-
-**Step 3. Measures**
-
-1. Click the `+Measure` to add a new measure.
-   ![]( /images/Kylin-Cube-Creation-Tutorial/9 meas-+meas.png)
-
-2. There are 5 different types of measure according to its expression: `SUM`, `MAX`, `MIN`, `COUNT` and `COUNT_DISTINCT`. Please be  carefully to choose the return type, which is related to the error rate of the `COUNT(DISTINCT)`.
-   * SUM
-
-     ![]( /images/Kylin-Cube-Creation-Tutorial/9 meas-sum.png)
-
-   * MIN
-
-     ![]( /images/Kylin-Cube-Creation-Tutorial/9 meas-min.png)
-
-   * MAX
-
-     ![]( /images/Kylin-Cube-Creation-Tutorial/9 meas-max.png)
-
-   * COUNT
-
-     ![]( /images/Kylin-Cube-Creation-Tutorial/9 meas-count.png)
-
-   * DISTINCT_COUNT
-
-     ![]( /images/Kylin-Cube-Creation-Tutorial/9 meas-distinct.png)
-
-**Step 4. Filter**
-
-This step is optional. You can add some condition filter in `SQL` format.
-
-![]( /images/Kylin-Cube-Creation-Tutorial/10 filter.png)
-
-**Step 5. Refresh Setting**
-
-This step is designed for incremental cube build. 
-
-![]( /images/Kylin-Cube-Creation-Tutorial/11 refresh-setting1.png)
-
-Choose partition type, partition column and start date.
-
-![]( /images/Kylin-Cube-Creation-Tutorial/11 refresh-setting2.png)
-
-**Step 6. Advanced Setting**
-
-![]( /images/Kylin-Cube-Creation-Tutorial/12 advanced.png)
-
-**Step 7. Overview & Save**
-
-You can overview your cube and go back to previous step to modify it. Click the `Save` button to complete the cube creation.
-
-![]( /images/Kylin-Cube-Creation-Tutorial/13 overview.png)

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_docs/tutorial/cube_build_job.cn.md
----------------------------------------------------------------------
diff --git a/website/_docs/tutorial/cube_build_job.cn.md b/website/_docs/tutorial/cube_build_job.cn.md
deleted file mode 100644
index d689e8c..0000000
--- a/website/_docs/tutorial/cube_build_job.cn.md
+++ /dev/null
@@ -1,66 +0,0 @@
----
-layout: docs-cn
-title:  Kylin Cube 建立和Job监控教程
-categories: 教程
-permalink: /cn/docs/tutorial/cube_build_job.html
-version: v1.2
-since: v0.7.1
----
-
-### Cube建立
-首先,确认你拥有你想要建立的cube的权限。
-
-1. 在`Cubes`页面中,点击cube栏右侧的`Action`下拉按钮并选择`Build`操作。
-
-   ![]( /images/Kylin-Cube-Build-and-Job-Monitoring-Tutorial/1 action-build.png)
-
-2. 选择后会出现一个弹出窗口。
-
-   ![]( /images/Kylin-Cube-Build-and-Job-Monitoring-Tutorial/2 pop-up.png)
-
-3. 点击`END DATE`输入框选择增量构建这个cube的结束日期。
-
-   ![]( /images/Kylin-Cube-Build-and-Job-Monitoring-Tutorial/3 end-date.png)
-
-4. 点击`Submit`提交请求。
-
-   ![]( /images/Kylin-Cube-Build-and-Job-Monitoring-Tutorial/4 submit.png)
-
-   ![]( /images/Kylin-Cube-Build-and-Job-Monitoring-Tutorial/4.1 success.png)
-
-   提交请求成功后,你将会看到`Jobs`页面新建了job。
-
-   ![]( /images/Kylin-Cube-Build-and-Job-Monitoring-Tutorial/5 jobs-page.png)
-
-5. 如要放弃这个job,点击`Discard`按钮。
-
-   ![]( /images/Kylin-Cube-Build-and-Job-Monitoring-Tutorial/6 discard.png)
-
-### Job监控
-在`Jobs`页面,点击job详情按钮查看显示于右侧的详细信息。
-
-![]( /images/Kylin-Cube-Build-and-Job-Monitoring-Tutorial/7 job-steps.png)
-
-job详细信息为跟踪一个job提供了它的每一步记录。你可以将光标停放在一个步骤状态图标上查看基本状态和信息。
-
-![]( /images/Kylin-Cube-Build-and-Job-Monitoring-Tutorial/8 hover-step.png)
-
-点击每个步骤显示的图标按钮查看详情:`Parameters`、`Log`、`MRJob`、`EagleMonitoring`。
-
-* Parameters
-
-   ![]( /images/Kylin-Cube-Build-and-Job-Monitoring-Tutorial/9 parameters.png)
-
-   ![]( /images/Kylin-Cube-Build-and-Job-Monitoring-Tutorial/9 parameters-d.png)
-
-* Log
-        
-   ![]( /images/Kylin-Cube-Build-and-Job-Monitoring-Tutorial/9 log.png)
-
-   ![]( /images/Kylin-Cube-Build-and-Job-Monitoring-Tutorial/9 log-d.png)
-
-* MRJob(MapReduce Job)
-
-   ![]( /images/Kylin-Cube-Build-and-Job-Monitoring-Tutorial/9 mrjob.png)
-
-   ![]( /images/Kylin-Cube-Build-and-Job-Monitoring-Tutorial/9 mrjob-d.png)

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_docs/tutorial/cube_build_job.md
----------------------------------------------------------------------
diff --git a/website/_docs/tutorial/cube_build_job.md b/website/_docs/tutorial/cube_build_job.md
deleted file mode 100644
index a1b1584..0000000
--- a/website/_docs/tutorial/cube_build_job.md
+++ /dev/null
@@ -1,66 +0,0 @@
----
-layout: docs
-title:  Kylin Cube Build and Job Monitoring Tutorial
-categories: tutorial
-permalink: /docs/tutorial/cube_build_job.html
-version: v1.2
-since: v0.7.1
----
-
-### Cube Build
-First of all, make sure that you have authority of the cube you want to build.
-
-1. In `Cubes` page, click the `Action` drop down button in the right of a cube column and select operation `Build`.
-
-   ![](/images/Kylin-Cube-Build-and-Job-Monitoring-Tutorial/1 action-build.png)
-
-2. There is a pop-up window after the selection. 
-
-   ![](/images/Kylin-Cube-Build-and-Job-Monitoring-Tutorial/2 pop-up.png)
-
-3. Click `END DATE` input box to choose end date of this incremental cube build.
-
-   ![](/images/Kylin-Cube-Build-and-Job-Monitoring-Tutorial/3 end-date.png)
-
-4. Click `Submit` to send request. 
-
-   ![](/images/Kylin-Cube-Build-and-Job-Monitoring-Tutorial/4 submit.png)
-
-   ![](/images/Kylin-Cube-Build-and-Job-Monitoring-Tutorial/4.1 success.png)
-
-   After submit the request successfully, you will see the job just be created in the `Jobs` page.
-
-   ![](/images/Kylin-Cube-Build-and-Job-Monitoring-Tutorial/5 jobs-page.png)
-
-5. To discard this job, just click the `Discard` button.
-
-   ![](/images/Kylin-Cube-Build-and-Job-Monitoring-Tutorial/6 discard.png)
-
-### Job Monitoring
-In the `Jobs` page, click the job detail button to see detail information show in the right side.
-
-![](/images/Kylin-Cube-Build-and-Job-Monitoring-Tutorial/7 job-steps.png)
-
-The detail information of a job provides a step-by-step record to trace a job. You can hover a step status icon to see the basic status and information.
-
-![](/images/Kylin-Cube-Build-and-Job-Monitoring-Tutorial/8 hover-step.png)
-
-Click the icon button show in each step to see the details: `Parameters`, `Log`, `MRJob`, `EagleMonitoring`.
-
-* Parameters
-
-   ![](/images/Kylin-Cube-Build-and-Job-Monitoring-Tutorial/9 parameters.png)
-
-   ![](/images/Kylin-Cube-Build-and-Job-Monitoring-Tutorial/9 parameters-d.png)
-
-* Log
-        
-   ![](/images/Kylin-Cube-Build-and-Job-Monitoring-Tutorial/9 log.png)
-
-   ![](/images/Kylin-Cube-Build-and-Job-Monitoring-Tutorial/9 log-d.png)
-
-* MRJob(MapReduce Job)
-
-   ![](/images/Kylin-Cube-Build-and-Job-Monitoring-Tutorial/9 mrjob.png)
-
-   ![](/images/Kylin-Cube-Build-and-Job-Monitoring-Tutorial/9 mrjob-d.png)

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_docs/tutorial/kylin_sample.md
----------------------------------------------------------------------
diff --git a/website/_docs/tutorial/kylin_sample.md b/website/_docs/tutorial/kylin_sample.md
deleted file mode 100644
index a6cde53..0000000
--- a/website/_docs/tutorial/kylin_sample.md
+++ /dev/null
@@ -1,23 +0,0 @@
----
-layout: docs
-title:  Quick Start with Sample Cube
-categories: tutorial
-permalink: /docs/tutorial/kylin_sample.html
-version: v1.2
-since: v0.7.1
----
-
-Kylin provides a script for you to create a sample Cube; the script will also create three sample hive tables:
-
-1. Run ${KYLIN_HOME}/bin/sample.sh ; Restart kylin server to flush the caches;
-2. Logon Kylin web, select project "learn_kylin";
-3. Select the sample cube "kylin_sales_cube", click "Actions" -> "Build", pick up a date later than 2014-01-01 (to cover all 10000 sample records);
-4. Check the build progress in "Jobs" tab, until 100%;
-5. Execute SQLs in the "Query" tab, for example:
-	select part_dt, sum(price) as total_selled, count(distinct seller_id) as sellers from kylin_sales group by part_dt order by part_dt
-6. You can verify the query result and compare the response time with hive;
-
-   
-## What's next
-
-After cube being built, please refer to other document of this tutorial for more detail information.

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_docs/tutorial/odbc.cn.md
----------------------------------------------------------------------
diff --git a/website/_docs/tutorial/odbc.cn.md b/website/_docs/tutorial/odbc.cn.md
deleted file mode 100644
index 4947494..0000000
--- a/website/_docs/tutorial/odbc.cn.md
+++ /dev/null
@@ -1,34 +0,0 @@
----
-layout: docs-cn
-title:  Kylin ODBC 驱动程序教程
-categories: 教程
-permalink: /cn/docs/tutorial/odbc.html
-version: v1.2
-since: v0.7.1
----
-
-> 我们提供Kylin ODBC驱动程序以支持ODBC兼容客户端应用的数据访问。
-> 
-> 32位版本或64位版本的驱动程序都是可用的。
-> 
-> 测试操作系统:Windows 7,Windows Server 2008 R2
-> 
-> 测试应用:Tableau 8.0.4 和 Tableau 8.1.3
-
-## 前提条件
-1. Microsoft Visual C++ 2012 再分配(Redistributable)
-   * 32位Windows或32位Tableau Desktop:下载:[32bit version](http://download.microsoft.com/download/1/6/B/16B06F60-3B20-4FF2-B699-5E9B7962F9AE/VSU_4/vcredist_x86.exe) 
-   * 64位Windows或64位Tableau Desktop:下载:[64bit version](http://download.microsoft.com/download/1/6/B/16B06F60-3B20-4FF2-B699-5E9B7962F9AE/VSU_4/vcredist_x64.exe)
-
-2. ODBC驱动程序内部从一个REST服务器获取结果,确保你能够访问一个
-
-## 安装
-1. 如果你已经安装,首先卸载已存在的Kylin ODBC
-2. 从[下载](../../download/)下载附件驱动安装程序,并运行。
-   * 32位Tableau Desktop:请安装KylinODBCDriver (x86).exe
-   * 64位Tableau Desktop:请安装KylinODBCDriver (x64).exe
-
-3. Both drivers already be installed on Tableau Server, you properly should be able to publish to there without issues
-
-## 错误报告
-如有问题,请报告错误至Apache Kylin JIRA,或者发送邮件到dev邮件列表。

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_docs/tutorial/odbc.md
----------------------------------------------------------------------
diff --git a/website/_docs/tutorial/odbc.md b/website/_docs/tutorial/odbc.md
deleted file mode 100644
index 96957ee..0000000
--- a/website/_docs/tutorial/odbc.md
+++ /dev/null
@@ -1,50 +0,0 @@
----
-layout: docs
-title:  Kylin ODBC Driver Tutorial
-categories: tutorial
-permalink: /docs/tutorial/odbc.html
-version: v1.2
-since: v0.7.1
----
-
-> We provide Kylin ODBC driver to enable data access from ODBC-compatible client applications.
-> 
-> Both 32-bit version or 64-bit version driver are available.
-> 
-> Tested Operation System: Windows 7, Windows Server 2008 R2
-> 
-> Tested Application: Tableau 8.0.4, Tableau 8.1.3 and Tableau 9.1
-
-## Prerequisites
-1. Microsoft Visual C++ 2012 Redistributable 
-   * For 32 bit Windows or 32 bit Tableau Desktop: Download: [32bit version](http://download.microsoft.com/download/1/6/B/16B06F60-3B20-4FF2-B699-5E9B7962F9AE/VSU_4/vcredist_x86.exe) 
-   * For 64 bit Windows or 64 bit Tableau Desktop: Download: [64bit version](http://download.microsoft.com/download/1/6/B/16B06F60-3B20-4FF2-B699-5E9B7962F9AE/VSU_4/vcredist_x64.exe)
-
-
-2. ODBC driver internally gets results from a REST server, make sure you have access to one
-
-## Installation
-1. Uninstall existing Kylin ODBC first, if you already installled it before
-2. Download ODBC Driver from [download](../../download/).
-   * For 32 bit Tableau Desktop: Please install KylinODBCDriver (x86).exe
-   * For 64 bit Tableau Desktop: Please install KylinODBCDriver (x64).exe
-
-3. Both drivers already be installed on Tableau Server, you properly should be able to publish to there without issues
-
-## DSN configuration
-1. Open ODBCAD to configure DSN.
-	* For 32 bit driver, please use the 32bit version in C:\Windows\SysWOW64\odbcad32.exe
-	* For 64 bit driver, please use the default "Data Sources (ODBC)" in Control Panel/Administrator Tools
-![]( /images/Kylin-ODBC-DSN/1.png)
-
-2. Open "System DSN" tab, and click "Add", you will see KylinODBCDriver listed as an option, Click "Finish" to continue.
-![]( /images/Kylin-ODBC-DSN/2.png)
-
-3. In the pop up dialog, fill in all the blanks, The server host is where your Kylin Rest Server is started.
-![]( /images/Kylin-ODBC-DSN/3.png)
-
-4. Click "Done", and you will see your new DSN listed in the "System Data Sources", you can use this DSN afterwards.
-![]( /images/Kylin-ODBC-DSN/4.png)
-
-## Bug Report
-Please open Apache Kylin JIRA to report bug, or send to dev mailing list.


[10/52] [abbrv] kylin git commit: KYLIN-1082 fix an issue in classpath

Posted by li...@apache.org.
KYLIN-1082 fix an issue in classpath


Project: http://git-wip-us.apache.org/repos/asf/kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/247394ff
Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/247394ff
Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/247394ff

Branch: refs/heads/1.x-HBase1.1.3
Commit: 247394ff9669353209277301b0150b89f93bef3e
Parents: 785efd8
Author: shaofengshi <sh...@apache.org>
Authored: Mon Jan 25 17:34:45 2016 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Mon Jan 25 17:35:05 2016 +0800

----------------------------------------------------------------------
 .../main/java/org/apache/kylin/job/hadoop/AbstractHadoopJob.java    | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/247394ff/job/src/main/java/org/apache/kylin/job/hadoop/AbstractHadoopJob.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/AbstractHadoopJob.java b/job/src/main/java/org/apache/kylin/job/hadoop/AbstractHadoopJob.java
index f5c85eb..698a978 100644
--- a/job/src/main/java/org/apache/kylin/job/hadoop/AbstractHadoopJob.java
+++ b/job/src/main/java/org/apache/kylin/job/hadoop/AbstractHadoopJob.java
@@ -170,6 +170,7 @@ public abstract class AbstractHadoopJob extends Configured implements Tool {
         if (classpath == null || classpath.length() == 0) {
             logger.info("Didn't find " + MAP_REDUCE_CLASSPATH + " in job configuration, will run 'mapred classpath' to get the default value.");
             classpath = getDefaultMapRedClasspath();
+            classpath = classpath.replace(":", ","); // yarn classpath is comma separated
             logger.info("The default mapred classpath is: " + classpath);
         }
 


[38/52] [abbrv] kylin git commit: KYLIN-1416 remove website from coding branch

Posted by li...@apache.org.
http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/assets/css/animate.css
----------------------------------------------------------------------
diff --git a/website/assets/css/animate.css b/website/assets/css/animate.css
deleted file mode 100644
index d71da17..0000000
--- a/website/assets/css/animate.css
+++ /dev/null
@@ -1,2744 +0,0 @@
-@charset "UTF-8";
-
-
-/*!
-Animate.css - http://daneden.me/animate
-Licensed under the MIT license
-
-Copyright (c) 2013 Daniel Eden
-
-Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated documentation files (the "Software"), to deal in the Software without restriction, including without limitation the rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to permit persons to whom the Software is furnished to do so, subject to the following conditions:
-
-The above copyright notice and this permission notice shall be included in all copies or substantial portions of the Software.
-
-THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
-*/
-
-.animated {
-  -webkit-animation-duration: 1s;
-  animation-duration: 1s;
-  -webkit-animation-fill-mode: both;
-  animation-fill-mode: both;
-}
-
-.animated.hinge {
-  -webkit-animation-duration: 2s;
-  animation-duration: 2s;
-}
-
-@-webkit-keyframes bounce {
-  0%, 20%, 50%, 80%, 100% {
-    -webkit-transform: translateY(0);
-    transform: translateY(0);
-  }
-
-  40% {
-    -webkit-transform: translateY(-30px);
-    transform: translateY(-30px);
-  }
-
-  60% {
-    -webkit-transform: translateY(-15px);
-    transform: translateY(-15px);
-  }
-}
-
-@keyframes bounce {
-  0%, 20%, 50%, 80%, 100% {
-    -webkit-transform: translateY(0);
-    -ms-transform: translateY(0);
-    transform: translateY(0);
-  }
-
-  40% {
-    -webkit-transform: translateY(-30px);
-    -ms-transform: translateY(-30px);
-    transform: translateY(-30px);
-  }
-
-  60% {
-    -webkit-transform: translateY(-15px);
-    -ms-transform: translateY(-15px);
-    transform: translateY(-15px);
-  }
-}
-
-.bounce {
-  -webkit-animation-name: bounce;
-  animation-name: bounce;
-}
-
-@-webkit-keyframes flash {
-  0%, 50%, 100% {
-    opacity: 1;
-  }
-
-  25%, 75% {
-    opacity: 0;
-  }
-}
-
-@keyframes flash {
-  0%, 50%, 100% {
-    opacity: 1;
-  }
-
-  25%, 75% {
-    opacity: 0;
-  }
-}
-
-.flash {
-  -webkit-animation-name: flash;
-  animation-name: flash;
-}
-
-/* originally authored by Nick Pettit - https://github.com/nickpettit/glide */
-
-@-webkit-keyframes pulse {
-  0% {
-    -webkit-transform: scale(1);
-    transform: scale(1);
-  }
-
-  50% {
-    -webkit-transform: scale(1.1);
-    transform: scale(1.1);
-  }
-
-  100% {
-    -webkit-transform: scale(1);
-    transform: scale(1);
-  }
-}
-
-@keyframes pulse {
-  0% {
-    -webkit-transform: scale(1);
-    -ms-transform: scale(1);
-    transform: scale(1);
-  }
-
-  50% {
-    -webkit-transform: scale(1.1);
-    -ms-transform: scale(1.1);
-    transform: scale(1.1);
-  }
-
-  100% {
-    -webkit-transform: scale(1);
-    -ms-transform: scale(1);
-    transform: scale(1);
-  }
-}
-
-.pulse {
-  -webkit-animation-name: pulse;
-  animation-name: pulse;
-}
-
-@-webkit-keyframes shake {
-  0%, 100% {
-    -webkit-transform: translateX(0);
-    transform: translateX(0);
-  }
-
-  10%, 30%, 50%, 70%, 90% {
-    -webkit-transform: translateX(-10px);
-    transform: translateX(-10px);
-  }
-
-  20%, 40%, 60%, 80% {
-    -webkit-transform: translateX(10px);
-    transform: translateX(10px);
-  }
-}
-
-@keyframes shake {
-  0%, 100% {
-    -webkit-transform: translateX(0);
-    -ms-transform: translateX(0);
-    transform: translateX(0);
-  }
-
-  10%, 30%, 50%, 70%, 90% {
-    -webkit-transform: translateX(-10px);
-    -ms-transform: translateX(-10px);
-    transform: translateX(-10px);
-  }
-
-  20%, 40%, 60%, 80% {
-    -webkit-transform: translateX(10px);
-    -ms-transform: translateX(10px);
-    transform: translateX(10px);
-  }
-}
-
-.shake {
-  -webkit-animation-name: shake;
-  animation-name: shake;
-}
-
-@-webkit-keyframes swing {
-  20% {
-    -webkit-transform: rotate(15deg);
-    transform: rotate(15deg);
-  }
-
-  40% {
-    -webkit-transform: rotate(-10deg);
-    transform: rotate(-10deg);
-  }
-
-  60% {
-    -webkit-transform: rotate(5deg);
-    transform: rotate(5deg);
-  }
-
-  80% {
-    -webkit-transform: rotate(-5deg);
-    transform: rotate(-5deg);
-  }
-
-  100% {
-    -webkit-transform: rotate(0deg);
-    transform: rotate(0deg);
-  }
-}
-
-@keyframes swing {
-  20% {
-    -webkit-transform: rotate(15deg);
-    -ms-transform: rotate(15deg);
-    transform: rotate(15deg);
-  }
-
-  40% {
-    -webkit-transform: rotate(-10deg);
-    -ms-transform: rotate(-10deg);
-    transform: rotate(-10deg);
-  }
-
-  60% {
-    -webkit-transform: rotate(5deg);
-    -ms-transform: rotate(5deg);
-    transform: rotate(5deg);
-  }
-
-  80% {
-    -webkit-transform: rotate(-5deg);
-    -ms-transform: rotate(-5deg);
-    transform: rotate(-5deg);
-  }
-
-  100% {
-    -webkit-transform: rotate(0deg);
-    -ms-transform: rotate(0deg);
-    transform: rotate(0deg);
-  }
-}
-
-.swing {
-  -webkit-transform-origin: top center;
-  -ms-transform-origin: top center;
-  transform-origin: top center;
-  -webkit-animation-name: swing;
-  animation-name: swing;
-}
-
-@-webkit-keyframes tada {
-  0% {
-    -webkit-transform: scale(1);
-    transform: scale(1);
-  }
-
-  10%, 20% {
-    -webkit-transform: scale(0.9) rotate(-3deg);
-    transform: scale(0.9) rotate(-3deg);
-  }
-
-  30%, 50%, 70%, 90% {
-    -webkit-transform: scale(1.1) rotate(3deg);
-    transform: scale(1.1) rotate(3deg);
-  }
-
-  40%, 60%, 80% {
-    -webkit-transform: scale(1.1) rotate(-3deg);
-    transform: scale(1.1) rotate(-3deg);
-  }
-
-  100% {
-    -webkit-transform: scale(1) rotate(0);
-    transform: scale(1) rotate(0);
-  }
-}
-
-@keyframes tada {
-  0% {
-    -webkit-transform: scale(1);
-    -ms-transform: scale(1);
-    transform: scale(1);
-  }
-
-  10%, 20% {
-    -webkit-transform: scale(0.9) rotate(-3deg);
-    -ms-transform: scale(0.9) rotate(-3deg);
-    transform: scale(0.9) rotate(-3deg);
-  }
-
-  30%, 50%, 70%, 90% {
-    -webkit-transform: scale(1.1) rotate(3deg);
-    -ms-transform: scale(1.1) rotate(3deg);
-    transform: scale(1.1) rotate(3deg);
-  }
-
-  40%, 60%, 80% {
-    -webkit-transform: scale(1.1) rotate(-3deg);
-    -ms-transform: scale(1.1) rotate(-3deg);
-    transform: scale(1.1) rotate(-3deg);
-  }
-
-  100% {
-    -webkit-transform: scale(1) rotate(0);
-    -ms-transform: scale(1) rotate(0);
-    transform: scale(1) rotate(0);
-  }
-}
-
-.tada {
-  -webkit-animation-name: tada;
-  animation-name: tada;
-}
-
-/* originally authored by Nick Pettit - https://github.com/nickpettit/glide */
-
-@-webkit-keyframes wobble {
-  0% {
-    -webkit-transform: translateX(0%);
-    transform: translateX(0%);
-  }
-
-  15% {
-    -webkit-transform: translateX(-25%) rotate(-5deg);
-    transform: translateX(-25%) rotate(-5deg);
-  }
-
-  30% {
-    -webkit-transform: translateX(20%) rotate(3deg);
-    transform: translateX(20%) rotate(3deg);
-  }
-
-  45% {
-    -webkit-transform: translateX(-15%) rotate(-3deg);
-    transform: translateX(-15%) rotate(-3deg);
-  }
-
-  60% {
-    -webkit-transform: translateX(10%) rotate(2deg);
-    transform: translateX(10%) rotate(2deg);
-  }
-
-  75% {
-    -webkit-transform: translateX(-5%) rotate(-1deg);
-    transform: translateX(-5%) rotate(-1deg);
-  }
-
-  100% {
-    -webkit-transform: translateX(0%);
-    transform: translateX(0%);
-  }
-}
-
-@keyframes wobble {
-  0% {
-    -webkit-transform: translateX(0%);
-    -ms-transform: translateX(0%);
-    transform: translateX(0%);
-  }
-
-  15% {
-    -webkit-transform: translateX(-25%) rotate(-5deg);
-    -ms-transform: translateX(-25%) rotate(-5deg);
-    transform: translateX(-25%) rotate(-5deg);
-  }
-
-  30% {
-    -webkit-transform: translateX(20%) rotate(3deg);
-    -ms-transform: translateX(20%) rotate(3deg);
-    transform: translateX(20%) rotate(3deg);
-  }
-
-  45% {
-    -webkit-transform: translateX(-15%) rotate(-3deg);
-    -ms-transform: translateX(-15%) rotate(-3deg);
-    transform: translateX(-15%) rotate(-3deg);
-  }
-
-  60% {
-    -webkit-transform: translateX(10%) rotate(2deg);
-    -ms-transform: translateX(10%) rotate(2deg);
-    transform: translateX(10%) rotate(2deg);
-  }
-
-  75% {
-    -webkit-transform: translateX(-5%) rotate(-1deg);
-    -ms-transform: translateX(-5%) rotate(-1deg);
-    transform: translateX(-5%) rotate(-1deg);
-  }
-
-  100% {
-    -webkit-transform: translateX(0%);
-    -ms-transform: translateX(0%);
-    transform: translateX(0%);
-  }
-}
-
-.wobble {
-  -webkit-animation-name: wobble;
-  animation-name: wobble;
-}
-
-@-webkit-keyframes bounceIn {
-  0% {
-    opacity: 0;
-    -webkit-transform: scale(.3);
-    transform: scale(.3);
-  }
-
-  50% {
-    opacity: 1;
-    -webkit-transform: scale(1.05);
-    transform: scale(1.05);
-  }
-
-  70% {
-    -webkit-transform: scale(.9);
-    transform: scale(.9);
-  }
-
-  100% {
-    -webkit-transform: scale(1);
-    transform: scale(1);
-  }
-}
-
-@keyframes bounceIn {
-  0% {
-    opacity: 0;
-    -webkit-transform: scale(.3);
-    -ms-transform: scale(.3);
-    transform: scale(.3);
-  }
-
-  50% {
-    opacity: 1;
-    -webkit-transform: scale(1.05);
-    -ms-transform: scale(1.05);
-    transform: scale(1.05);
-  }
-
-  70% {
-    -webkit-transform: scale(.9);
-    -ms-transform: scale(.9);
-    transform: scale(.9);
-  }
-
-  100% {
-    -webkit-transform: scale(1);
-    -ms-transform: scale(1);
-    transform: scale(1);
-  }
-}
-
-.bounceIn {
-  -webkit-animation-name: bounceIn;
-  animation-name: bounceIn;
-}
-
-@-webkit-keyframes bounceInDown {
-  0% {
-    opacity: 0;
-    -webkit-transform: translateY(-2000px);
-    transform: translateY(-2000px);
-  }
-
-  60% {
-    opacity: 1;
-    -webkit-transform: translateY(30px);
-    transform: translateY(30px);
-  }
-
-  80% {
-    -webkit-transform: translateY(-10px);
-    transform: translateY(-10px);
-  }
-
-  100% {
-    -webkit-transform: translateY(0);
-    transform: translateY(0);
-  }
-}
-
-@keyframes bounceInDown {
-  0% {
-    opacity: 0;
-    -webkit-transform: translateY(-2000px);
-    -ms-transform: translateY(-2000px);
-    transform: translateY(-2000px);
-  }
-
-  60% {
-    opacity: 1;
-    -webkit-transform: translateY(30px);
-    -ms-transform: translateY(30px);
-    transform: translateY(30px);
-  }
-
-  80% {
-    -webkit-transform: translateY(-10px);
-    -ms-transform: translateY(-10px);
-    transform: translateY(-10px);
-  }
-
-  100% {
-    -webkit-transform: translateY(0);
-    -ms-transform: translateY(0);
-    transform: translateY(0);
-  }
-}
-
-.bounceInDown {
-  -webkit-animation-name: bounceInDown;
-  animation-name: bounceInDown;
-}
-
-@-webkit-keyframes bounceInLeft {
-  0% {
-    opacity: 0;
-    -webkit-transform: translateX(-2000px);
-    transform: translateX(-2000px);
-  }
-
-  60% {
-    opacity: 1;
-    -webkit-transform: translateX(30px);
-    transform: translateX(30px);
-  }
-
-  80% {
-    -webkit-transform: translateX(-10px);
-    transform: translateX(-10px);
-  }
-
-  100% {
-    -webkit-transform: translateX(0);
-    transform: translateX(0);
-  }
-}
-
-@keyframes bounceInLeft {
-  0% {
-    opacity: 0;
-    -webkit-transform: translateX(-2000px);
-    -ms-transform: translateX(-2000px);
-    transform: translateX(-2000px);
-  }
-
-  60% {
-    opacity: 1;
-    -webkit-transform: translateX(30px);
-    -ms-transform: translateX(30px);
-    transform: translateX(30px);
-  }
-
-  80% {
-    -webkit-transform: translateX(-10px);
-    -ms-transform: translateX(-10px);
-    transform: translateX(-10px);
-  }
-
-  100% {
-    -webkit-transform: translateX(0);
-    -ms-transform: translateX(0);
-    transform: translateX(0);
-  }
-}
-
-.bounceInLeft {
-  -webkit-animation-name: bounceInLeft;
-  animation-name: bounceInLeft;
-}
-
-@-webkit-keyframes bounceInRight {
-  0% {
-    opacity: 0;
-    -webkit-transform: translateX(2000px);
-    transform: translateX(2000px);
-  }
-
-  60% {
-    opacity: 1;
-    -webkit-transform: translateX(-30px);
-    transform: translateX(-30px);
-  }
-
-  80% {
-    -webkit-transform: translateX(10px);
-    transform: translateX(10px);
-  }
-
-  100% {
-    -webkit-transform: translateX(0);
-    transform: translateX(0);
-  }
-}
-
-@keyframes bounceInRight {
-  0% {
-    opacity: 0;
-    -webkit-transform: translateX(2000px);
-    -ms-transform: translateX(2000px);
-    transform: translateX(2000px);
-  }
-
-  60% {
-    opacity: 1;
-    -webkit-transform: translateX(-30px);
-    -ms-transform: translateX(-30px);
-    transform: translateX(-30px);
-  }
-
-  80% {
-    -webkit-transform: translateX(10px);
-    -ms-transform: translateX(10px);
-    transform: translateX(10px);
-  }
-
-  100% {
-    -webkit-transform: translateX(0);
-    -ms-transform: translateX(0);
-    transform: translateX(0);
-  }
-}
-
-.bounceInRight {
-  -webkit-animation-name: bounceInRight;
-  animation-name: bounceInRight;
-}
-
-@-webkit-keyframes bounceInUp {
-  0% {
-    opacity: 0;
-    -webkit-transform: translateY(2000px);
-    transform: translateY(2000px);
-  }
-
-  60% {
-    opacity: 1;
-    -webkit-transform: translateY(-30px);
-    transform: translateY(-30px);
-  }
-
-  80% {
-    -webkit-transform: translateY(10px);
-    transform: translateY(10px);
-  }
-
-  100% {
-    -webkit-transform: translateY(0);
-    transform: translateY(0);
-  }
-}
-
-@keyframes bounceInUp {
-  0% {
-    opacity: 0;
-    -webkit-transform: translateY(2000px);
-    -ms-transform: translateY(2000px);
-    transform: translateY(2000px);
-  }
-
-  60% {
-    opacity: 1;
-    -webkit-transform: translateY(-30px);
-    -ms-transform: translateY(-30px);
-    transform: translateY(-30px);
-  }
-
-  80% {
-    -webkit-transform: translateY(10px);
-    -ms-transform: translateY(10px);
-    transform: translateY(10px);
-  }
-
-  100% {
-    -webkit-transform: translateY(0);
-    -ms-transform: translateY(0);
-    transform: translateY(0);
-  }
-}
-
-.bounceInUp {
-  -webkit-animation-name: bounceInUp;
-  animation-name: bounceInUp;
-}
-
-@-webkit-keyframes bounceOut {
-  0% {
-    -webkit-transform: scale(1);
-    transform: scale(1);
-  }
-
-  25% {
-    -webkit-transform: scale(.95);
-    transform: scale(.95);
-  }
-
-  50% {
-    opacity: 1;
-    -webkit-transform: scale(1.1);
-    transform: scale(1.1);
-  }
-
-  100% {
-    opacity: 0;
-    -webkit-transform: scale(.3);
-    transform: scale(.3);
-  }
-}
-
-@keyframes bounceOut {
-  0% {
-    -webkit-transform: scale(1);
-    -ms-transform: scale(1);
-    transform: scale(1);
-  }
-
-  25% {
-    -webkit-transform: scale(.95);
-    -ms-transform: scale(.95);
-    transform: scale(.95);
-  }
-
-  50% {
-    opacity: 1;
-    -webkit-transform: scale(1.1);
-    -ms-transform: scale(1.1);
-    transform: scale(1.1);
-  }
-
-  100% {
-    opacity: 0;
-    -webkit-transform: scale(.3);
-    -ms-transform: scale(.3);
-    transform: scale(.3);
-  }
-}
-
-.bounceOut {
-  -webkit-animation-name: bounceOut;
-  animation-name: bounceOut;
-}
-
-@-webkit-keyframes bounceOutDown {
-  0% {
-    -webkit-transform: translateY(0);
-    transform: translateY(0);
-  }
-
-  20% {
-    opacity: 1;
-    -webkit-transform: translateY(-20px);
-    transform: translateY(-20px);
-  }
-
-  100% {
-    opacity: 0;
-    -webkit-transform: translateY(2000px);
-    transform: translateY(2000px);
-  }
-}
-
-@keyframes bounceOutDown {
-  0% {
-    -webkit-transform: translateY(0);
-    -ms-transform: translateY(0);
-    transform: translateY(0);
-  }
-
-  20% {
-    opacity: 1;
-    -webkit-transform: translateY(-20px);
-    -ms-transform: translateY(-20px);
-    transform: translateY(-20px);
-  }
-
-  100% {
-    opacity: 0;
-    -webkit-transform: translateY(2000px);
-    -ms-transform: translateY(2000px);
-    transform: translateY(2000px);
-  }
-}
-
-.bounceOutDown {
-  -webkit-animation-name: bounceOutDown;
-  animation-name: bounceOutDown;
-}
-
-@-webkit-keyframes bounceOutLeft {
-  0% {
-    -webkit-transform: translateX(0);
-    transform: translateX(0);
-  }
-
-  20% {
-    opacity: 1;
-    -webkit-transform: translateX(20px);
-    transform: translateX(20px);
-  }
-
-  100% {
-    opacity: 0;
-    -webkit-transform: translateX(-2000px);
-    transform: translateX(-2000px);
-  }
-}
-
-@keyframes bounceOutLeft {
-  0% {
-    -webkit-transform: translateX(0);
-    -ms-transform: translateX(0);
-    transform: translateX(0);
-  }
-
-  20% {
-    opacity: 1;
-    -webkit-transform: translateX(20px);
-    -ms-transform: translateX(20px);
-    transform: translateX(20px);
-  }
-
-  100% {
-    opacity: 0;
-    -webkit-transform: translateX(-2000px);
-    -ms-transform: translateX(-2000px);
-    transform: translateX(-2000px);
-  }
-}
-
-.bounceOutLeft {
-  -webkit-animation-name: bounceOutLeft;
-  animation-name: bounceOutLeft;
-}
-
-@-webkit-keyframes bounceOutRight {
-  0% {
-    -webkit-transform: translateX(0);
-    transform: translateX(0);
-  }
-
-  20% {
-    opacity: 1;
-    -webkit-transform: translateX(-20px);
-    transform: translateX(-20px);
-  }
-
-  100% {
-    opacity: 0;
-    -webkit-transform: translateX(2000px);
-    transform: translateX(2000px);
-  }
-}
-
-@keyframes bounceOutRight {
-  0% {
-    -webkit-transform: translateX(0);
-    -ms-transform: translateX(0);
-    transform: translateX(0);
-  }
-
-  20% {
-    opacity: 1;
-    -webkit-transform: translateX(-20px);
-    -ms-transform: translateX(-20px);
-    transform: translateX(-20px);
-  }
-
-  100% {
-    opacity: 0;
-    -webkit-transform: translateX(2000px);
-    -ms-transform: translateX(2000px);
-    transform: translateX(2000px);
-  }
-}
-
-.bounceOutRight {
-  -webkit-animation-name: bounceOutRight;
-  animation-name: bounceOutRight;
-}
-
-@-webkit-keyframes bounceOutUp {
-  0% {
-    -webkit-transform: translateY(0);
-    transform: translateY(0);
-  }
-
-  20% {
-    opacity: 1;
-    -webkit-transform: translateY(20px);
-    transform: translateY(20px);
-  }
-
-  100% {
-    opacity: 0;
-    -webkit-transform: translateY(-2000px);
-    transform: translateY(-2000px);
-  }
-}
-
-@keyframes bounceOutUp {
-  0% {
-    -webkit-transform: translateY(0);
-    -ms-transform: translateY(0);
-    transform: translateY(0);
-  }
-
-  20% {
-    opacity: 1;
-    -webkit-transform: translateY(20px);
-    -ms-transform: translateY(20px);
-    transform: translateY(20px);
-  }
-
-  100% {
-    opacity: 0;
-    -webkit-transform: translateY(-2000px);
-    -ms-transform: translateY(-2000px);
-    transform: translateY(-2000px);
-  }
-}
-
-.bounceOutUp {
-  -webkit-animation-name: bounceOutUp;
-  animation-name: bounceOutUp;
-}
-
-@-webkit-keyframes fadeIn {
-  0% {
-    opacity: 0;
-  }
-
-  100% {
-    opacity: 1;
-  }
-}
-
-@keyframes fadeIn {
-  0% {
-    opacity: 0;
-  }
-
-  100% {
-    opacity: 1;
-  }
-}
-
-.fadeIn {
-  -webkit-animation-name: fadeIn;
-  animation-name: fadeIn;
-}
-
-@-webkit-keyframes fadeInDown {
-  0% {
-    opacity: 0;
-    -webkit-transform: translateY(-20px);
-    transform: translateY(-20px);
-  }
-
-  100% {
-    opacity: 1;
-    -webkit-transform: translateY(0);
-    transform: translateY(0);
-  }
-}
-
-@keyframes fadeInDown {
-  0% {
-    opacity: 0;
-    -webkit-transform: translateY(-20px);
-    -ms-transform: translateY(-20px);
-    transform: translateY(-20px);
-  }
-
-  100% {
-    opacity: 1;
-    -webkit-transform: translateY(0);
-    -ms-transform: translateY(0);
-    transform: translateY(0);
-  }
-}
-
-.fadeInDown {
-  -webkit-animation-name: fadeInDown;
-  animation-name: fadeInDown;
-}
-
-@-webkit-keyframes fadeInDownBig {
-  0% {
-    opacity: 0;
-    -webkit-transform: translateY(-2000px);
-    transform: translateY(-2000px);
-  }
-
-  100% {
-    opacity: 1;
-    -webkit-transform: translateY(0);
-    transform: translateY(0);
-  }
-}
-
-@keyframes fadeInDownBig {
-  0% {
-    opacity: 0;
-    -webkit-transform: translateY(-2000px);
-    -ms-transform: translateY(-2000px);
-    transform: translateY(-2000px);
-  }
-
-  100% {
-    opacity: 1;
-    -webkit-transform: translateY(0);
-    -ms-transform: translateY(0);
-    transform: translateY(0);
-  }
-}
-
-.fadeInDownBig {
-  -webkit-animation-name: fadeInDownBig;
-  animation-name: fadeInDownBig;
-}
-
-@-webkit-keyframes fadeInLeft {
-  0% {
-    opacity: 0;
-    -webkit-transform: translateX(-20px);
-    transform: translateX(-20px);
-  }
-
-  100% {
-    opacity: 1;
-    -webkit-transform: translateX(0);
-    transform: translateX(0);
-  }
-}
-
-@keyframes fadeInLeft {
-  0% {
-    opacity: 0;
-    -webkit-transform: translateX(-20px);
-    -ms-transform: translateX(-20px);
-    transform: translateX(-20px);
-  }
-
-  100% {
-    opacity: 1;
-    -webkit-transform: translateX(0);
-    -ms-transform: translateX(0);
-    transform: translateX(0);
-  }
-}
-
-.fadeInLeft {
-  -webkit-animation-name: fadeInLeft;
-  animation-name: fadeInLeft;
-}
-
-@-webkit-keyframes fadeInLeftBig {
-  0% {
-    opacity: 0;
-    -webkit-transform: translateX(-2000px);
-    transform: translateX(-2000px);
-  }
-
-  100% {
-    opacity: 1;
-    -webkit-transform: translateX(0);
-    transform: translateX(0);
-  }
-}
-
-@keyframes fadeInLeftBig {
-  0% {
-    opacity: 0;
-    -webkit-transform: translateX(-2000px);
-    -ms-transform: translateX(-2000px);
-    transform: translateX(-2000px);
-  }
-
-  100% {
-    opacity: 1;
-    -webkit-transform: translateX(0);
-    -ms-transform: translateX(0);
-    transform: translateX(0);
-  }
-}
-
-.fadeInLeftBig {
-  -webkit-animation-name: fadeInLeftBig;
-  animation-name: fadeInLeftBig;
-}
-
-@-webkit-keyframes fadeInRight {
-  0% {
-    opacity: 0;
-    -webkit-transform: translateX(20px);
-    transform: translateX(20px);
-  }
-
-  100% {
-    opacity: 1;
-    -webkit-transform: translateX(0);
-    transform: translateX(0);
-  }
-}
-
-@keyframes fadeInRight {
-  0% {
-    opacity: 0;
-    -webkit-transform: translateX(20px);
-    -ms-transform: translateX(20px);
-    transform: translateX(20px);
-  }
-
-  100% {
-    opacity: 1;
-    -webkit-transform: translateX(0);
-    -ms-transform: translateX(0);
-    transform: translateX(0);
-  }
-}
-
-.fadeInRight {
-  -webkit-animation-name: fadeInRight;
-  animation-name: fadeInRight;
-}
-
-@-webkit-keyframes fadeInRightBig {
-  0% {
-    opacity: 0;
-    -webkit-transform: translateX(2000px);
-    transform: translateX(2000px);
-  }
-
-  100% {
-    opacity: 1;
-    -webkit-transform: translateX(0);
-    transform: translateX(0);
-  }
-}
-
-@keyframes fadeInRightBig {
-  0% {
-    opacity: 0;
-    -webkit-transform: translateX(2000px);
-    -ms-transform: translateX(2000px);
-    transform: translateX(2000px);
-  }
-
-  100% {
-    opacity: 1;
-    -webkit-transform: translateX(0);
-    -ms-transform: translateX(0);
-    transform: translateX(0);
-  }
-}
-
-.fadeInRightBig {
-  -webkit-animation-name: fadeInRightBig;
-  animation-name: fadeInRightBig;
-}
-
-@-webkit-keyframes fadeInUp {
-  0% {
-    opacity: 0;
-    -webkit-transform: translateY(20px);
-    transform: translateY(20px);
-  }
-
-  100% {
-    opacity: 1;
-    -webkit-transform: translateY(0);
-    transform: translateY(0);
-  }
-}
-
-@keyframes fadeInUp {
-  0% {
-    opacity: 0;
-    -webkit-transform: translateY(20px);
-    -ms-transform: translateY(20px);
-    transform: translateY(20px);
-  }
-
-  100% {
-    opacity: 1;
-    -webkit-transform: translateY(0);
-    -ms-transform: translateY(0);
-    transform: translateY(0);
-  }
-}
-
-.fadeInUp {
-  -webkit-animation-name: fadeInUp;
-  animation-name: fadeInUp;
-}
-
-@-webkit-keyframes fadeInUpBig {
-  0% {
-    opacity: 0;
-    -webkit-transform: translateY(2000px);
-    transform: translateY(2000px);
-  }
-
-  100% {
-    opacity: 1;
-    -webkit-transform: translateY(0);
-    transform: translateY(0);
-  }
-}
-
-@keyframes fadeInUpBig {
-  0% {
-    opacity: 0;
-    -webkit-transform: translateY(2000px);
-    -ms-transform: translateY(2000px);
-    transform: translateY(2000px);
-  }
-
-  100% {
-    opacity: 1;
-    -webkit-transform: translateY(0);
-    -ms-transform: translateY(0);
-    transform: translateY(0);
-  }
-}
-
-.fadeInUpBig {
-  -webkit-animation-name: fadeInUpBig;
-  animation-name: fadeInUpBig;
-}
-
-@-webkit-keyframes fadeOut {
-  0% {
-    opacity: 1;
-  }
-
-  100% {
-    opacity: 0;
-  }
-}
-
-@keyframes fadeOut {
-  0% {
-    opacity: 1;
-  }
-
-  100% {
-    opacity: 0;
-  }
-}
-
-.fadeOut {
-  -webkit-animation-name: fadeOut;
-  animation-name: fadeOut;
-}
-
-@-webkit-keyframes fadeOutDown {
-  0% {
-    opacity: 1;
-    -webkit-transform: translateY(0);
-    transform: translateY(0);
-  }
-
-  100% {
-    opacity: 0;
-    -webkit-transform: translateY(20px);
-    transform: translateY(20px);
-  }
-}
-
-@keyframes fadeOutDown {
-  0% {
-    opacity: 1;
-    -webkit-transform: translateY(0);
-    -ms-transform: translateY(0);
-    transform: translateY(0);
-  }
-
-  100% {
-    opacity: 0;
-    -webkit-transform: translateY(20px);
-    -ms-transform: translateY(20px);
-    transform: translateY(20px);
-  }
-}
-
-.fadeOutDown {
-  -webkit-animation-name: fadeOutDown;
-  animation-name: fadeOutDown;
-}
-
-@-webkit-keyframes fadeOutDownBig {
-  0% {
-    opacity: 1;
-    -webkit-transform: translateY(0);
-    transform: translateY(0);
-  }
-
-  100% {
-    opacity: 0;
-    -webkit-transform: translateY(2000px);
-    transform: translateY(2000px);
-  }
-}
-
-@keyframes fadeOutDownBig {
-  0% {
-    opacity: 1;
-    -webkit-transform: translateY(0);
-    -ms-transform: translateY(0);
-    transform: translateY(0);
-  }
-
-  100% {
-    opacity: 0;
-    -webkit-transform: translateY(2000px);
-    -ms-transform: translateY(2000px);
-    transform: translateY(2000px);
-  }
-}
-
-.fadeOutDownBig {
-  -webkit-animation-name: fadeOutDownBig;
-  animation-name: fadeOutDownBig;
-}
-
-@-webkit-keyframes fadeOutLeft {
-  0% {
-    opacity: 1;
-    -webkit-transform: translateX(0);
-    transform: translateX(0);
-  }
-
-  100% {
-    opacity: 0;
-    -webkit-transform: translateX(-20px);
-    transform: translateX(-20px);
-  }
-}
-
-@keyframes fadeOutLeft {
-  0% {
-    opacity: 1;
-    -webkit-transform: translateX(0);
-    -ms-transform: translateX(0);
-    transform: translateX(0);
-  }
-
-  100% {
-    opacity: 0;
-    -webkit-transform: translateX(-20px);
-    -ms-transform: translateX(-20px);
-    transform: translateX(-20px);
-  }
-}
-
-.fadeOutLeft {
-  -webkit-animation-name: fadeOutLeft;
-  animation-name: fadeOutLeft;
-}
-
-@-webkit-keyframes fadeOutLeftBig {
-  0% {
-    opacity: 1;
-    -webkit-transform: translateX(0);
-    transform: translateX(0);
-  }
-
-  100% {
-    opacity: 0;
-    -webkit-transform: translateX(-2000px);
-    transform: translateX(-2000px);
-  }
-}
-
-@keyframes fadeOutLeftBig {
-  0% {
-    opacity: 1;
-    -webkit-transform: translateX(0);
-    -ms-transform: translateX(0);
-    transform: translateX(0);
-  }
-
-  100% {
-    opacity: 0;
-    -webkit-transform: translateX(-2000px);
-    -ms-transform: translateX(-2000px);
-    transform: translateX(-2000px);
-  }
-}
-
-.fadeOutLeftBig {
-  -webkit-animation-name: fadeOutLeftBig;
-  animation-name: fadeOutLeftBig;
-}
-
-@-webkit-keyframes fadeOutRight {
-  0% {
-    opacity: 1;
-    -webkit-transform: translateX(0);
-    transform: translateX(0);
-  }
-
-  100% {
-    opacity: 0;
-    -webkit-transform: translateX(20px);
-    transform: translateX(20px);
-  }
-}
-
-@keyframes fadeOutRight {
-  0% {
-    opacity: 1;
-    -webkit-transform: translateX(0);
-    -ms-transform: translateX(0);
-    transform: translateX(0);
-  }
-
-  100% {
-    opacity: 0;
-    -webkit-transform: translateX(20px);
-    -ms-transform: translateX(20px);
-    transform: translateX(20px);
-  }
-}
-
-.fadeOutRight {
-  -webkit-animation-name: fadeOutRight;
-  animation-name: fadeOutRight;
-}
-
-@-webkit-keyframes fadeOutRightBig {
-  0% {
-    opacity: 1;
-    -webkit-transform: translateX(0);
-    transform: translateX(0);
-  }
-
-  100% {
-    opacity: 0;
-    -webkit-transform: translateX(2000px);
-    transform: translateX(2000px);
-  }
-}
-
-@keyframes fadeOutRightBig {
-  0% {
-    opacity: 1;
-    -webkit-transform: translateX(0);
-    -ms-transform: translateX(0);
-    transform: translateX(0);
-  }
-
-  100% {
-    opacity: 0;
-    -webkit-transform: translateX(2000px);
-    -ms-transform: translateX(2000px);
-    transform: translateX(2000px);
-  }
-}
-
-.fadeOutRightBig {
-  -webkit-animation-name: fadeOutRightBig;
-  animation-name: fadeOutRightBig;
-}
-
-@-webkit-keyframes fadeOutUp {
-  0% {
-    opacity: 1;
-    -webkit-transform: translateY(0);
-    transform: translateY(0);
-  }
-
-  100% {
-    opacity: 0;
-    -webkit-transform: translateY(-20px);
-    transform: translateY(-20px);
-  }
-}
-
-@keyframes fadeOutUp {
-  0% {
-    opacity: 1;
-    -webkit-transform: translateY(0);
-    -ms-transform: translateY(0);
-    transform: translateY(0);
-  }
-
-  100% {
-    opacity: 0;
-    -webkit-transform: translateY(-20px);
-    -ms-transform: translateY(-20px);
-    transform: translateY(-20px);
-  }
-}
-
-.fadeOutUp {
-  -webkit-animation-name: fadeOutUp;
-  animation-name: fadeOutUp;
-}
-
-@-webkit-keyframes fadeOutUpBig {
-  0% {
-    opacity: 1;
-    -webkit-transform: translateY(0);
-    transform: translateY(0);
-  }
-
-  100% {
-    opacity: 0;
-    -webkit-transform: translateY(-2000px);
-    transform: translateY(-2000px);
-  }
-}
-
-@keyframes fadeOutUpBig {
-  0% {
-    opacity: 1;
-    -webkit-transform: translateY(0);
-    -ms-transform: translateY(0);
-    transform: translateY(0);
-  }
-
-  100% {
-    opacity: 0;
-    -webkit-transform: translateY(-2000px);
-    -ms-transform: translateY(-2000px);
-    transform: translateY(-2000px);
-  }
-}
-
-.fadeOutUpBig {
-  -webkit-animation-name: fadeOutUpBig;
-  animation-name: fadeOutUpBig;
-}
-
-@-webkit-keyframes flip {
-  0% {
-    -webkit-transform: perspective(400px) translateZ(0) rotateY(0) scale(1);
-    transform: perspective(400px) translateZ(0) rotateY(0) scale(1);
-    -webkit-animation-timing-function: ease-out;
-    animation-timing-function: ease-out;
-  }
-
-  40% {
-    -webkit-transform: perspective(400px) translateZ(150px) rotateY(170deg) scale(1);
-    transform: perspective(400px) translateZ(150px) rotateY(170deg) scale(1);
-    -webkit-animation-timing-function: ease-out;
-    animation-timing-function: ease-out;
-  }
-
-  50% {
-    -webkit-transform: perspective(400px) translateZ(150px) rotateY(190deg) scale(1);
-    transform: perspective(400px) translateZ(150px) rotateY(190deg) scale(1);
-    -webkit-animation-timing-function: ease-in;
-    animation-timing-function: ease-in;
-  }
-
-  80% {
-    -webkit-transform: perspective(400px) translateZ(0) rotateY(360deg) scale(.95);
-    transform: perspective(400px) translateZ(0) rotateY(360deg) scale(.95);
-    -webkit-animation-timing-function: ease-in;
-    animation-timing-function: ease-in;
-  }
-
-  100% {
-    -webkit-transform: perspective(400px) translateZ(0) rotateY(360deg) scale(1);
-    transform: perspective(400px) translateZ(0) rotateY(360deg) scale(1);
-    -webkit-animation-timing-function: ease-in;
-    animation-timing-function: ease-in;
-  }
-}
-
-@keyframes flip {
-  0% {
-    -webkit-transform: perspective(400px) translateZ(0) rotateY(0) scale(1);
-    -ms-transform: perspective(400px) translateZ(0) rotateY(0) scale(1);
-    transform: perspective(400px) translateZ(0) rotateY(0) scale(1);
-    -webkit-animation-timing-function: ease-out;
-    animation-timing-function: ease-out;
-  }
-
-  40% {
-    -webkit-transform: perspective(400px) translateZ(150px) rotateY(170deg) scale(1);
-    -ms-transform: perspective(400px) translateZ(150px) rotateY(170deg) scale(1);
-    transform: perspective(400px) translateZ(150px) rotateY(170deg) scale(1);
-    -webkit-animation-timing-function: ease-out;
-    animation-timing-function: ease-out;
-  }
-
-  50% {
-    -webkit-transform: perspective(400px) translateZ(150px) rotateY(190deg) scale(1);
-    -ms-transform: perspective(400px) translateZ(150px) rotateY(190deg) scale(1);
-    transform: perspective(400px) translateZ(150px) rotateY(190deg) scale(1);
-    -webkit-animation-timing-function: ease-in;
-    animation-timing-function: ease-in;
-  }
-
-  80% {
-    -webkit-transform: perspective(400px) translateZ(0) rotateY(360deg) scale(.95);
-    -ms-transform: perspective(400px) translateZ(0) rotateY(360deg) scale(.95);
-    transform: perspective(400px) translateZ(0) rotateY(360deg) scale(.95);
-    -webkit-animation-timing-function: ease-in;
-    animation-timing-function: ease-in;
-  }
-
-  100% {
-    -webkit-transform: perspective(400px) translateZ(0) rotateY(360deg) scale(1);
-    -ms-transform: perspective(400px) translateZ(0) rotateY(360deg) scale(1);
-    transform: perspective(400px) translateZ(0) rotateY(360deg) scale(1);
-    -webkit-animation-timing-function: ease-in;
-    animation-timing-function: ease-in;
-  }
-}
-
-.animated.flip {
-  -webkit-backface-visibility: visible;
-  -ms-backface-visibility: visible;
-  backface-visibility: visible;
-  -webkit-animation-name: flip;
-  animation-name: flip;
-}
-
-@-webkit-keyframes flipInX {
-  0% {
-    -webkit-transform: perspective(400px) rotateX(90deg);
-    transform: perspective(400px) rotateX(90deg);
-    opacity: 0;
-  }
-
-  40% {
-    -webkit-transform: perspective(400px) rotateX(-10deg);
-    transform: perspective(400px) rotateX(-10deg);
-  }
-
-  70% {
-    -webkit-transform: perspective(400px) rotateX(10deg);
-    transform: perspective(400px) rotateX(10deg);
-  }
-
-  100% {
-    -webkit-transform: perspective(400px) rotateX(0deg);
-    transform: perspective(400px) rotateX(0deg);
-    opacity: 1;
-  }
-}
-
-@keyframes flipInX {
-  0% {
-    -webkit-transform: perspective(400px) rotateX(90deg);
-    -ms-transform: perspective(400px) rotateX(90deg);
-    transform: perspective(400px) rotateX(90deg);
-    opacity: 0;
-  }
-
-  40% {
-    -webkit-transform: perspective(400px) rotateX(-10deg);
-    -ms-transform: perspective(400px) rotateX(-10deg);
-    transform: perspective(400px) rotateX(-10deg);
-  }
-
-  70% {
-    -webkit-transform: perspective(400px) rotateX(10deg);
-    -ms-transform: perspective(400px) rotateX(10deg);
-    transform: perspective(400px) rotateX(10deg);
-  }
-
-  100% {
-    -webkit-transform: perspective(400px) rotateX(0deg);
-    -ms-transform: perspective(400px) rotateX(0deg);
-    transform: perspective(400px) rotateX(0deg);
-    opacity: 1;
-  }
-}
-
-.flipInX {
-  -webkit-backface-visibility: visible !important;
-  -ms-backface-visibility: visible !important;
-  backface-visibility: visible !important;
-  -webkit-animation-name: flipInX;
-  animation-name: flipInX;
-}
-
-@-webkit-keyframes flipInY {
-  0% {
-    -webkit-transform: perspective(400px) rotateY(90deg);
-    transform: perspective(400px) rotateY(90deg);
-    opacity: 0;
-  }
-
-  40% {
-    -webkit-transform: perspective(400px) rotateY(-10deg);
-    transform: perspective(400px) rotateY(-10deg);
-  }
-
-  70% {
-    -webkit-transform: perspective(400px) rotateY(10deg);
-    transform: perspective(400px) rotateY(10deg);
-  }
-
-  100% {
-    -webkit-transform: perspective(400px) rotateY(0deg);
-    transform: perspective(400px) rotateY(0deg);
-    opacity: 1;
-  }
-}
-
-@keyframes flipInY {
-  0% {
-    -webkit-transform: perspective(400px) rotateY(90deg);
-    -ms-transform: perspective(400px) rotateY(90deg);
-    transform: perspective(400px) rotateY(90deg);
-    opacity: 0;
-  }
-
-  40% {
-    -webkit-transform: perspective(400px) rotateY(-10deg);
-    -ms-transform: perspective(400px) rotateY(-10deg);
-    transform: perspective(400px) rotateY(-10deg);
-  }
-
-  70% {
-    -webkit-transform: perspective(400px) rotateY(10deg);
-    -ms-transform: perspective(400px) rotateY(10deg);
-    transform: perspective(400px) rotateY(10deg);
-  }
-
-  100% {
-    -webkit-transform: perspective(400px) rotateY(0deg);
-    -ms-transform: perspective(400px) rotateY(0deg);
-    transform: perspective(400px) rotateY(0deg);
-    opacity: 1;
-  }
-}
-
-.flipInY {
-  -webkit-backface-visibility: visible !important;
-  -ms-backface-visibility: visible !important;
-  backface-visibility: visible !important;
-  -webkit-animation-name: flipInY;
-  animation-name: flipInY;
-}
-
-@-webkit-keyframes flipOutX {
-  0% {
-    -webkit-transform: perspective(400px) rotateX(0deg);
-    transform: perspective(400px) rotateX(0deg);
-    opacity: 1;
-  }
-
-  100% {
-    -webkit-transform: perspective(400px) rotateX(90deg);
-    transform: perspective(400px) rotateX(90deg);
-    opacity: 0;
-  }
-}
-
-@keyframes flipOutX {
-  0% {
-    -webkit-transform: perspective(400px) rotateX(0deg);
-    -ms-transform: perspective(400px) rotateX(0deg);
-    transform: perspective(400px) rotateX(0deg);
-    opacity: 1;
-  }
-
-  100% {
-    -webkit-transform: perspective(400px) rotateX(90deg);
-    -ms-transform: perspective(400px) rotateX(90deg);
-    transform: perspective(400px) rotateX(90deg);
-    opacity: 0;
-  }
-}
-
-.flipOutX {
-  -webkit-animation-name: flipOutX;
-  animation-name: flipOutX;
-  -webkit-backface-visibility: visible !important;
-  -ms-backface-visibility: visible !important;
-  backface-visibility: visible !important;
-}
-
-@-webkit-keyframes flipOutY {
-  0% {
-    -webkit-transform: perspective(400px) rotateY(0deg);
-    transform: perspective(400px) rotateY(0deg);
-    opacity: 1;
-  }
-
-  100% {
-    -webkit-transform: perspective(400px) rotateY(90deg);
-    transform: perspective(400px) rotateY(90deg);
-    opacity: 0;
-  }
-}
-
-@keyframes flipOutY {
-  0% {
-    -webkit-transform: perspective(400px) rotateY(0deg);
-    -ms-transform: perspective(400px) rotateY(0deg);
-    transform: perspective(400px) rotateY(0deg);
-    opacity: 1;
-  }
-
-  100% {
-    -webkit-transform: perspective(400px) rotateY(90deg);
-    -ms-transform: perspective(400px) rotateY(90deg);
-    transform: perspective(400px) rotateY(90deg);
-    opacity: 0;
-  }
-}
-
-.flipOutY {
-  -webkit-backface-visibility: visible !important;
-  -ms-backface-visibility: visible !important;
-  backface-visibility: visible !important;
-  -webkit-animation-name: flipOutY;
-  animation-name: flipOutY;
-}
-
-@-webkit-keyframes lightSpeedIn {
-  0% {
-    -webkit-transform: translateX(100%) skewX(-30deg);
-    transform: translateX(100%) skewX(-30deg);
-    opacity: 0;
-  }
-
-  60% {
-    -webkit-transform: translateX(-20%) skewX(30deg);
-    transform: translateX(-20%) skewX(30deg);
-    opacity: 1;
-  }
-
-  80% {
-    -webkit-transform: translateX(0%) skewX(-15deg);
-    transform: translateX(0%) skewX(-15deg);
-    opacity: 1;
-  }
-
-  100% {
-    -webkit-transform: translateX(0%) skewX(0deg);
-    transform: translateX(0%) skewX(0deg);
-    opacity: 1;
-  }
-}
-
-@keyframes lightSpeedIn {
-  0% {
-    -webkit-transform: translateX(100%) skewX(-30deg);
-    -ms-transform: translateX(100%) skewX(-30deg);
-    transform: translateX(100%) skewX(-30deg);
-    opacity: 0;
-  }
-
-  60% {
-    -webkit-transform: translateX(-20%) skewX(30deg);
-    -ms-transform: translateX(-20%) skewX(30deg);
-    transform: translateX(-20%) skewX(30deg);
-    opacity: 1;
-  }
-
-  80% {
-    -webkit-transform: translateX(0%) skewX(-15deg);
-    -ms-transform: translateX(0%) skewX(-15deg);
-    transform: translateX(0%) skewX(-15deg);
-    opacity: 1;
-  }
-
-  100% {
-    -webkit-transform: translateX(0%) skewX(0deg);
-    -ms-transform: translateX(0%) skewX(0deg);
-    transform: translateX(0%) skewX(0deg);
-    opacity: 1;
-  }
-}
-
-.lightSpeedIn {
-  -webkit-animation-name: lightSpeedIn;
-  animation-name: lightSpeedIn;
-  -webkit-animation-timing-function: ease-out;
-  animation-timing-function: ease-out;
-}
-
-@-webkit-keyframes lightSpeedOut {
-  0% {
-    -webkit-transform: translateX(0%) skewX(0deg);
-    transform: translateX(0%) skewX(0deg);
-    opacity: 1;
-  }
-
-  100% {
-    -webkit-transform: translateX(100%) skewX(-30deg);
-    transform: translateX(100%) skewX(-30deg);
-    opacity: 0;
-  }
-}
-
-@keyframes lightSpeedOut {
-  0% {
-    -webkit-transform: translateX(0%) skewX(0deg);
-    -ms-transform: translateX(0%) skewX(0deg);
-    transform: translateX(0%) skewX(0deg);
-    opacity: 1;
-  }
-
-  100% {
-    -webkit-transform: translateX(100%) skewX(-30deg);
-    -ms-transform: translateX(100%) skewX(-30deg);
-    transform: translateX(100%) skewX(-30deg);
-    opacity: 0;
-  }
-}
-
-.lightSpeedOut {
-  -webkit-animation-name: lightSpeedOut;
-  animation-name: lightSpeedOut;
-  -webkit-animation-timing-function: ease-in;
-  animation-timing-function: ease-in;
-}
-
-@-webkit-keyframes rotateIn {
-  0% {
-    -webkit-transform-origin: center center;
-    transform-origin: center center;
-    -webkit-transform: rotate(-200deg);
-    transform: rotate(-200deg);
-    opacity: 0;
-  }
-
-  100% {
-    -webkit-transform-origin: center center;
-    transform-origin: center center;
-    -webkit-transform: rotate(0);
-    transform: rotate(0);
-    opacity: 1;
-  }
-}
-
-@keyframes rotateIn {
-  0% {
-    -webkit-transform-origin: center center;
-    -ms-transform-origin: center center;
-    transform-origin: center center;
-    -webkit-transform: rotate(-200deg);
-    -ms-transform: rotate(-200deg);
-    transform: rotate(-200deg);
-    opacity: 0;
-  }
-
-  100% {
-    -webkit-transform-origin: center center;
-    -ms-transform-origin: center center;
-    transform-origin: center center;
-    -webkit-transform: rotate(0);
-    -ms-transform: rotate(0);
-    transform: rotate(0);
-    opacity: 1;
-  }
-}
-
-.rotateIn {
-  -webkit-animation-name: rotateIn;
-  animation-name: rotateIn;
-}
-
-@-webkit-keyframes rotateInDownLeft {
-  0% {
-    -webkit-transform-origin: left bottom;
-    transform-origin: left bottom;
-    -webkit-transform: rotate(-90deg);
-    transform: rotate(-90deg);
-    opacity: 0;
-  }
-
-  100% {
-    -webkit-transform-origin: left bottom;
-    transform-origin: left bottom;
-    -webkit-transform: rotate(0);
-    transform: rotate(0);
-    opacity: 1;
-  }
-}
-
-@keyframes rotateInDownLeft {
-  0% {
-    -webkit-transform-origin: left bottom;
-    -ms-transform-origin: left bottom;
-    transform-origin: left bottom;
-    -webkit-transform: rotate(-90deg);
-    -ms-transform: rotate(-90deg);
-    transform: rotate(-90deg);
-    opacity: 0;
-  }
-
-  100% {
-    -webkit-transform-origin: left bottom;
-    -ms-transform-origin: left bottom;
-    transform-origin: left bottom;
-    -webkit-transform: rotate(0);
-    -ms-transform: rotate(0);
-    transform: rotate(0);
-    opacity: 1;
-  }
-}
-
-.rotateInDownLeft {
-  -webkit-animation-name: rotateInDownLeft;
-  animation-name: rotateInDownLeft;
-}
-
-@-webkit-keyframes rotateInDownRight {
-  0% {
-    -webkit-transform-origin: right bottom;
-    transform-origin: right bottom;
-    -webkit-transform: rotate(90deg);
-    transform: rotate(90deg);
-    opacity: 0;
-  }
-
-  100% {
-    -webkit-transform-origin: right bottom;
-    transform-origin: right bottom;
-    -webkit-transform: rotate(0);
-    transform: rotate(0);
-    opacity: 1;
-  }
-}
-
-@keyframes rotateInDownRight {
-  0% {
-    -webkit-transform-origin: right bottom;
-    -ms-transform-origin: right bottom;
-    transform-origin: right bottom;
-    -webkit-transform: rotate(90deg);
-    -ms-transform: rotate(90deg);
-    transform: rotate(90deg);
-    opacity: 0;
-  }
-
-  100% {
-    -webkit-transform-origin: right bottom;
-    -ms-transform-origin: right bottom;
-    transform-origin: right bottom;
-    -webkit-transform: rotate(0);
-    -ms-transform: rotate(0);
-    transform: rotate(0);
-    opacity: 1;
-  }
-}
-
-.rotateInDownRight {
-  -webkit-animation-name: rotateInDownRight;
-  animation-name: rotateInDownRight;
-}
-
-@-webkit-keyframes rotateInUpLeft {
-  0% {
-    -webkit-transform-origin: left bottom;
-    transform-origin: left bottom;
-    -webkit-transform: rotate(90deg);
-    transform: rotate(90deg);
-    opacity: 0;
-  }
-
-  100% {
-    -webkit-transform-origin: left bottom;
-    transform-origin: left bottom;
-    -webkit-transform: rotate(0);
-    transform: rotate(0);
-    opacity: 1;
-  }
-}
-
-@keyframes rotateInUpLeft {
-  0% {
-    -webkit-transform-origin: left bottom;
-    -ms-transform-origin: left bottom;
-    transform-origin: left bottom;
-    -webkit-transform: rotate(90deg);
-    -ms-transform: rotate(90deg);
-    transform: rotate(90deg);
-    opacity: 0;
-  }
-
-  100% {
-    -webkit-transform-origin: left bottom;
-    -ms-transform-origin: left bottom;
-    transform-origin: left bottom;
-    -webkit-transform: rotate(0);
-    -ms-transform: rotate(0);
-    transform: rotate(0);
-    opacity: 1;
-  }
-}
-
-.rotateInUpLeft {
-  -webkit-animation-name: rotateInUpLeft;
-  animation-name: rotateInUpLeft;
-}
-
-@-webkit-keyframes rotateInUpRight {
-  0% {
-    -webkit-transform-origin: right bottom;
-    transform-origin: right bottom;
-    -webkit-transform: rotate(-90deg);
-    transform: rotate(-90deg);
-    opacity: 0;
-  }
-
-  100% {
-    -webkit-transform-origin: right bottom;
-    transform-origin: right bottom;
-    -webkit-transform: rotate(0);
-    transform: rotate(0);
-    opacity: 1;
-  }
-}
-
-@keyframes rotateInUpRight {
-  0% {
-    -webkit-transform-origin: right bottom;
-    -ms-transform-origin: right bottom;
-    transform-origin: right bottom;
-    -webkit-transform: rotate(-90deg);
-    -ms-transform: rotate(-90deg);
-    transform: rotate(-90deg);
-    opacity: 0;
-  }
-
-  100% {
-    -webkit-transform-origin: right bottom;
-    -ms-transform-origin: right bottom;
-    transform-origin: right bottom;
-    -webkit-transform: rotate(0);
-    -ms-transform: rotate(0);
-    transform: rotate(0);
-    opacity: 1;
-  }
-}
-
-.rotateInUpRight {
-  -webkit-animation-name: rotateInUpRight;
-  animation-name: rotateInUpRight;
-}
-
-@-webkit-keyframes rotateOut {
-  0% {
-    -webkit-transform-origin: center center;
-    transform-origin: center center;
-    -webkit-transform: rotate(0);
-    transform: rotate(0);
-    opacity: 1;
-  }
-
-  100% {
-    -webkit-transform-origin: center center;
-    transform-origin: center center;
-    -webkit-transform: rotate(200deg);
-    transform: rotate(200deg);
-    opacity: 0;
-  }
-}
-
-@keyframes rotateOut {
-  0% {
-    -webkit-transform-origin: center center;
-    -ms-transform-origin: center center;
-    transform-origin: center center;
-    -webkit-transform: rotate(0);
-    -ms-transform: rotate(0);
-    transform: rotate(0);
-    opacity: 1;
-  }
-
-  100% {
-    -webkit-transform-origin: center center;
-    -ms-transform-origin: center center;
-    transform-origin: center center;
-    -webkit-transform: rotate(200deg);
-    -ms-transform: rotate(200deg);
-    transform: rotate(200deg);
-    opacity: 0;
-  }
-}
-
-.rotateOut {
-  -webkit-animation-name: rotateOut;
-  animation-name: rotateOut;
-}
-
-@-webkit-keyframes rotateOutDownLeft {
-  0% {
-    -webkit-transform-origin: left bottom;
-    transform-origin: left bottom;
-    -webkit-transform: rotate(0);
-    transform: rotate(0);
-    opacity: 1;
-  }
-
-  100% {
-    -webkit-transform-origin: left bottom;
-    transform-origin: left bottom;
-    -webkit-transform: rotate(90deg);
-    transform: rotate(90deg);
-    opacity: 0;
-  }
-}
-
-@keyframes rotateOutDownLeft {
-  0% {
-    -webkit-transform-origin: left bottom;
-    -ms-transform-origin: left bottom;
-    transform-origin: left bottom;
-    -webkit-transform: rotate(0);
-    -ms-transform: rotate(0);
-    transform: rotate(0);
-    opacity: 1;
-  }
-
-  100% {
-    -webkit-transform-origin: left bottom;
-    -ms-transform-origin: left bottom;
-    transform-origin: left bottom;
-    -webkit-transform: rotate(90deg);
-    -ms-transform: rotate(90deg);
-    transform: rotate(90deg);
-    opacity: 0;
-  }
-}
-
-.rotateOutDownLeft {
-  -webkit-animation-name: rotateOutDownLeft;
-  animation-name: rotateOutDownLeft;
-}
-
-@-webkit-keyframes rotateOutDownRight {
-  0% {
-    -webkit-transform-origin: right bottom;
-    transform-origin: right bottom;
-    -webkit-transform: rotate(0);
-    transform: rotate(0);
-    opacity: 1;
-  }
-
-  100% {
-    -webkit-transform-origin: right bottom;
-    transform-origin: right bottom;
-    -webkit-transform: rotate(-90deg);
-    transform: rotate(-90deg);
-    opacity: 0;
-  }
-}
-
-@keyframes rotateOutDownRight {
-  0% {
-    -webkit-transform-origin: right bottom;
-    -ms-transform-origin: right bottom;
-    transform-origin: right bottom;
-    -webkit-transform: rotate(0);
-    -ms-transform: rotate(0);
-    transform: rotate(0);
-    opacity: 1;
-  }
-
-  100% {
-    -webkit-transform-origin: right bottom;
-    -ms-transform-origin: right bottom;
-    transform-origin: right bottom;
-    -webkit-transform: rotate(-90deg);
-    -ms-transform: rotate(-90deg);
-    transform: rotate(-90deg);
-    opacity: 0;
-  }
-}
-
-.rotateOutDownRight {
-  -webkit-animation-name: rotateOutDownRight;
-  animation-name: rotateOutDownRight;
-}
-
-@-webkit-keyframes rotateOutUpLeft {
-  0% {
-    -webkit-transform-origin: left bottom;
-    transform-origin: left bottom;
-    -webkit-transform: rotate(0);
-    transform: rotate(0);
-    opacity: 1;
-  }
-
-  100% {
-    -webkit-transform-origin: left bottom;
-    transform-origin: left bottom;
-    -webkit-transform: rotate(-90deg);
-    transform: rotate(-90deg);
-    opacity: 0;
-  }
-}
-
-@keyframes rotateOutUpLeft {
-  0% {
-    -webkit-transform-origin: left bottom;
-    -ms-transform-origin: left bottom;
-    transform-origin: left bottom;
-    -webkit-transform: rotate(0);
-    -ms-transform: rotate(0);
-    transform: rotate(0);
-    opacity: 1;
-  }
-
-  100% {
-    -webkit-transform-origin: left bottom;
-    -ms-transform-origin: left bottom;
-    transform-origin: left bottom;
-    -webkit-transform: rotate(-90deg);
-    -ms-transform: rotate(-90deg);
-    transform: rotate(-90deg);
-    opacity: 0;
-  }
-}
-
-.rotateOutUpLeft {
-  -webkit-animation-name: rotateOutUpLeft;
-  animation-name: rotateOutUpLeft;
-}
-
-@-webkit-keyframes rotateOutUpRight {
-  0% {
-    -webkit-transform-origin: right bottom;
-    transform-origin: right bottom;
-    -webkit-transform: rotate(0);
-    transform: rotate(0);
-    opacity: 1;
-  }
-
-  100% {
-    -webkit-transform-origin: right bottom;
-    transform-origin: right bottom;
-    -webkit-transform: rotate(90deg);
-    transform: rotate(90deg);
-    opacity: 0;
-  }
-}
-
-@keyframes rotateOutUpRight {
-  0% {
-    -webkit-transform-origin: right bottom;
-    -ms-transform-origin: right bottom;
-    transform-origin: right bottom;
-    -webkit-transform: rotate(0);
-    -ms-transform: rotate(0);
-    transform: rotate(0);
-    opacity: 1;
-  }
-
-  100% {
-    -webkit-transform-origin: right bottom;
-    -ms-transform-origin: right bottom;
-    transform-origin: right bottom;
-    -webkit-transform: rotate(90deg);
-    -ms-transform: rotate(90deg);
-    transform: rotate(90deg);
-    opacity: 0;
-  }
-}
-
-.rotateOutUpRight {
-  -webkit-animation-name: rotateOutUpRight;
-  animation-name: rotateOutUpRight;
-}
-
-@-webkit-keyframes slideInDown {
-  0% {
-    opacity: 0;
-    -webkit-transform: translateY(-2000px);
-    transform: translateY(-2000px);
-  }
-
-  100% {
-    -webkit-transform: translateY(0);
-    transform: translateY(0);
-  }
-}
-
-@keyframes slideInDown {
-  0% {
-    opacity: 0;
-    -webkit-transform: translateY(-2000px);
-    -ms-transform: translateY(-2000px);
-    transform: translateY(-2000px);
-  }
-
-  100% {
-    -webkit-transform: translateY(0);
-    -ms-transform: translateY(0);
-    transform: translateY(0);
-  }
-}
-
-.slideInDown {
-  -webkit-animation-name: slideInDown;
-  animation-name: slideInDown;
-}
-
-@-webkit-keyframes slideInLeft {
-  0% {
-    opacity: 0;
-    -webkit-transform: translateX(-2000px);
-    transform: translateX(-2000px);
-  }
-
-  100% {
-    -webkit-transform: translateX(0);
-    transform: translateX(0);
-  }
-}
-
-@keyframes slideInLeft {
-  0% {
-    opacity: 0;
-    -webkit-transform: translateX(-2000px);
-    -ms-transform: translateX(-2000px);
-    transform: translateX(-2000px);
-  }
-
-  100% {
-    -webkit-transform: translateX(0);
-    -ms-transform: translateX(0);
-    transform: translateX(0);
-  }
-}
-
-.slideInLeft {
-  -webkit-animation-name: slideInLeft;
-  animation-name: slideInLeft;
-}
-
-@-webkit-keyframes slideInRight {
-  0% {
-    opacity: 0;
-    -webkit-transform: translateX(2000px);
-    transform: translateX(2000px);
-  }
-
-  100% {
-    -webkit-transform: translateX(0);
-    transform: translateX(0);
-  }
-}
-
-@keyframes slideInRight {
-  0% {
-    opacity: 0;
-    -webkit-transform: translateX(2000px);
-    -ms-transform: translateX(2000px);
-    transform: translateX(2000px);
-  }
-
-  100% {
-    -webkit-transform: translateX(0);
-    -ms-transform: translateX(0);
-    transform: translateX(0);
-  }
-}
-
-.slideInRight {
-  -webkit-animation-name: slideInRight;
-  animation-name: slideInRight;
-}
-
-@-webkit-keyframes slideOutLeft {
-  0% {
-    -webkit-transform: translateX(0);
-    transform: translateX(0);
-  }
-
-  100% {
-    opacity: 0;
-    -webkit-transform: translateX(-2000px);
-    transform: translateX(-2000px);
-  }
-}
-
-@keyframes slideOutLeft {
-  0% {
-    -webkit-transform: translateX(0);
-    -ms-transform: translateX(0);
-    transform: translateX(0);
-  }
-
-  100% {
-    opacity: 0;
-    -webkit-transform: translateX(-2000px);
-    -ms-transform: translateX(-2000px);
-    transform: translateX(-2000px);
-  }
-}
-
-.slideOutLeft {
-  -webkit-animation-name: slideOutLeft;
-  animation-name: slideOutLeft;
-}
-
-@-webkit-keyframes slideOutRight {
-  0% {
-    -webkit-transform: translateX(0);
-    transform: translateX(0);
-  }
-
-  100% {
-    opacity: 0;
-    -webkit-transform: translateX(2000px);
-    transform: translateX(2000px);
-  }
-}
-
-@keyframes slideOutRight {
-  0% {
-    -webkit-transform: translateX(0);
-    -ms-transform: translateX(0);
-    transform: translateX(0);
-  }
-
-  100% {
-    opacity: 0;
-    -webkit-transform: translateX(2000px);
-    -ms-transform: translateX(2000px);
-    transform: translateX(2000px);
-  }
-}
-
-.slideOutRight {
-  -webkit-animation-name: slideOutRight;
-  animation-name: slideOutRight;
-}
-
-@-webkit-keyframes slideOutUp {
-  0% {
-    -webkit-transform: translateY(0);
-    transform: translateY(0);
-  }
-
-  100% {
-    opacity: 0;
-    -webkit-transform: translateY(-2000px);
-    transform: translateY(-2000px);
-  }
-}
-
-@keyframes slideOutUp {
-  0% {
-    -webkit-transform: translateY(0);
-    -ms-transform: translateY(0);
-    transform: translateY(0);
-  }
-
-  100% {
-    opacity: 0;
-    -webkit-transform: translateY(-2000px);
-    -ms-transform: translateY(-2000px);
-    transform: translateY(-2000px);
-  }
-}
-
-.slideOutUp {
-  -webkit-animation-name: slideOutUp;
-  animation-name: slideOutUp;
-}
-
-@-webkit-keyframes hinge {
-  0% {
-    -webkit-transform: rotate(0);
-    transform: rotate(0);
-    -webkit-transform-origin: top left;
-    transform-origin: top left;
-    -webkit-animation-timing-function: ease-in-out;
-    animation-timing-function: ease-in-out;
-  }
-
-  20%, 60% {
-    -webkit-transform: rotate(80deg);
-    transform: rotate(80deg);
-    -webkit-transform-origin: top left;
-    transform-origin: top left;
-    -webkit-animation-timing-function: ease-in-out;
-    animation-timing-function: ease-in-out;
-  }
-
-  40% {
-    -webkit-transform: rotate(60deg);
-    transform: rotate(60deg);
-    -webkit-transform-origin: top left;
-    transform-origin: top left;
-    -webkit-animation-timing-function: ease-in-out;
-    animation-timing-function: ease-in-out;
-  }
-
-  80% {
-    -webkit-transform: rotate(60deg) translateY(0);
-    transform: rotate(60deg) translateY(0);
-    opacity: 1;
-    -webkit-transform-origin: top left;
-    transform-origin: top left;
-    -webkit-animation-timing-function: ease-in-out;
-    animation-timing-function: ease-in-out;
-  }
-
-  100% {
-    -webkit-transform: translateY(700px);
-    transform: translateY(700px);
-    opacity: 0;
-  }
-}
-
-@keyframes hinge {
-  0% {
-    -webkit-transform: rotate(0);
-    -ms-transform: rotate(0);
-    transform: rotate(0);
-    -webkit-transform-origin: top left;
-    -ms-transform-origin: top left;
-    transform-origin: top left;
-    -webkit-animation-timing-function: ease-in-out;
-    animation-timing-function: ease-in-out;
-  }
-
-  20%, 60% {
-    -webkit-transform: rotate(80deg);
-    -ms-transform: rotate(80deg);
-    transform: rotate(80deg);
-    -webkit-transform-origin: top left;
-    -ms-transform-origin: top left;
-    transform-origin: top left;
-    -webkit-animation-timing-function: ease-in-out;
-    animation-timing-function: ease-in-out;
-  }
-
-  40% {
-    -webkit-transform: rotate(60deg);
-    -ms-transform: rotate(60deg);
-    transform: rotate(60deg);
-    -webkit-transform-origin: top left;
-    -ms-transform-origin: top left;
-    transform-origin: top left;
-    -webkit-animation-timing-function: ease-in-out;
-    animation-timing-function: ease-in-out;
-  }
-
-  80% {
-    -webkit-transform: rotate(60deg) translateY(0);
-    -ms-transform: rotate(60deg) translateY(0);
-    transform: rotate(60deg) translateY(0);
-    opacity: 1;
-    -webkit-transform-origin: top left;
-    -ms-transform-origin: top left;
-    transform-origin: top left;
-    -webkit-animation-timing-function: ease-in-out;
-    animation-timing-function: ease-in-out;
-  }
-
-  100% {
-    -webkit-transform: translateY(700px);
-    -ms-transform: translateY(700px);
-    transform: translateY(700px);
-    opacity: 0;
-  }
-}
-
-.hinge {
-  -webkit-animation-name: hinge;
-  animation-name: hinge;
-}
-
-/* originally authored by Nick Pettit - https://github.com/nickpettit/glide */
-
-@-webkit-keyframes rollIn {
-  0% {
-    opacity: 0;
-    -webkit-transform: translateX(-100%) rotate(-120deg);
-    transform: translateX(-100%) rotate(-120deg);
-  }
-
-  100% {
-    opacity: 1;
-    -webkit-transform: translateX(0px) rotate(0deg);
-    transform: translateX(0px) rotate(0deg);
-  }
-}
-
-@keyframes rollIn {
-  0% {
-    opacity: 0;
-    -webkit-transform: translateX(-100%) rotate(-120deg);
-    -ms-transform: translateX(-100%) rotate(-120deg);
-    transform: translateX(-100%) rotate(-120deg);
-  }
-
-  100% {
-    opacity: 1;
-    -webkit-transform: translateX(0px) rotate(0deg);
-    -ms-transform: translateX(0px) rotate(0deg);
-    transform: translateX(0px) rotate(0deg);
-  }
-}
-
-.rollIn {
-  -webkit-animation-name: rollIn;
-  animation-name: rollIn;
-}
-
-/* originally authored by Nick Pettit - https://github.com/nickpettit/glide */
-
-@-webkit-keyframes rollOut {
-  0% {
-    opacity: 1;
-    -webkit-transform: translateX(0px) rotate(0deg);
-    transform: translateX(0px) rotate(0deg);
-  }
-
-  100% {
-    opacity: 0;
-    -webkit-transform: translateX(100%) rotate(120deg);
-    transform: translateX(100%) rotate(120deg);
-  }
-}
-
-@keyframes rollOut {
-  0% {
-    opacity: 1;
-    -webkit-transform: translateX(0px) rotate(0deg);
-    -ms-transform: translateX(0px) rotate(0deg);
-    transform: translateX(0px) rotate(0deg);
-  }
-
-  100% {
-    opacity: 0;
-    -webkit-transform: translateX(100%) rotate(120deg);
-    -ms-transform: translateX(100%) rotate(120deg);
-    transform: translateX(100%) rotate(120deg);
-  }
-}
-
-.rollOut {
-  -webkit-animation-name: rollOut;
-  animation-name: rollOut;
-}
\ No newline at end of file


[43/52] [abbrv] kylin git commit: KYLIN-1416 remove website from coding branch

Posted by li...@apache.org.
KYLIN-1416 remove website from coding branch


Project: http://git-wip-us.apache.org/repos/asf/kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/0a74e9cb
Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/0a74e9cb
Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/0a74e9cb

Branch: refs/heads/1.x-HBase1.1.3
Commit: 0a74e9cb46797dbbc1ba095b6b77b3700cf3ee6b
Parents: fc66cdb
Author: Yang Li <li...@apache.org>
Authored: Thu Feb 11 20:51:20 2016 +0800
Committer: Yang Li <li...@apache.org>
Committed: Thu Feb 11 20:51:20 2016 +0800

----------------------------------------------------------------------
 website/.gitignore                              |    4 -
 website/BingSiteAuth.xml                        |   18 -
 website/CNAME                                   |    1 -
 website/LICENSE                                 |  202 -
 website/README.md                               |   64 -
 website/_config.yml                             |   62 -
 website/_config_prod.yml                        |   24 -
 website/_data/contributors.yml                  |  118 -
 website/_data/development.yml                   |   30 -
 website/_data/docs-cn.yml                       |   20 -
 website/_data/docs.yml                          |   58 -
 website/_dev/about_temp_files.md                |   29 -
 website/_dev/dev_env.md                         |  129 -
 website/_dev/howto_contribute.md                |   28 -
 website/_dev/howto_docs.md                      |  152 -
 website/_dev/howto_package.md                   |   22 -
 website/_dev/howto_release.md                   |  430 --
 website/_dev/howto_test.md                      |   49 -
 website/_dev/index.md                           |   17 -
 website/_dev/new_metadata.md                    |   30 -
 website/_dev/web_tech.md                        |   46 -
 website/_docs/gettingstarted/concepts.md        |   65 -
 website/_docs/gettingstarted/events.md          |   27 -
 website/_docs/gettingstarted/faq.md             |   90 -
 website/_docs/gettingstarted/terminology.md     |   26 -
 website/_docs/howto/howto_backup_hbase.md       |   29 -
 website/_docs/howto/howto_backup_metadata.md    |   62 -
 .../howto/howto_build_cube_with_restapi.md      |   55 -
 website/_docs/howto/howto_cleanup_storage.md    |   23 -
 website/_docs/howto/howto_jdbc.md               |   94 -
 website/_docs/howto/howto_ldap_and_sso.md       |  124 -
 website/_docs/howto/howto_optimize_cubes.md     |  214 -
 website/_docs/howto/howto_upgrade.md            |  103 -
 website/_docs/howto/howto_use_restapi.md        | 1006 ----
 website/_docs/howto/howto_use_restapi_in_js.md  |   48 -
 website/_docs/index.cn.md                       |   22 -
 website/_docs/index.md                          |   52 -
 website/_docs/install/advance_settings.md       |   45 -
 website/_docs/install/hadoop_evn.md             |   35 -
 website/_docs/install/index.cn.md               |   46 -
 website/_docs/install/index.md                  |   47 -
 website/_docs/install/kylin_cluster.md          |   30 -
 website/_docs/install/kylin_docker.md           |   46 -
 .../_docs/install/manual_install_guide.cn.md    |   48 -
 website/_docs/install/manual_install_guide.md   |   48 -
 website/_docs/release_notes.md                  |  706 ---
 website/_docs/tutorial/acl.cn.md                |   35 -
 website/_docs/tutorial/acl.md                   |   35 -
 website/_docs/tutorial/create_cube.cn.md        |  129 -
 website/_docs/tutorial/create_cube.md           |  129 -
 website/_docs/tutorial/cube_build_job.cn.md     |   66 -
 website/_docs/tutorial/cube_build_job.md        |   66 -
 website/_docs/tutorial/kylin_sample.md          |   23 -
 website/_docs/tutorial/odbc.cn.md               |   34 -
 website/_docs/tutorial/odbc.md                  |   50 -
 website/_docs/tutorial/powerbi.cn.md            |   56 -
 website/_docs/tutorial/powerbi.md               |   55 -
 website/_docs/tutorial/tableau.cn.md            |  116 -
 website/_docs/tutorial/tableau.md               |  115 -
 website/_docs/tutorial/tableau_91.cn.md         |   50 -
 website/_docs/tutorial/tableau_91.md            |   51 -
 website/_docs/tutorial/web.cn.md                |  139 -
 website/_docs/tutorial/web.md                   |  139 -
 website/_includes/dev_nav.html                  |   33 -
 website/_includes/dev_ul.html                   |   26 -
 website/_includes/docs_nav.cn.html              |   33 -
 website/_includes/docs_nav.html                 |   33 -
 website/_includes/docs_ul.cn.html               |   28 -
 website/_includes/docs_ul.html                  |   29 -
 website/_includes/footer.html                   |   42 -
 website/_includes/head.cn.html                  |   69 -
 website/_includes/head.html                     |   69 -
 website/_includes/header.cn.html                |   58 -
 website/_includes/header.html                   |   61 -
 website/_includes/navside.cn.html               |  112 -
 website/_includes/navside.html                  |   90 -
 website/_layouts/blog.html                      |   47 -
 website/_layouts/default-cn.html                |   41 -
 website/_layouts/default.html                   |   48 -
 website/_layouts/dev.html                       |   46 -
 website/_layouts/docs-cn.html                   |   46 -
 website/_layouts/docs.html                      |   50 -
 website/_layouts/download-cn.html               |   53 -
 website/_layouts/download.html                  |   53 -
 website/_layouts/page.html                      |   33 -
 website/_layouts/post-blog.html                 |   38 -
 website/_layouts/post-cn.html                   |   38 -
 website/_layouts/post.html                      |   38 -
 .../blog/2015-01-25-introduce-data-model.md     |   40 -
 .../2015-06-10-release-v0.7.1-incubating.md     |  149 -
 .../_posts/blog/2015-08-13-kylin-dictionary.md  |   68 -
 website/_posts/blog/2015-08-15-fast-cubing.md   |  119 -
 .../blog/2015-09-06-release-v1.0-incubating.md  |   44 -
 .../blog/2015-09-09-fast-cubing-on-spark.md     |  117 -
 website/_posts/blog/2015-09-22-hybrid-model.md  |  136 -
 .../blog/2015-10-14-Apache-Kylin-Meetup.md      |   63 -
 .../2015-10-25-release-v1.1-incubating.cn.md    |   53 -
 .../blog/2015-10-25-release-v1.1-incubating.md  |   53 -
 .../_posts/blog/2015-12-23-release-v1.2.cn.md   |   54 -
 website/_posts/blog/2015-12-23-release-v1.2.md  |   54 -
 .../2015-12-25-support-powerbi-tableau9.cn.md   |   28 -
 .../blog/2015-12-25-support-powerbi-tableau9.md |   28 -
 website/_sass/_base.scss                        |  221 -
 website/_sass/_layout.scss                      |  253 -
 website/_sass/_syntax-highlighting.scss         |   84 -
 website/about/index.md                          |   35 -
 website/assets/css/animate.css                  | 2744 ----------
 website/assets/css/bootstrap.min.css            | 5083 ------------------
 website/assets/css/docs.css                     |  126 -
 website/assets/css/font-awesome.min.css         |    4 -
 website/assets/css/pygments.css                 |   61 -
 website/assets/css/styles.css                   |  656 ---
 website/assets/fonts/FontAwesome.otf            |  Bin 85908 -> 0 bytes
 website/assets/fonts/fontawesome-webfont.eot    |  Bin 56006 -> 0 bytes
 website/assets/fonts/fontawesome-webfont.svg    |  520 --
 website/assets/fonts/fontawesome-webfont.ttf    |  Bin 112160 -> 0 bytes
 website/assets/fonts/fontawesome-webfont.woff   |  Bin 65452 -> 0 bytes
 website/assets/fonts/helveticaneue-webfont.css  |    8 -
 website/assets/fonts/helveticaneue-webfont.eot  |  Bin 50668 -> 0 bytes
 website/assets/fonts/helveticaneue-webfont.svg  |  693 ---
 website/assets/fonts/helveticaneue-webfont.ttf  |  Bin 50456 -> 0 bytes
 website/assets/fonts/helveticaneue-webfont.woff |  Bin 26140 -> 0 bytes
 website/assets/fonts/helveticaneue_lt.css       |    8 -
 website/assets/fonts/helveticaneue_lt.eot       |  Bin 35346 -> 0 bytes
 website/assets/fonts/helveticaneue_lt.svg       |  573 --
 website/assets/fonts/helveticaneue_lt.ttf       |  Bin 35092 -> 0 bytes
 website/assets/fonts/helveticaneue_lt.woff      |  Bin 18208 -> 0 bytes
 website/assets/fonts/opensans-light-webfont.eot |  Bin 20556 -> 0 bytes
 website/assets/fonts/opensans-light-webfont.svg | 1824 -------
 website/assets/fonts/opensans-light-webfont.ttf |  Bin 44484 -> 0 bytes
 .../assets/fonts/opensans-light-webfont.woff    |  Bin 24100 -> 0 bytes
 .../assets/fonts/opensans-regular-webfont.eot   |  Bin 21103 -> 0 bytes
 .../assets/fonts/opensans-regular-webfont.svg   | 1824 -------
 .../assets/fonts/opensans-regular-webfont.ttf   |  Bin 45112 -> 0 bytes
 .../assets/fonts/opensans-regular-webfont.woff  |  Bin 24736 -> 0 bytes
 website/assets/images/core.png                  |  Bin 70377 -> 0 bytes
 website/assets/images/data.png                  |  Bin 913990 -> 0 bytes
 website/assets/images/egg-logo.png              |  Bin 8626 -> 0 bytes
 website/assets/images/feather-small.gif         |  Bin 7129 -> 0 bytes
 website/assets/images/kylin_diagram.png         |  Bin 441427 -> 0 bytes
 website/assets/images/kylin_logo.jpg            |  Bin 120174 -> 0 bytes
 website/assets/js/bootstrap.min.js              |    6 -
 website/assets/js/jquery-1.9.1.min.js           |    5 -
 website/assets/js/main.js                       |   74 -
 website/assets/js/nnav.js                       |   16 -
 website/assets/js/nside.js                      |   13 -
 website/blog/index.md                           |   39 -
 website/community/index.md                      |   60 -
 website/community/poweredby.md                  |   33 -
 website/css/main.scss                           |   52 -
 website/download/index.cn.md                    |   40 -
 website/download/index.md                       |   44 -
 website/fav.png                                 |  Bin 9450 -> 0 bytes
 website/favicon.ico                             |  Bin 1150 -> 0 bytes
 website/feed.xml                                |   44 -
 website/google0f781a4ad9402c21.html             |   19 -
 .../1 action-build.png                          |  Bin 93893 -> 0 bytes
 .../2 pop-up.png                                |  Bin 40055 -> 0 bytes
 .../3 end-date.png                              |  Bin 66610 -> 0 bytes
 .../4 submit.png                                |  Bin 36292 -> 0 bytes
 .../4.1 success.png                             |  Bin 14426 -> 0 bytes
 .../5 jobs-page.png                             |  Bin 96398 -> 0 bytes
 .../6 discard.png                               |  Bin 55747 -> 0 bytes
 .../7 job-steps.png                             |  Bin 157315 -> 0 bytes
 .../8 hover-step.png                            |  Bin 90405 -> 0 bytes
 .../9 log-d.png                                 |  Bin 85634 -> 0 bytes
 .../9 log.png                                   |  Bin 42840 -> 0 bytes
 .../9 mrjob-d.png                               |  Bin 197651 -> 0 bytes
 .../9 mrjob.png                                 |  Bin 43079 -> 0 bytes
 .../9 parameters-d.png                          |  Bin 102501 -> 0 bytes
 .../9 parameters.png                            |  Bin 43674 -> 0 bytes
 .../1 manage-prject.png                         |  Bin 44662 -> 0 bytes
 .../Kylin-Cube-Creation-Tutorial/10 filter.png  |  Bin 71620 -> 0 bytes
 .../11 refresh-setting1.png                     |  Bin 125565 -> 0 bytes
 .../11 refresh-setting2.png                     |  Bin 94895 -> 0 bytes
 .../12 advanced.png                             |  Bin 63237 -> 0 bytes
 .../13 overview.png                             |  Bin 59024 -> 0 bytes
 .../Kylin-Cube-Creation-Tutorial/2 +project.png |  Bin 57637 -> 0 bytes
 .../3 new-project.png                           |  Bin 35511 -> 0 bytes
 .../3.1 pj-created.png                          |  Bin 13207 -> 0 bytes
 .../Kylin-Cube-Creation-Tutorial/4 +table.png   |  Bin 58675 -> 0 bytes
 .../5 hive-table.png                            |  Bin 29548 -> 0 bytes
 .../Kylin-Cube-Creation-Tutorial/6 +cube.png    |  Bin 79558 -> 0 bytes
 .../7 cube-info.png                             |  Bin 159724 -> 0 bytes
 .../Kylin-Cube-Creation-Tutorial/8 dim-+dim.png |  Bin 56207 -> 0 bytes
 .../Kylin-Cube-Creation-Tutorial/8 dim-edit.png |  Bin 95337 -> 0 bytes
 .../8 dim-factable.png                          |  Bin 68022 -> 0 bytes
 .../8 dim-typeA.png                             |  Bin 61561 -> 0 bytes
 .../8 dim-typeB-1.png                           |  Bin 54939 -> 0 bytes
 .../8 dim-typeB-2.png                           |  Bin 50999 -> 0 bytes
 .../8 dim-typeC.png                             |  Bin 62033 -> 0 bytes
 .../8 dim-typeD.png                             |  Bin 54444 -> 0 bytes
 .../9 meas-+meas.png                            |  Bin 49541 -> 0 bytes
 .../9 meas-count.png                            |  Bin 49857 -> 0 bytes
 .../9 meas-distinct.png                         |  Bin 57420 -> 0 bytes
 .../Kylin-Cube-Creation-Tutorial/9 meas-max.png |  Bin 51820 -> 0 bytes
 .../Kylin-Cube-Creation-Tutorial/9 meas-min.png |  Bin 47379 -> 0 bytes
 .../Kylin-Cube-Creation-Tutorial/9 meas-sum.png |  Bin 44017 -> 0 bytes
 .../14 +grant.png                               |  Bin 96078 -> 0 bytes
 .../15 grantInfo.png                            |  Bin 131013 -> 0 bytes
 .../16 grant-user.png                           |  Bin 70868 -> 0 bytes
 .../16 user-update.png                          |  Bin 74230 -> 0 bytes
 website/images/Kylin-ODBC-DSN/1.png             |  Bin 103154 -> 0 bytes
 website/images/Kylin-ODBC-DSN/2.png             |  Bin 147630 -> 0 bytes
 website/images/Kylin-ODBC-DSN/3.png             |  Bin 71034 -> 0 bytes
 website/images/Kylin-ODBC-DSN/4.png             |  Bin 113661 -> 0 bytes
 website/images/Kylin-Web-Tutorial/1 login.png   |  Bin 19030 -> 0 bytes
 .../Kylin-Web-Tutorial/10 query-result.png      |  Bin 229099 -> 0 bytes
 .../Kylin-Web-Tutorial/11 save-query-2.png      |  Bin 217907 -> 0 bytes
 .../images/Kylin-Web-Tutorial/11 save-query.png |  Bin 29373 -> 0 bytes
 .../images/Kylin-Web-Tutorial/12 general.png    |  Bin 32194 -> 0 bytes
 .../images/Kylin-Web-Tutorial/13 results.png    |  Bin 77124 -> 0 bytes
 website/images/Kylin-Web-Tutorial/14 drag.png   |  Bin 81057 -> 0 bytes
 .../images/Kylin-Web-Tutorial/15 bar-chart.png  |  Bin 79660 -> 0 bytes
 .../images/Kylin-Web-Tutorial/16 pie-chart.png  |  Bin 91917 -> 0 bytes
 .../images/Kylin-Web-Tutorial/17 line-chart.png |  Bin 83227 -> 0 bytes
 website/images/Kylin-Web-Tutorial/2 tables.png  |  Bin 131629 -> 0 bytes
 website/images/Kylin-Web-Tutorial/3 cubes.png   |  Bin 106631 -> 0 bytes
 .../images/Kylin-Web-Tutorial/4 form-view.png   |  Bin 106172 -> 0 bytes
 .../images/Kylin-Web-Tutorial/5 sql-view.png    |  Bin 111612 -> 0 bytes
 .../Kylin-Web-Tutorial/6 visualization.png      |  Bin 73256 -> 0 bytes
 website/images/Kylin-Web-Tutorial/7 access.png  |  Bin 63248 -> 0 bytes
 .../images/Kylin-Web-Tutorial/7 job-steps.png   |  Bin 157315 -> 0 bytes
 website/images/Kylin-Web-Tutorial/8 query.png   |  Bin 86323 -> 0 bytes
 .../images/Kylin-Web-Tutorial/9 query-table.png |  Bin 106686 -> 0 bytes
 .../Kylin-and-Tableau-Tutorial/1 odbc.png       |  Bin 77143 -> 0 bytes
 .../10 connectLive.jpg                          |  Bin 36887 -> 0 bytes
 .../Kylin-and-Tableau-Tutorial/11 analysis.jpg  |  Bin 109334 -> 0 bytes
 .../12 edit tables.jpg                          |  Bin 87229 -> 0 bytes
 .../Kylin-and-Tableau-Tutorial/13 visual.jpg    |  Bin 170415 -> 0 bytes
 .../Kylin-and-Tableau-Tutorial/14 publish.jpg   |  Bin 93989 -> 0 bytes
 .../16 prepare-publish.png                      |  Bin 19791 -> 0 bytes
 .../17 embedded-pwd.png                         |  Bin 12464 -> 0 bytes
 .../18 groupby-folder.jpg                       |  Bin 112470 -> 0 bytes
 .../Kylin-and-Tableau-Tutorial/19 custom.jpg    |  Bin 57976 -> 0 bytes
 .../Kylin-and-Tableau-Tutorial/2 serverhost.jpg |  Bin 50146 -> 0 bytes
 .../Kylin-and-Tableau-Tutorial/3 project.jpg    |  Bin 50594 -> 0 bytes
 .../Kylin-and-Tableau-Tutorial/4 done.jpg       |  Bin 48155 -> 0 bytes
 .../5 multipleTable.jpg                         |  Bin 65831 -> 0 bytes
 .../Kylin-and-Tableau-Tutorial/6 facttable.jpg  |  Bin 58203 -> 0 bytes
 .../Kylin-and-Tableau-Tutorial/6 facttable2.jpg |  Bin 58269 -> 0 bytes
 .../Kylin-and-Tableau-Tutorial/7 lkptable.jpg   |  Bin 61669 -> 0 bytes
 .../Kylin-and-Tableau-Tutorial/8 join.jpg       |  Bin 67952 -> 0 bytes
 .../Kylin-and-Tableau-Tutorial/9 connName.jpg   |  Bin 82397 -> 0 bytes
 website/images/blog/by-layer-cubing.png         |  Bin 130733 -> 0 bytes
 website/images/blog/by-segment-cubing.png       |  Bin 80771 -> 0 bytes
 website/images/blog/cube-spanning-tree.png      |  Bin 27581 -> 0 bytes
 website/images/blog/hybrid-model.png            |  Bin 118183 -> 0 bytes
 website/images/blog/meetup_1.jpeg               |  Bin 25392 -> 0 bytes
 website/images/blog/meetup_2.jpeg               |  Bin 34317 -> 0 bytes
 website/images/blog/meetup_3.jpeg               |  Bin 33195 -> 0 bytes
 website/images/blog/meetup_4.jpeg               |  Bin 21101 -> 0 bytes
 website/images/blog/meetup_5.jpeg               |  Bin 15216 -> 0 bytes
 website/images/blog/meetup_6.jpeg               |  Bin 29530 -> 0 bytes
 website/images/blog/meetup_7.jpeg               |  Bin 24020 -> 0 bytes
 website/images/blog/meetup_8.jpeg               |  Bin 16689 -> 0 bytes
 website/images/blog/meetup_9.jpeg               |  Bin 34820 -> 0 bytes
 .../images/develop/dictionary-bytes-format.png  |  Bin 29293 -> 0 bytes
 .../images/develop/dictionary-cache-layer.png   |  Bin 10883 -> 0 bytes
 .../images/develop/dictionary-trie-sample.png   |  Bin 19555 -> 0 bytes
 .../develop/disable_import_on_the_fly.png       |  Bin 128510 -> 0 bytes
 .../develop/eclipse_code_formatter_config.png   |  Bin 121789 -> 0 bytes
 .../images/docs/concepts/AggregationGroup.png   |  Bin 105363 -> 0 bytes
 website/images/docs/concepts/CubeAction.png     |  Bin 110592 -> 0 bytes
 website/images/docs/concepts/CubeDesc.png       |  Bin 190025 -> 0 bytes
 website/images/docs/concepts/CubeInstance.png   |  Bin 285222 -> 0 bytes
 website/images/docs/concepts/CubeSegment.png    |  Bin 96393 -> 0 bytes
 website/images/docs/concepts/DataModel.png      |  Bin 193661 -> 0 bytes
 website/images/docs/concepts/DataSource.png     |  Bin 180295 -> 0 bytes
 website/images/docs/concepts/Dimension.png      |  Bin 190495 -> 0 bytes
 website/images/docs/concepts/Job.png            |  Bin 299095 -> 0 bytes
 website/images/docs/concepts/JobAction.png      |  Bin 53369 -> 0 bytes
 website/images/docs/concepts/Measure.png        |  Bin 160824 -> 0 bytes
 website/images/docs/concepts/Partition.png      |  Bin 148494 -> 0 bytes
 website/images/install/Picture1.png             |  Bin 53722 -> 0 bytes
 website/images/install/Picture2.png             |  Bin 72567 -> 0 bytes
 website/images/install/cube.png                 |  Bin 151306 -> 0 bytes
 website/images/install/installation.pptx        |  Bin 83098 -> 0 bytes
 website/images/install/job.png                  |  Bin 226745 -> 0 bytes
 website/images/install/kylin_server_modes.png   |  Bin 97534 -> 0 bytes
 .../images/install/off_cli_install_scene.png    |  Bin 72567 -> 0 bytes
 website/images/install/on_cli_install_scene.png |  Bin 53722 -> 0 bytes
 website/images/install/query.png                |  Bin 112305 -> 0 bytes
 website/images/install/starthbase.png           |  Bin 210066 -> 0 bytes
 .../images/tutorial/odbc/ms_tool/Picture1.png   |  Bin 193409 -> 0 bytes
 .../images/tutorial/odbc/ms_tool/Picture2.png   |  Bin 62979 -> 0 bytes
 .../images/tutorial/odbc/ms_tool/Picture3.png   |  Bin 178436 -> 0 bytes
 .../images/tutorial/odbc/ms_tool/Picture4.png   |  Bin 296377 -> 0 bytes
 .../images/tutorial/odbc/ms_tool/Picture5.png   |  Bin 107448 -> 0 bytes
 .../images/tutorial/odbc/ms_tool/Picture6.png   |  Bin 61709 -> 0 bytes
 .../images/tutorial/odbc/ms_tool/Picture7.png   |  Bin 183660 -> 0 bytes
 .../images/tutorial/odbc/ms_tool/Picture8.png   |  Bin 158242 -> 0 bytes
 website/images/tutorial/odbc/tableau_91/1.png   |  Bin 117626 -> 0 bytes
 website/images/tutorial/odbc/tableau_91/2.png   |  Bin 32326 -> 0 bytes
 website/images/tutorial/odbc/tableau_91/3.png   |  Bin 93340 -> 0 bytes
 website/images/tutorial/odbc/tableau_91/4.png   |  Bin 85446 -> 0 bytes
 website/images/tutorial/odbc/tableau_91/5.png   |  Bin 85900 -> 0 bytes
 website/images/tutorial/odbc/tableau_91/6.png   |  Bin 107324 -> 0 bytes
 website/images/tutorial/odbc/tableau_91/7.png   |  Bin 112122 -> 0 bytes
 website/index.cn.md                             |   90 -
 website/index.md                                |   90 -
 301 files changed, 23474 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/.gitignore
----------------------------------------------------------------------
diff --git a/website/.gitignore b/website/.gitignore
deleted file mode 100644
index ab40dbc..0000000
--- a/website/.gitignore
+++ /dev/null
@@ -1,4 +0,0 @@
-.DS_Store
-.sass-cache
-_site
-

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/BingSiteAuth.xml
----------------------------------------------------------------------
diff --git a/website/BingSiteAuth.xml b/website/BingSiteAuth.xml
deleted file mode 100644
index 3997596..0000000
--- a/website/BingSiteAuth.xml
+++ /dev/null
@@ -1,18 +0,0 @@
-<?xml version="1.0"?>
-<!--
-  Licensed 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. See accompanying LICENSE file.
--->
-
-<users>
-	<user>403B76701CE9848BFBBE0912251DDAA9</user>
-</users>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/CNAME
----------------------------------------------------------------------
diff --git a/website/CNAME b/website/CNAME
deleted file mode 100644
index 4470516..0000000
--- a/website/CNAME
+++ /dev/null
@@ -1 +0,0 @@
-www.kylin.io

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/LICENSE
----------------------------------------------------------------------
diff --git a/website/LICENSE b/website/LICENSE
deleted file mode 100644
index e06d208..0000000
--- a/website/LICENSE
+++ /dev/null
@@ -1,202 +0,0 @@
-Apache License
-                           Version 2.0, January 2004
-                        http://www.apache.org/licenses/
-
-   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
-
-   1. Definitions.
-
-      "License" shall mean the terms and conditions for use, reproduction,
-      and distribution as defined by Sections 1 through 9 of this document.
-
-      "Licensor" shall mean the copyright owner or entity authorized by
-      the copyright owner that is granting the License.
-
-      "Legal Entity" shall mean the union of the acting entity and all
-      other entities that control, are controlled by, or are under common
-      control with that entity. For the purposes of this definition,
-      "control" means (i) the power, direct or indirect, to cause the
-      direction or management of such entity, whether by contract or
-      otherwise, or (ii) ownership of fifty percent (50%) or more of the
-      outstanding shares, or (iii) beneficial ownership of such entity.
-
-      "You" (or "Your") shall mean an individual or Legal Entity
-      exercising permissions granted by this License.
-
-      "Source" form shall mean the preferred form for making modifications,
-      including but not limited to software source code, documentation
-      source, and configuration files.
-
-      "Object" form shall mean any form resulting from mechanical
-      transformation or translation of a Source form, including but
-      not limited to compiled object code, generated documentation,
-      and conversions to other media types.
-
-      "Work" shall mean the work of authorship, whether in Source or
-      Object form, made available under the License, as indicated by a
-      copyright notice that is included in or attached to the work
-      (an example is provided in the Appendix below).
-
-      "Derivative Works" shall mean any work, whether in Source or Object
-      form, that is based on (or derived from) the Work and for which the
-      editorial revisions, annotations, elaborations, or other modifications
-      represent, as a whole, an original work of authorship. For the purposes
-      of this License, Derivative Works shall not include works that remain
-      separable from, or merely link (or bind by name) to the interfaces of,
-      the Work and Derivative Works thereof.
-
-      "Contribution" shall mean any work of authorship, including
-      the original version of the Work and any modifications or additions
-      to that Work or Derivative Works thereof, that is intentionally
-      submitted to Licensor for inclusion in the Work by the copyright owner
-      or by an individual or Legal Entity authorized to submit on behalf of
-      the copyright owner. For the purposes of this definition, "submitted"
-      means any form of electronic, verbal, or written communication sent
-      to the Licensor or its representatives, including but not limited to
-      communication on electronic mailing lists, source code control systems,
-      and issue tracking systems that are managed by, or on behalf of, the
-      Licensor for the purpose of discussing and improving the Work, but
-      excluding communication that is conspicuously marked or otherwise
-      designated in writing by the copyright owner as "Not a Contribution."
-
-      "Contributor" shall mean Licensor and any individual or Legal Entity
-      on behalf of whom a Contribution has been received by Licensor and
-      subsequently incorporated within the Work.
-
-   2. Grant of Copyright License. Subject to the terms and conditions of
-      this License, each Contributor hereby grants to You a perpetual,
-      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
-      copyright license to reproduce, prepare Derivative Works of,
-      publicly display, publicly perform, sublicense, and distribute the
-      Work and such Derivative Works in Source or Object form.
-
-   3. Grant of Patent License. Subject to the terms and conditions of
-      this License, each Contributor hereby grants to You a perpetual,
-      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
-      (except as stated in this section) patent license to make, have made,
-      use, offer to sell, sell, import, and otherwise transfer the Work,
-      where such license applies only to those patent claims licensable
-      by such Contributor that are necessarily infringed by their
-      Contribution(s) alone or by combination of their Contribution(s)
-      with the Work to which such Contribution(s) was submitted. If You
-      institute patent litigation against any entity (including a
-      cross-claim or counterclaim in a lawsuit) alleging that the Work
-      or a Contribution incorporated within the Work constitutes direct
-      or contributory patent infringement, then any patent licenses
-      granted to You under this License for that Work shall terminate
-      as of the date such litigation is filed.
-
-   4. Redistribution. You may reproduce and distribute copies of the
-      Work or Derivative Works thereof in any medium, with or without
-      modifications, and in Source or Object form, provided that You
-      meet the following conditions:
-
-      (a) You must give any other recipients of the Work or
-          Derivative Works a copy of this License; and
-
-      (b) You must cause any modified files to carry prominent notices
-          stating that You changed the files; and
-
-      (c) You must retain, in the Source form of any Derivative Works
-          that You distribute, all copyright, patent, trademark, and
-          attribution notices from the Source form of the Work,
-          excluding those notices that do not pertain to any part of
-          the Derivative Works; and
-
-      (d) If the Work includes a "NOTICE" text file as part of its
-          distribution, then any Derivative Works that You distribute must
-          include a readable copy of the attribution notices contained
-          within such NOTICE file, excluding those notices that do not
-          pertain to any part of the Derivative Works, in at least one
-          of the following places: within a NOTICE text file distributed
-          as part of the Derivative Works; within the Source form or
-          documentation, if provided along with the Derivative Works; or,
-          within a display generated by the Derivative Works, if and
-          wherever such third-party notices normally appear. The contents
-          of the NOTICE file are for informational purposes only and
-          do not modify the License. You may add Your own attribution
-          notices within Derivative Works that You distribute, alongside
-          or as an addendum to the NOTICE text from the Work, provided
-          that such additional attribution notices cannot be construed
-          as modifying the License.
-
-      You may add Your own copyright statement to Your modifications and
-      may provide additional or different license terms and conditions
-      for use, reproduction, or distribution of Your modifications, or
-      for any such Derivative Works as a whole, provided Your use,
-      reproduction, and distribution of the Work otherwise complies with
-      the conditions stated in this License.
-
-   5. Submission of Contributions. Unless You explicitly state otherwise,
-      any Contribution intentionally submitted for inclusion in the Work
-      by You to the Licensor shall be under the terms and conditions of
-      this License, without any additional terms or conditions.
-      Notwithstanding the above, nothing herein shall supersede or modify
-      the terms of any separate license agreement you may have executed
-      with Licensor regarding such Contributions.
-
-   6. Trademarks. This License does not grant permission to use the trade
-      names, trademarks, service marks, or product names of the Licensor,
-      except as required for reasonable and customary use in describing the
-      origin of the Work and reproducing the content of the NOTICE file.
-
-   7. Disclaimer of Warranty. Unless required by applicable law or
-      agreed to in writing, Licensor provides the Work (and each
-      Contributor provides its Contributions) on an "AS IS" BASIS,
-      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
-      implied, including, without limitation, any warranties or conditions
-      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
-      PARTICULAR PURPOSE. You are solely responsible for determining the
-      appropriateness of using or redistributing the Work and assume any
-      risks associated with Your exercise of permissions under this License.
-
-   8. Limitation of Liability. In no event and under no legal theory,
-      whether in tort (including negligence), contract, or otherwise,
-      unless required by applicable law (such as deliberate and grossly
-      negligent acts) or agreed to in writing, shall any Contributor be
-      liable to You for damages, including any direct, indirect, special,
-      incidental, or consequential damages of any character arising as a
-      result of this License or out of the use or inability to use the
-      Work (including but not limited to damages for loss of goodwill,
-      work stoppage, computer failure or malfunction, or any and all
-      other commercial damages or losses), even if such Contributor
-      has been advised of the possibility of such damages.
-
-   9. Accepting Warranty or Additional Liability. While redistributing
-      the Work or Derivative Works thereof, You may choose to offer,
-      and charge a fee for, acceptance of support, warranty, indemnity,
-      or other liability obligations and/or rights consistent with this
-      License. However, in accepting such obligations, You may act only
-      on Your own behalf and on Your sole responsibility, not on behalf
-      of any other Contributor, and only if You agree to indemnify,
-      defend, and hold each Contributor harmless for any liability
-      incurred by, or claims asserted against, such Contributor by reason
-      of your accepting any such warranty or additional liability.
-
-   END OF TERMS AND CONDITIONS
-
-   APPENDIX: How to apply the Apache License to your work.
-
-      To apply the Apache License to your work, attach the following
-      boilerplate notice, with the fields enclosed by brackets "{}"
-      replaced with your own identifying information. (Don't include
-      the brackets!)  The text should be enclosed in the appropriate
-      comment syntax for the file format. We also recommend that a
-      file or class name and description of purpose be included on the
-      same "printed page" as the copyright notice for easier
-      identification within third-party archives.
-
-   Copyright {yyyy} {name of copyright owner}
-
-   Licensed 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.
-

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/README.md
----------------------------------------------------------------------
diff --git a/website/README.md b/website/README.md
deleted file mode 100644
index 654a7b0..0000000
--- a/website/README.md
+++ /dev/null
@@ -1,64 +0,0 @@
-<!--
-{% comment %}
-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.
-{% endcomment %}
--->
-
-# Apache Kylin website  
-This directory contains the source code for the Apache Kylin website:
-[http://kyin.apache.org](http://kylin.apache.org/).
-
-## Kylin website's structure:
-
-1. ___config.yml__: Jekyll settings
-2. ___layouts__: Page layout template
-3. ___includes__: Page template like header, footer...
-2. ___data__: Jekyll collections, docs.yml is for Docs menu generation
-3. ___docs__: Documentation folder
-4. ___posts__: Blog folder
-5. __download__: Download folder, including released source code package, binary package, ODBC Driver and development version.
-6. __cn__: Chinese version 
-
-## Multi-Language
-To draft Chinese version document or translate existing one, just add or copy that doc and name with .cn.md as sufffix. It will generate under /cn folder with same name as html file.  
-To add other language, please update _config.yml and follow the same pattern as Chinese version.
-
-## Setup
-
-1. `cd website`
-2. `svn co https://svn.apache.org/repos/asf/kylin/site _site`
-3. `sudo apt-get install rubygems ruby2.1-dev zlib1g-dev` (linux)
-4. `sudo gem install bundler github-pages jekyll`
-5. `bundle install`
-
-___site_ folder is working dir which will be removed anytime by maven or git, please make sure only check out from svn when you want to publish to website.
-
-## Running locally  
-Before opening a pull request or push to git repo, you can preview changes from your local box with following:
-
-1. `cd website`
-2. `jekyll s`
-3. Open [http://localhost:4000](http://localhost:4000) in your browser
-
-## Pushing to site (for committer only)  
-1. `cd website/_site`
-2. `svn status`
-3. You'll need to `svn add` any new files
-4. `svn commit -m 'UPDATE MESSAGE'`
-
-Within a few minutes, svnpubsub should kick in and you'll be able to
-see the results at
-[http://kylin.apache.org](http://kylin.apache.org/).
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_config.yml
----------------------------------------------------------------------
diff --git a/website/_config.yml b/website/_config.yml
deleted file mode 100644
index 7531ff1..0000000
--- a/website/_config.yml
+++ /dev/null
@@ -1,62 +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.
-#
-
-# Site settings
-title: Apache Kylin
-email: dev@kylin.apache.org
-description: Apache Kylin Home
-baseurl: "" # the subpath of your site, e.g. /blog/
-url: "http://kylin.apache.org" # the base hostname & protocol for your site
-twitter_username: ApacheKylin
-
-encoding: UTF-8
-timezone: America/Dawson 
-
-exclude: ["README.md", "Rakefile", "*.scss", "*.haml", "*.sh"]
-include: [_docs,_dev]
-
-# Build settings
-markdown: kramdown
-kramdown:
-  input: GFM
-  syntax_highlighter: rouge
-
-highlighter: rouge
-
-# Permalinks, default is pretty
-# permalink: /:categories/:title.html
-permalink: pretty
-
-google_analytics_id: UA-55534813-2
-
-# Multiple Languages Plugin
-gems:           ['jekyll-multiple-languages', 'kramdown', 'rouge']
-# Supported Languages
-languages: ['en','cn']
-language_default: 'en'
-
-# If a post of default language not set `no_fill_default_content` to true
-# Its content will use to replace if the corresponding content of other languages is not exist.
-# fill_default_content: true
-
-collections:
-  docs:
-    output: true
-  docs-cn:
-    output: true    
-  dev:
-    output: true  

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_config_prod.yml
----------------------------------------------------------------------
diff --git a/website/_config_prod.yml b/website/_config_prod.yml
deleted file mode 100644
index c71bbad..0000000
--- a/website/_config_prod.yml
+++ /dev/null
@@ -1,24 +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.
-#
-
-# Site settings
-
-# Notes: 
-# To release to website, use following script to generate real docs:
-# >jekyll build -config _config.yml _config_prod.yml
-# 
-url: "http://kylin.apache.org" # the base hostname & protocol for your site

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_data/contributors.yml
----------------------------------------------------------------------
diff --git a/website/_data/contributors.yml b/website/_data/contributors.yml
deleted file mode 100644
index d32b93f..0000000
--- a/website/_data/contributors.yml
+++ /dev/null
@@ -1,118 +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.
-
-# Database of contributors to Apache Kylin.
-- name: George Song (宋轶)
-  apacheId: yisong
-  githubId: songyi10011001
-  org: eBay
-  role: PMC
-- name: Henry Saputra
-  apacheId: hsaputra
-  githubId: hsaputra
-  org: 
-  role: Mentor
-- name: Hongbin Ma (马洪宾)
-  apacheId: mahongbin
-  githubId: binmahone
-  org: eBay
-  role: PMC
-- name: Jason Zhong (仲俭)
-  apacheId: zhongjian
-  githubId: janzhongi
-  org: eBay
-  role: PMC
-- name: Julian Hyde
-  apacheId: jhyde
-  githubId: julianhyde
-  org: Hortonworks
-  role: Mentor
-- name: Luke Han (韩卿) (Chair)
-  apacheId: lukehan
-  githubId: lukehan
-  org: eBay
-  role: PMC
-- name: Owen O'Malley
-  apacheId: omalley
-  githubId: omalley
-  org: Hortonworks
-  role: Mentor
-- name: P. Taylor Goetz
-  apacheId: ptgoetz
-  githubId: ptgoetz
-  org: 
-  role: Mentor
-- name: Qianhao Zhou (周千昊)
-  apacheId: qhzhou
-  githubId: qhzhou
-  org: eBay
-  role: PMC
-- name: Ted Dunning
-  apacheId: tdunning
-  githubId: tdunning
-  org: MapR
-  role: Champion
-  avatar: https://www.mapr.com/sites/default/files/otherpageimages/ted-circle-80.png
-- name: Shaofeng Shi (史少锋) 
-  apacheId: shaofengshi
-  githubId: shaofengshi
-  org: eBay
-  role: PMC
-- name: Xiaodong Duo (朵晓东)
-  apacheId: xduo
-  githubId: xduo
-  org: Alipay
-  role: Emeritus PMC
-- name: Ankur Bansal
-  apacheId: abansal
-  githubId: abansal
-  org: eBay
-  role: Emeritus PMC
-- name: Xu Jiang (蒋旭)
-  apacheId: jiangxu
-  githubId: jiangxuchina
-  org: Alibaba
-  role: PMC
-- name: Yang Li (李扬)
-  apacheId: liyang
-  githubId: liyang-gmt8
-  org: eBay
-  role: PMC
-- name: Dayue Gao (高大月)
-  apacheId: gaodayue
-  githubId: gaodayue
-  org: Meituan
-  role: PMC
-- name: Hua Huang (黄桦)
-  apacheId: hhuang
-  githubId: superhua
-  org: MiningLAMP
-  role: PMC
-- name: Dong Li (李栋)
-  apacheId: lidong
-  githubId: lidongsjtu
-  org: eBay
-  role: committer
-- name: Xiaoyu Wang (王晓雨)
-  apacheId: wangxiaoyu
-  githubId: xiaowangyu
-  org: JD.com
-  role: committer
-- name: Yerui Sun (孙叶锐)
-  apacheId: sunyerui
-  githubId: sunyerui
-  org: Meituan
-  role: PMC
-

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_data/development.yml
----------------------------------------------------------------------
diff --git a/website/_data/development.yml b/website/_data/development.yml
deleted file mode 100644
index 8b03f76..0000000
--- a/website/_data/development.yml
+++ /dev/null
@@ -1,30 +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.
-
-# Development menu items, for English one, dev-cn.yml is for Chinese one
-# The development menu is constructed in dev_nav.html with these data
-
-- title: Development Guide
-  dev:
-  - index
-  - dev_env
-  - howto_test
-  - howto_contribute
-  - howto_docs
-  - howto_package
-  - howto_release
-  - new_metadata
-  - web_tech
-  - about_temp_files

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_data/docs-cn.yml
----------------------------------------------------------------------
diff --git a/website/_data/docs-cn.yml b/website/_data/docs-cn.yml
deleted file mode 100644
index f69fbe5..0000000
--- a/website/_data/docs-cn.yml
+++ /dev/null
@@ -1,20 +0,0 @@
-- title: 开始
-  docs:
-  - index
-
-- title: 安装
-  docs:
-  - install/install_guide
-  - install/manual_install_guide
-
-- title: 教程
-  docs:
-  - tutorial/create_cube_cn
-  - tutorial/cube_build_job
-  - tutorial/acl
-  - tutorial/web
-  - tutorial/tableau
-  - tutorial/tableau_91
-  - tutorial/powerbi
-  - tutorial/odbc
-

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_data/docs.yml
----------------------------------------------------------------------
diff --git a/website/_data/docs.yml b/website/_data/docs.yml
deleted file mode 100644
index c557ae7..0000000
--- a/website/_data/docs.yml
+++ /dev/null
@@ -1,58 +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.
-
-# Docs menu items, for English one, docs-cn.yml is for Chinese one
-# The docs menu is constructed in docs_nav.html with these data
-- title: Getting Started
-  docs:
-  - index
-  - release_notes
-  - gettingstarted/faq
-  - gettingstarted/events
-  - gettingstarted/terminology
-  - gettingstarted/concepts
-
-- title: Installation
-  docs:
-  - install/index
-  - install/hadoop_env
-  - install/manual_install_guide
-  - install/kylin_cluster
-  - install/advance_settings
-  - install/kylin_docker
-
-- title: Tutorial
-  docs:
-  - tutorial/kylin_sample
-  - tutorial/create_cube
-  - tutorial/cube_build_job
-  - tutorial/acl
-  - tutorial/web
-  - tutorial/tableau
-  - tutorial/tableau_91
-  - tutorial/powerbi
-  - tutorial/odbc
-
-- title: How To
-  docs:
-  - howto/howto_build_cube_with_restapi
-  - howto/howto_use_restapi_in_js
-  - howto/howto_use_restapi
-  - howto/howto_optimize_cubes
-  - howto/howto_backup_metadata
-  - howto/howto_cleanup_storage
-  - howto/howto_jdbc
-  - howto/howto_upgrade
-  - howto/howto_ldap_and_sso

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_dev/about_temp_files.md
----------------------------------------------------------------------
diff --git a/website/_dev/about_temp_files.md b/website/_dev/about_temp_files.md
deleted file mode 100644
index b46a764..0000000
--- a/website/_dev/about_temp_files.md
+++ /dev/null
@@ -1,29 +0,0 @@
----
-layout: dev
-title:  "About Temp Files"
-categories: development
-permalink: /development/about_temp_files.html
----
-
-As we reviewed the code we found that Kylin left lots of garbage files in:
-
-* Local file system of the CLI
-* HDFS
-* Local file system of the hadoop nodes.
-
-A ticket was opened to track this issue:
-[https://issues.apache.org/jira/browse/KYLIN-926](https://issues.apache.org/jira/browse/KYLIN-926)
-
-For future developments, please:
-
-* Whenever you want to create temp files at Local, choose
-File.createTempFile or use the folder:
-BatchConstants.CFG_KYLIN_LOCAL_TEMP_DIR(/tmp/kylin), do not randomly use
-another folder in /tmp, it will end up a mess, and look unprofessional.
-* Whenever you create temp files at Local, remember to delete it after
-using it. It's best to use FileUtils.forceDelete, as it also works for
-deleting folders. Try avoid deleteOnExit, in case Kylin exits abnormally.
-* Whenever you want to create files in HDFS, try to create it under
-kylin.hdfs.working.dir or BatchConstants.CFG_KYLIN_HDFS_TEMP_DIR, and
-remember to delete it after it is no longer useful. Try avoid throwing
-everything into hdfs:///tmp and leave it as garbage.

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_dev/dev_env.md
----------------------------------------------------------------------
diff --git a/website/_dev/dev_env.md b/website/_dev/dev_env.md
deleted file mode 100644
index 4a9aa9e..0000000
--- a/website/_dev/dev_env.md
+++ /dev/null
@@ -1,129 +0,0 @@
----
-layout: dev
-title:  Setup Development Env
-categories: development
-permalink: /development/dev_env.html
----
-
-Developers want to run kylin test cases or applications at their development machine. 
-
-By following this tutorial, you will be able to build kylin test cubes by running a specific test case, and you can further run other test cases against the cubes having been built.
-
-## Environment on the Hadoop CLI
-
-Off-Hadoop-CLI installation requires you having a hadoop CLI machine (or a hadoop sandbox) as well as your local develop machine. To make things easier we strongly recommend you starting with running Kylin on a hadoop sandbox, like <http://hortonworks.com/products/hortonworks-sandbox/>. In the following tutorial we'll go with **Hortonworks Sandbox 2.2.4**. It is recommended that you provide enough memory to your sandbox, 8G or more is preferred.
-
-### Start Hadoop
-
-In Hortonworks sandbox, ambari helps to launch hadoop:
-
-{% highlight Groff markup %}
-ambari-agent start
-ambari-server start
-{% endhighlight %}
-	
-With both command successfully run you can go to ambari home page at <http://yoursandboxip:8080> (user:admin,password:admin) to check everything's status. By default ambari disables Hbase, you'll need manually start the `Hbase` service.
-![start hbase in ambari](https://raw.githubusercontent.com/KylinOLAP/kylinolap.github.io/master/docs/installation/starthbase.png)
-
-For other hadoop distribution, basically start the hadoop cluster, make sure HDFS, YARN, Hive, HBase are running.
-
-
-## Environment on the dev machine
-
-
-### Install maven
-
-The latest maven can be found at <http://maven.apache.org/download.cgi>, we create a symbolic so that `mvn` can be run anywhere.
-
-{% highlight Groff markup %}
-cd ~
-wget http://xenia.sote.hu/ftp/mirrors/www.apache.org/maven/maven-3/3.2.5/binaries/apache-maven-3.2.5-bin.tar.gz
-tar -xzvf apache-maven-3.2.5-bin.tar.gz
-ln -s /root/apache-maven-3.2.5/bin/mvn /usr/bin/mvn
-{% endhighlight %}
-
-### Compile
-
-First clone the Kylin project to your local:
-
-{% highlight Groff markup %}
-git clone https://github.com/apache/kylin.git
-{% endhighlight %}
-	
-Install Kylin artifacts to the maven repo
-
-{% highlight Groff markup %}
-mvn clean install -DskipTests
-{% endhighlight %}
-
-### Modify local configuration
-
-Local configuration must be modified to point to your hadoop sandbox (or CLI) machine. 
-
-* In **examples/test_case_data/sandbox/kylin.properties**
-   * Find `sandbox` and replace with your hadoop hosts (if you're using HDP sandbox, this can be skipped)
-   * Find `kylin.job.run.as.remote.cmd` and change it to "true" (in code repository the default is false, which assume running it on hadoop CLI)
-   * Find `kylin.job.remote.cli.username` and `kylin.job.remote.cli.password`, fill in the user name and password used to login hadoop cluster for hadoop command execution; If you're using HDP sandbox, the default username is "root" and password is "hadoop".
-
-* In **examples/test_case_data/sandbox**
-   * For each configuration xml file, find all occurrence of `sandbox` and replace with your hadoop hosts; (if you're using HDP sandbox, this can be skipped)
-
-An alternative to the host replacement is updating your `hosts` file to resolve `sandbox` and `sandbox.hortonworks.com` to the IP of your sandbox machine.
-
-### Run unit tests
-
-Run a end-to-end cube building test, these special test cases will populate some sample cubes in your metadata store and build them ready.
-It might take a while (maybe one hour), please keep patient.
- 
-{% highlight Groff markup %}
-mvn test -Dtest=org.apache.kylin.job.BuildCubeWithEngineTest -DfailIfNoTests=false -Dhdp.version=<hdp-version> -P sandbox
-	
-mvn test -Dtest=org.apache.kylin.job.BuildIIWithEngineTest -DfailIfNoTests=false -Dhdp.version=<hdp-version> -P sandbox
-{% endhighlight %}
-	
-Run other tests, the end-to-end cube building test is exclueded
-
-{% highlight Groff markup %}
-mvn test -fae -P sandbox
-{% endhighlight %}
-
-### Launch Kylin Web Server
-
-Copy server/src/main/webapp/WEB-INF to webapp/app/WEB-INF 
-
-{% highlight Groff markup %}
-cp -r server/src/main/webapp/WEB-INF webapp/app/WEB-INF 
-{% endhighlight %}
-
-Download JS for Kylin web GUI. `npm` is part of `Node.js`, please search about how to install it on your OS.
-
-{% highlight Groff markup %}
-cd webapp
-npm install -g bower
-bower --allow-root install
-{% endhighlight %}
-
-In IDE, launch `org.apache.kylin.rest.DebugTomcat` with working directory set to the /server folder. (By default Kylin server will listen on 7070 port; If you want to use another port, please specify it as a parameter when run `DebugTomcat)
-
-Check Kylin Web available at http://localhost:7070/kylin (user:ADMIN,password:KYLIN)
-
-For IntelliJ IDEA users, need modify "server/kylin-server.iml" file, replace all "PROVIDED" to "COMPILE", otherwise an "java.lang.NoClassDefFoundError: org/apache/catalina/LifecycleListener" error may be thrown;
-
-
-## Setup IDE code formatter
-
-In case you're writting code for Kylin, you should make sure that your code in expected formats.
-
-For Eclipse users, just format the code before committing the code.
-
-For intellij IDEA users, you have to do a few more steps:
-
-1. Install "Eclipse Code Formatter" and configure it as follows:
-
-	![Eclipse_Code_Formatter_Config](/images/develop/eclipse_code_formatter_config.png)
-
-2. Disable intellij IDEA's "Optimize imports on the fly"
-
-	![Disable_Optimize_On_The_Fly](/images/develop/disable_import_on_the_fly.png)
-
-3. Format the code before committing the code.

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_dev/howto_contribute.md
----------------------------------------------------------------------
diff --git a/website/_dev/howto_contribute.md b/website/_dev/howto_contribute.md
deleted file mode 100644
index c5d0e29..0000000
--- a/website/_dev/howto_contribute.md
+++ /dev/null
@@ -1,28 +0,0 @@
----
-layout: dev
-title:  How to Contribute
-categories: development
-permalink: /development/howto_contribute.html
----
-
-## Current branches
-* `1.x-staging`: Dev branch for 1,x versions, this branch spawns releases like 1.0, 1.1 etc. This is where new patches go to.
-* `2.x-staging`: Dev branch for 2.x versions, 2.x is the next generation Kylin (with streaming, spark support), it has fundamental difference with 1.x version, which means any changes on 1.x cannot merge to 2.x anymore. So if your patch affects both branches, you should make patches for both branches.
-* `master`: always point to the latest stable release (stable, but not up to date)
-
-## Making Changes
-* [Setup dev env](/development/dev_env.html)
-* Raise a JIRA, describe the feature/enhancement/bug
-* Discuss with others in mailing list or issue comments, make sure the proposed changes fit in with what others are doing and have planned for the project
-* Make changes in your fork
-	* No strict code style at the moment, but the general rule is keep consistent with existing files. E.g. use 4-space indent for java files.
-	* Sufficient unit test and integration test is a mandatory part of code change.
-* [Run tests](/development/howto_test.html) to ensure your change is in good quality and does not break anything
-* Generate patch and attach it to relative JIRA. Please use `git format-patch` command to generate the patch, for a detail guide you can refer to [How to create and apply a patch with Git](https://ariejan.net/2009/10/26/how-to-create-and-apply-a-patch-with-git/)
-
-
-## Apply Patch
-* Committer will review in terms of correctness, performance, design, coding style, test coverage
-* Discuss and revise if necessary
-* Finally committer merge code into target branch
-	* We use `git rebase` to ensure the merged result is a streamline of commits.

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_dev/howto_docs.md
----------------------------------------------------------------------
diff --git a/website/_dev/howto_docs.md b/website/_dev/howto_docs.md
deleted file mode 100644
index 4850750..0000000
--- a/website/_dev/howto_docs.md
+++ /dev/null
@@ -1,152 +0,0 @@
----
-layout: dev
-title:  How to Write Document
-categories: development
-permalink: /development/howto_docs.html
----
-
-## Before your work
-
-Install following tools before you add or edit documentation:  
-
-1. First, make sure Ruby and Gem work on your machine  
-	* For Mac user, please refer [this](https://github.com/sstephenson/rbenv#homebrew-on-mac-os-x) to setup ruby env.
-	* For Windows user, use the [ruby installer](http://rubyinstaller.org/downloads/).
-	* For China user, consider use a [local gem repository](https://ruby.taobao.org/) in case of network issues.
-
-2. Then, install [Jekyll](http://jekyllrb.com), and required plugins
-	* `gem install jekyll jekyll-multiple-languages kramdown rouge`  
-	* __Note__: Some specific version of jekyll and jekyll-multiple-languages does not work together (I got a "undefined method" error with jekyll 3.0.1 and jekyll-multiple-languages 2.0.3). In that case, `jekyll 2.5.3` and `jekyll-multiple-languages 1.0.8` is the known working version.
-        * eg. Use `gem install jekyll --version "=2.5.3"` to install a specific version.
-	
-3. And optionally any markdown editor you prefer
-
-## About Jekyll
-Jekyll is a Ruby script to generate a static HTML website from source text and themes, the HTML is generated before being deployed to the web server. Jekyll also happens to be the engine behind GitHub Pages.
-
-Here are good reference about basic usage of Jekyll: [Learning Jekyll By Example](http://learn.andrewmunsell.com/learn/jekyll-by-example/tutorial)
-
-Apache Kylin's website and documentation is using Jekyll to manage and generate final content which avaliable at [http://kylin.apache.org](http://kylin.apache.org).
-
-## Multi-Language
-To draft Chinese version document or translate existing one, just add or copy that doc and name with .cn.md as sufffix. It will generate under /cn folder with same name as html file.  
-To add other language, please update _config.yml and follow the same pattern as Chinese version.
-
-# Kylin document sturcture and navigation menu
-
-1. __Home Page__: _"index.md"_ Home page of Docs
-2. __Getting Started__: _"gettingstarted"_ General docs about Apache Kylin, including FAQ, Terminology
-3. __Installation__: _"install"_ Apache Kylin installation guide
-4. __Tutorial__: _"tutorial"_ User tutorial about how to use Apache Kylin
-5. __How To__: _"howto"_ Guide for more detail help
-6. __Development__: _"development"_ For developer to contribute, integration with other application and extend Apache Kylin
-7. __Others__: Other docs.
-
-The menu is managed by Jekyll collection:
-
-* ___data/docs.yml__: English version menu structure  
-* ___data/docs-cn.yml__: Chinese version menu structure   
-* __add new menu item__: To add new item: create new docs under relative folder, e.g howto_example.md. add following Front Mark:  
-
-```
----
-layout: docs
-title:  How to expamle
-categories: howto
-permalink: /docs/howto/howto_example.html
-version: v0.7.2
-since: v0.7.2
----
-```
-
-change the __permalink__ to exactly link   
-Then add item to docs.yml like:
-
-```
-- title: How To
-  docs:
-  - howto/howto_contribute
-  - howto/howto_jdbc
-  - howto/howto_example
-```
-
-# How to edit document
-Open doc with any markdown editor, draft content and preview in local.
-
-Sample Doc:
-
-```
----
-layout: docs
-title:  How to example
-categories: howto
-permalink: /docs/howto/howto_example.html
-version: v0.7.2
-since: v0.7.2
----
-
-## This is example doc
-The quick brown fox jump over the lazy dog.
-
-```
-
-# How to add image
-All impage please put under _images_ folder, in your document, please using below sample to include image:  
-
-```
-![](/images/Kylin-Web-Tutorial/2 tables.png)
-
-```
-
-# How to add link
-Using relative path for site links, for example:
-
-```
-[REST API](docs/development/rest_api.html). 
-
-```
-
-# How to add code highlight
-We are using [Rouge](https://github.com/jneen/rouge) to highlight code syntax.
-check this doc's source code for more detail sample.
-
-# How to preview in your local
-You can preview in your markdown editor, to check exactly what it will looks like on website, please run Jekyll from `website` folder:
-
-```
-jekyll server
-
-```
-Then access http://127.0.0.1:4000 in your browser.
-
-## How to publish to website (for committer only)  
-
-### Setup
-
-1. `cd website`
-2. `svn co https://svn.apache.org/repos/asf/kylin/site _site`
-3. `sudo apt-get install rubygems ruby2.1-dev zlib1g-dev` (linux)
-4. `sudo gem install bundler github-pages jekyll`
-5. `bundle install`
-
-___site__ folder is working dir which will be removed anytime by maven or git, please make sure only check out from svn when you want to publish to website.
-
-### Running locally  
-Before opening a pull request or push to git repo, you can preview changes from your local box with following:
-
-1. `cd website`
-2. `jekyll s`
-3. Open [http://127.0.0.1:4000](http://127.0.0.1:4000) in your browser
-
-### Pushing to site 
-
-1. `cd website/_site`
-2. `svn status`
-3. You'll need to `svn add` any new files
-4. `svn commit -m 'UPDATE MESSAGE'`
-
-Within a few minutes, svnpubsub should kick in and you'll be able to
-see the results at
-[http://kylin.apache.org](http://kylin.apache.org/).
-
-

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_dev/howto_package.md
----------------------------------------------------------------------
diff --git a/website/_dev/howto_package.md b/website/_dev/howto_package.md
deleted file mode 100644
index a9dec33..0000000
--- a/website/_dev/howto_package.md
+++ /dev/null
@@ -1,22 +0,0 @@
----
-layout: dev
-title:  How to Build Binary Package
-categories: development
-permalink: /development/howto_package.html
----
-
-### Generate Binary Package
-{% highlight bash %}
-git clone https://github.com/apache/kylin kylin
-cd kylin
-./script/package.sh
-{% endhighlight %}
-
-In order to generate binary package, **maven** and **npm** are pre-requisites.
-
-If you're behind a proxy server, both npm and bower need be told with the proxy info before running ./script/package.sh:
-
-{% highlight bash %}
-export http_proxy=http://your-proxy-host:port
-npm config set proxy http://your-proxy-host:port
-{% endhighlight %}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_dev/howto_release.md
----------------------------------------------------------------------
diff --git a/website/_dev/howto_release.md b/website/_dev/howto_release.md
deleted file mode 100644
index cad3901..0000000
--- a/website/_dev/howto_release.md
+++ /dev/null
@@ -1,430 +0,0 @@
----
-layout: dev
-title:  How to Making a Release
-categories: development
-permalink: /development/howto_release.html
----
-
-_This guide is for Apache Kylin Committers only._  
-_Shell commands is on Mac OS X as sample._  
-_For people in China, please aware using proxy to avoid potential firewall issue._  
-
-## Setup Account
-Make sure you have avaliable account and privilege for following applications:
-
-* Apache account: [https://id.apache.org](https://id.apache.org/)    
-* Apache Kylin git repo (main cobe base): [https://git-wip-us.apache.org/repos/asf/kylin.git](https://git-wip-us.apache.org/repos/asf/kylin.git)  
-* Apache Kylin svn repo (for website only): [https://svn.apache.org/repos/asf/kylin](https://svn.apache.org/repos/asf/kylin)  
-* Apache Nexus (maven repo): [https://repository.apache.org](https://repository.apache.org)  
-* Apache Kylin dist repo: [https://dist.apache.org/repos/dist/dev/kylin](https://dist.apache.org/repos/dist/dev/kylin)  
-
-## Setup PGP signing keys  
-Follow instructions at [http://www.apache.org/dev/release-signing](http://www.apache.org/dev/release-signing) to create a key pair  
-Install gpg (On Mac OS X as sample):  
-`brew install gpg and gpg --gen-key`
-
-Generate gpg key:  
-Reference: [https://www.gnupg.org/gph/en/manual/c14.html](https://www.gnupg.org/gph/en/manual/c14.html)  
-_All new RSA keys generated should be at least 4096 bits. Do not generate new DSA keys_  
-`gpg --gen-key`  
-
-Verify your key:  
-`gpg --list-sigs YOUR_NAME`
-
-Get the fingerprint of your key:
-`gpg --fingerprint YOUR_NAME`
-
-It will display the fingerprint like "Key fingerprint = XXXX XXXX ...", then add the fingerprint to your apache account at [https://id.apache.org/](https://id.apache.org/) in "OpenPGP Public Key Primary Fingerprint" field; wait for a few hours the key will added to [https://people.apache.org/keys/](https://people.apache.org/keys/), for example:
-[https://people.apache.org/keys/committer/lukehan.asc](https://people.apache.org/keys/committer/lukehan.asc)
-
-Generate ASCII Amromed Key:  
-`gpg -a --export YOUR_MAIL_ADDRESS > YOUR_NAME.asc &`
-
-Upload key to public server:  
-`gpg --send-keys YOUR_KEY_HASH`
-
-or Submit key via web:  
-Open and Submit to [http://pool.sks-keyservers.net:11371](http://pool.sks-keyservers.net:11371) (you can pickup any avaliable public key server)
-
-Once your key submitted to server, you can verify using following command:  
-`gpg --recv-keys YOUR_KEY_HASH`
-for example:  
-`gpg --recv-keys 027DC364`
-
-Add your public key to the KEYS file by following instructions in the KEYS file.:  
-_KEYS file location:_ __${kylin}/KEYS__  
-For example:  
-`(gpg --list-sigs YOURNAME && gpg --armor --export YOURNAME) >> KEYS`
-
-Commit your changes.
-
-## Prepare artifacts for release  
-__Before you start:__
-
-* Set up signing keys as described above.
-* Make sure you are using JDK 1.7 (not 1.8).
-* Make sure you are working on right release version number.
-* Make sure that every “resolved” JIRA case (including duplicates) has a fix version assigned.
-
-__Configure Apache repository server in Maven__
-If you're the first time to do release, you need update the server authentication information in ~/.m2/settings.xml; If this file doesn't exist, copy a template from $M2_HOME/conf/settings.xml;
-
-In the "servers" section, make sure the following servers be added, and replace #YOUR_APACHE_ID#, #YOUR_APACHE_PWD#, #YOUR_GPG_PASSPHRASE# with your ID, password, and passphrase:
-{% highlight bash %}
-<servers>
-    <!-- To publish a snapshot of some part of Maven -->
-    <server>
-      <id>apache.snapshots.https</id>
-      <username>#YOUR_APACHE_ID#</username>
-      <password>#YOUR_APACHE_PWD#</password>
-    </server>
-    <!-- To stage a release of some part of Maven -->
-    <server>
-      <id>apache.releases.https</id>
-      <username>#YOUR_APACHE_ID#</username>
-      <password>#YOUR_APACHE_PWD#</password>
-    </server>
-
-    <!-- To publish a website of some part of Maven -->
-    <server>
-      <id>apache.website</id>
-      <username>#YOUR_APACHE_ID#</username>
-      <password>#YOUR_APACHE_PWD#</password>
-      <!-- Either
-      <privateKey>...</privateKey>
-      --> 
-      <filePermissions>664</filePermissions>
-      <directoryPermissions>775</directoryPermissions>
-    </server>
-
-    <!-- To stage a website of some part of Maven -->
-    <server>
-      <id>stagingSite</id> 
-      <!-- must match hard-coded repository identifier in site:stage-deploy -->
-      <username>#YOUR_APACHE_ID#</username>
-      <filePermissions>664</filePermissions>
-      <directoryPermissions>775</directoryPermissions>
-    </server>
-    <server>
-      <id>gpg.passphrase</id>
-      <passphrase>#YOUR_GPG_PASSPHRASE#</passphrase>
-    </server>
-  </servers>
-{% endhighlight %}
-
-__Fix license issues and make a snapshot__
-{% highlight bash %}
-# Set passphrase variable without putting it into shell history
-$ read -s GPG_PASSPHRASE
-
-# Make sure that there are no junk files in the sandbox
-$ git clean -xn
-$ mvn clean
-
-# Fix any license issues as prompted
-$ mvn -Papache-release -Dgpg.passphrase=${GPG_PASSPHRASE} install
-{% endhighlight %}
-
-Optionally, when the dry-run has succeeded, change install to deploy:
-{% highlight bash %}
-$ mvn -Papache-release -Dgpg.passphrase=${GPG_PASSPHRASE} deploy
-{% endhighlight %}
-
-__Making a release__
-
-Create a release branch named after the release, e.g. v0.7.2-release, and push it to Apache.  
-{% highlight bash %}
-$ git checkout -b vX.Y.Z-release
-$ git push -u origin vX.Y.Z-release
-{% endhighlight %}
-We will use the branch for the entire the release process. Meanwhile, we do not allow commits to the master branch. After the release is final, we can use `git merge --ff-only` to append the changes on the release branch onto the master branch. (Apache does not allow reverts to the master branch, which makes it difficult to clean up the kind of messy commits that inevitably happen while you are trying to finalize a release.)
-
-Now, set up your environment and do a dry run. The dry run will not commit any changes back to git and gives you the opportunity to verify that the release process will complete as expected.
-
-If any of the steps fail, clean up (see below), fix the problem, and start again from the top.  
-{% highlight bash %}
-# Set passphrase variable without putting it into shell history
-$ read -s GPG_PASSPHRASE
-
-# Make sure that there are no junk files in the sandbox
-$ git clean -xn
-$ mvn clean
-
-# Do a dry run of the release:prepare step, which sets version numbers.
-$ mvn -DdryRun=true -DskipTests -DreleaseVersion=X.Y.Z -DdevelopmentVersion=(X.Y.Z+1)-SNAPSHOT -Papache-release -Darguments="-Dgpg.passphrase=${GPG_PASSPHRASE}" release:prepare 2>&1 | tee /tmp/prepare-dry.log
-{% endhighlight %}
-
-__Check the artifacts:__
-
-* In the `target` directory should be these 8 files, among others:
-  * apache-kylin-X.Y.Z-SNAPSHOT-src.tar.gz
-  * apache-kylin-X.Y.Z-SNAPSHOT-src.tar.gz.asc
-  * apache-kylin-X.Y.Z-SNAPSHOT-src.tar.gz.md5
-  * apache-kylin-X.Y.Z-SNAPSHOT-src.tar.gz.sha1
-  * apache-kylin-X.Y.Z-SNAPSHOT-src.zip
-  * apache-kylin-X.Y.Z-SNAPSHOT-src.zip.asc
-  * apache-kylin-X.Y.Z-SNAPSHOT-src.zip.md5
-  * apache-kylin-X.Y.Z-SNAPSHOT-src.zip.sha1
-* Remove the .zip, .zip.asc, .zip.md5 and zip.sha1 file as they are not needed.
-* Note that the file names start `apache-kylin-`.
-* In the source distro `.tar.gz`, check that all files belong to a directory called
-  `apache-kylin-X.Y.Z-src`.
-* That directory must contain files `NOTICE`, `LICENSE`, `README.md`
-* Check PGP, per [this](https://httpd.apache.org/dev/verification.html)
-
-__Run real release:__
-Now, run the release for real.  
-{% highlight bash %}
-# Prepare sets the version numbers, creates a tag, and pushes it to git.
-$ mvn -DskipTests -DreleaseVersion=X.Y.Z -DdevelopmentVersion=(X.Y.Z+1)-SNAPSHOT -Papache-release -Darguments="-Dgpg.passphrase=${GPG_PASSPHRASE}" release:prepare
-
-# Perform checks out the tagged version, builds, and deploys to the staging repository
-$ mvn -DskipTests -Papache-release -Darguments="-Dgpg.passphrase=${GPG_PASSPHRASE} -DskipTests" release:perform
-{% endhighlight %}
-
-__Close the staged artifacts in the Nexus repository:__
-
-* Go to [https://repository.apache.org/](https://repository.apache.org/) and login
-* Under `Build Promotion`, click `Staging Repositories`
-* In the `Staging Repositories` tab there should be a line with profile `org.apache.kylin`
-* Navigate through the artifact tree and make sure the .jar, .pom, .asc files are present
-* Check the box on in the first column of the row, and press the 'Close' button to publish the repository at
-  [https://repository.apache.org/content/repositories/orgapachekylin-1006](https://repository.apache.org/content/repositories/orgapachekylin-1006)
-  (or a similar URL)
-
-__Upload to staging area:__  
-Upload the artifacts via subversion to a staging area, https://dist.apache.org/repos/dist/dev/kylin/apache-kylin-X.Y.Z-rcN:
-{% highlight bash %}
-# Create a subversion workspace, if you haven't already
-$ mkdir -p ~/dist/dev
-$ pushd ~/dist/dev
-$ svn co https://dist.apache.org/repos/dist/dev/kylin
-$ popd
-
-## Move the files into a directory
-$ cd target
-$ mkdir ~/dist/dev/kylin/apache-kylin-X.Y.Z-rcN
-$ mv apache-kylin-* ~/dist/dev/kylin/apache-kylin-X.Y.Z-rcN
-
-## Check in
-$ cd ~/dist/dev/kylin
-$ svn add apache-kylin-X.Y.Z-rcN
-$ svn commit -m 'Upload release artifacts to staging' --username <YOUR_APACHE_ID>
-{% endhighlight %}
-
-__Cleaning up after a failed release attempt:__
-{% highlight bash %}
-# Make sure that the tag you are about to generate does not already
-# exist (due to a failed release attempt)
-$ git tag
-
-# If the tag exists, delete it locally and remotely
-$ git tag -d apache-kylin-X.Y.Z
-$ git push origin :refs/tags/apache-kylin-X.Y.Z
-
-# Remove modified files
-$ mvn release:clean
-
-# Check whether there are modified files and if so, go back to the
-# original git commit
-$ git status
-$ git reset --hard HEAD
-{% endhighlight %}
-
-# Validate a release
-{% highlight bash %}
-# Check unit test
-$ mvn test
-
-# Check that the signing key (e.g. 2AD3FAE3) is pushed
-$ gpg --recv-keys key
-
-# Check keys
-$ curl -O https://dist.apache.org/repos/dist/release/kylin/KEYS
-
-## Sign/check md5 and sha1 hashes
- _(Assumes your O/S has 'md5' and 'sha1' commands.)_
-function checkHash() {
-  cd "$1"
-  for i in *.{zip,gz}; do
-    if [ ! -f $i ]; then
-      continue
-    fi
-    if [ -f $i.md5 ]; then
-      if [ "$(cat $i.md5)" = "$(md5 -q $i)" ]; then
-        echo $i.md5 present and correct
-      else
-        echo $i.md5 does not match
-      fi
-    else
-      md5 -q $i > $i.md5
-      echo $i.md5 created
-    fi
-    if [ -f $i.sha1 ]; then
-      if [ "$(cat $i.sha1)" = "$(sha1 -q $i)" ]; then
-        echo $i.sha1 present and correct
-      else
-        echo $i.sha1 does not match
-      fi
-    else
-      sha1 -q $i > $i.sha1
-      echo $i.sha1 created
-    fi
-  done
-}
-$ checkHash apache-kylin-X.Y.Z-rcN
-{% endhighlight %}
-
-## Apache voting process  
-
-__Vote on Apache Kylin dev mailing list__  
-Release vote on dev list, use the commit id that generated by Maven release plugin, whose message looks like "[maven-release-plugin] prepare release kylin-x.x.x":  
-
-{% highlight text %}
-To: dev@kylin.apache.org
-Subject: [VOTE] Release apache-kylin-X.Y.Z (release candidate N)
-
-Hi all,
-
-I have created a build for Apache Kylin X.Y.Z, release candidate N.
-
-Changes highlights:
-...
-
-Thanks to everyone who has contributed to this release.
-Here’s release notes:
-https://github.com/apache/kylin/blob/XXX/docs/release_notes.md
-
-The commit to be voted upon:
-
-https://github.com/apache/kylin/commit/xxx
-
-Its hash is xxx.
-
-The artifacts to be voted on are located here:
-https://dist.apache.org/repos/dist/dev/kylin/apache-kylin-X.Y.Z-rcN/
-
-The hashes of the artifacts are as follows:
-src.zip.md5 xxx
-src.zip.sha1 xxx
-src.tar.gz.md5 xxx
-src.tar.gz.sha1 xxx
-
-A staged Maven repository is available for review at:
-https://repository.apache.org/content/repositories/orgapachekylin-XXXX/
-
-Release artifacts are signed with the following key:
-https://people.apache.org/keys/committer/lukehan.asc
-
-Please vote on releasing this package as Apache Kylin X.Y.Z.
-
-The vote is open for the next 72 hours and passes if a majority of
-at least three +1 PPMC votes are cast.
-
-[ ] +1 Release this package as Apache Kylin X.Y.Z
-[ ]  0 I don't feel strongly about it, but I'm okay with the release
-[ ] -1 Do not release this package because...
-
-
-Here is my vote:
-
-+1 (binding)
-
-Luke
-
-{% endhighlight %}
-
-After vote finishes, send out the result:  
-{% highlight text %}
-Subject: [RESULT] [VOTE] Release apache-kylin-X.Y.Z (release candidate N)
-To: dev@kylin.apache.org
-
-Thanks to everyone who has tested the release candidate and given
-their comments and votes.
-
-The tally is as follows.
-
-N binding +1s:
-
-N non-binding +1s:
-
-No 0s or -1s.
-
-Therefore I am delighted to announce that the proposal to release
-Apache-Kylin-X.Y.Z has passed.
-
-I'll now start a vote on the general list. Those of you in the IPMC,
-please recast your vote on the new thread.
-
-Luke
-
-{% endhighlight %}
-
-## Publishing a release  
-After a successful release vote, we need to push the release
-out to mirrors, and other tasks.
-
-In JIRA, search for
-[all issues resolved in this release](https://issues.apache.org/jira/issues/?jql=project%20%3D%20KYLIN%20),
-and do a bulk update changing their status to "Closed",
-with a change comment
-"Resolved in release X.Y.Z (YYYY-MM-DD)"
-(fill in release number and date appropriately).  
-__Uncheck "Send mail for this update".__
-
-Promote the staged nexus artifacts.
-
-* Go to [https://repository.apache.org/](https://repository.apache.org/) and login
-* Under "Build Promotion" click "Staging Repositories"
-* In the line with "orgapachekylin-xxxx", check the box
-* Press "Release" button
-
-Check the artifacts into svn.
-{% highlight bash %}
-# Get the release candidate.
-$ mkdir -p ~/dist/dev
-$ cd ~/dist/dev
-$ svn co https://dist.apache.org/repos/dist/dev/kylin
-
-# Copy the artifacts. Note that the copy does not have '-rcN' suffix.
-$ mkdir -p ~/dist/release
-$ cd ~/dist/release
-$ svn co https://dist.apache.org/repos/dist/release/kylin
-$ cd kylin
-$ cp -rp ../../dev/kylin/apache-kylin-X.Y.Z-rcN apache-kylin-X.Y.Z
-$ svn add apache-kylin-X.Y.Z
-
-# Check in.
-svn commit -m 'checkin release artifacts'
-{% endhighlight %}
-
-Svnpubsub will publish to
-[https://dist.apache.org/repos/dist/release/kylin](https://dist.apache.org/repos/dist/release/kylin) and propagate to
-[http://www.apache.org/dyn/closer.cgi/kylin](http://www.apache.org/dyn/closer.cgi/kylin) within 24 hours.
-
-If there are now more than 2 releases, clear out the oldest ones:
-
-{% highlight bash %}
-cd ~/dist/release/kylin
-svn rm apache-kylin-X.Y.Z
-svn commit -m 'Remove old release'
-{% endhighlight %}
-
-The old releases will remain available in the
-[release archive](http://archive.apache.org/dist/kylin/).
-
-Release same version in JIRA, check [Change Log](https://issues.apache.org/jira/browse/KYLIN/?selectedTab=com.atlassian.jira.jira-projects-plugin:changelog-panel) for the latest released version.
-
-## Publishing the web site  
-Refer to [How to document](howto_docs.html) for more detail.
-
-## Send announcement mail to mailing list
-Send one mail with subject like "[Announce] Apache Kylin x.y released" to following list:
-* Apache Kylin Dev mailing list: dev@kylin.apache.org
-* Apache Kylin User mailing list: user@kylin.apache.org
-* Apache Announce mailing list: announce@apache.org
-Please notice to always use your Apache mail address to send this
-
-# Thanks  
-This guide drafted with reference from [Apache Calcite](http://calcite.apache.org) Howto doc, Thank you very much.
-

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_dev/howto_test.md
----------------------------------------------------------------------
diff --git a/website/_dev/howto_test.md b/website/_dev/howto_test.md
deleted file mode 100644
index 150ece8..0000000
--- a/website/_dev/howto_test.md
+++ /dev/null
@@ -1,49 +0,0 @@
----
-layout: dev
-title:  "How to Test"
-categories: development
-permalink: /development/howto_test.html
----
-
-In general, there should be unit tests to cover individual classes; there must be integration test to cover end-to-end scenarios like build, merge, and query. Unit test must run independently (does not require an external sandbox).
-
-
-## 2.x branches
-
-* `mvn test` to run unit tests, which has a limited test coverage.
-    * Unit tests has no external dependency and can run on any machine.
-    * The unit tests do not cover end-to-end scenarios like build, merge, and query.
-    * The unit tests take a few minutes to complete.
-* `dev-support/test_all_against_hdp_2_2_4_2_2.sh` to run integration tests, which has the best test coverage.
-    * Integration tests __must run on a Hadoop sandbox__. Make sure all changes you want to test are avaiable on sandbox.
-    * As the name indicates, the script is only for hdp 2.2.4.2, but you get the idea of how integration test run from it.
-    * The integration tests start from generate random data, then build cube, merge cube, and finally query the result and compare to H2 DB.
-    * The integration tests take a few hours to complete.
-
-
-## 1.x branches
-
-* `mvn test` to run unit tests, which has a limited test coverage.
-    * What's special about 1.x is that a hadoop/hbase mini cluster is used to cover queries in unit test.
-* Run the following to run integration tests.
-    * `mvn clean package -DskipTests`
-    * `mvn test  -Dtest=org.apache.kylin.job.BuildCubeWithEngineTest -Dhdp.version=2.2.0.0-2041 -DfailIfNoTests=false -P sandbox`
-    * `mvn test  -Dtest=org.apache.kylin.job.BuildIIWithEngineTest -Dhdp.version=2.2.0.0-2041 -DfailIfNoTests=false -P sandbox`
-    * `mvn test  -fae -P sandbox`
-    * `mvn test  -fae  -Dtest=org.apache.kylin.query.test.IIQueryTest -Dhdp.version=2.2.0.0-2041 -DfailIfNoTests=false -P sandbox`
-
-
-## More on 1.x Mini Cluster
-
-Kylin 1.x used to move as many as possible unit test cases from sandbox to HBase mini cluster (not any more in 2.x), so that user can run tests easily in local without a hadoop sandbox. Two maven profiles are created in the root pom.xml, "default" and "sandbox". The default profile will startup a HBase Mini Cluster to prepare the test data and run the unit tests (the test cases that are not supported by Mini cluster have been added in the "exclude" list). If you want to keep using Sandbox to run test, just run `mvn test -P sandbox`
-
-
-### When use the "default" profile, Kylin will
-
-* Startup a HBase minicluster and update KylinConfig with the dynamic HBase configurations
-* Create Kylin metadata tables and import six example cube tables
-* Import the hbase data from a tar ball from local: `examples/test_case_data/minicluster/hbase-export.tar.gz` (the hbase-export.tar.gz will be updated on complete of running BuildCubeWithEngineTest)
-* After all test cases be completed, shutdown minicluster and cleanup KylinConfig cache
-
-### To ensure Mini cluster can run successfully, you need
-* Make sure JAVA_HOME is properly set

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_dev/index.md
----------------------------------------------------------------------
diff --git a/website/_dev/index.md b/website/_dev/index.md
deleted file mode 100644
index 3a52267..0000000
--- a/website/_dev/index.md
+++ /dev/null
@@ -1,17 +0,0 @@
----
-layout: dev
-title: Development Quick Guide
-permalink: /development/index.html
----
-
-### Source Code  
-Apache Kylin™ source code is version controlled using Git version control:
-Commits [Summary](https://git-wip-us.apache.org/repos/asf?p=kylin.git;a=summary)  
-Source Repo: [git://git.apache.org/kylin.git](git://git.apache.org/kylin.git)  
-Mirrored to Github: [https://github.com/apache/kylin](https://github.com/apache/kylin)
-
-### Issue Tracking  
-Track issues on the "Kylin" Project on the Apache JIRA ([browse](http://issues.apache.org/jira/browse/KYLIN)).
-
-### Setup Development Env  
-Refer to this link for how to setup dev env: [Setup Development Env](/development/dev_env.html)
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_dev/new_metadata.md
----------------------------------------------------------------------
diff --git a/website/_dev/new_metadata.md b/website/_dev/new_metadata.md
deleted file mode 100644
index d52aa61..0000000
--- a/website/_dev/new_metadata.md
+++ /dev/null
@@ -1,30 +0,0 @@
----
-layout: dev
-title:  "New Metadata Model"
-categories: development
-permalink: /development/new_metadata.html
----
-
-Kylin is doing a round code refactor which will introduce the following two changes on meta data:
-
-* Abstract a "model_desc" layer from "cube_desc"
-
-Before define a cube, user will need firstly define a model ("model_desc"); The model defines which is the fact table, which are lookup tables and how they be joined;
-
-With the model be defined, when user define a cube ("cube_desc"), he/she only need to specify the table/column name for a dimension, as the join conditions have already been defined;
-
-This abstraction is to extend the meta data to fulfill non-cube queries (coming soon);
-
-* Support data tables from multiple hive databases;
-
-User has the case that tables are from multiple hive database, and the table name might be the same; To support this case Kylin will use the database name + table name as the unique name for tables; And user need to specify the database name (if it is not "default") in SQL when query Kylin. 
-
-Here is a sample; the fact table "test_kylin_fact" is from default hive database, you don't need to specify the db name; while lookup table is from "edw", you need use "edw.test_cal_dt" in the query:
-
-{% highlight Groff markup %}
-select test_cal_dt.Week_Beg_Dt, sum(price) as c1, count(1) as c2 
- from test_kylin_fact
- inner JOIN edw.test_cal_dt as test_cal_dt
- ON test_kylin_fact.cal_dt = test_cal_dt.cal_dt 
- where test_kylin_fact.lstg_format_name='ABIN' 
-{% endhighlight %}

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_dev/web_tech.md
----------------------------------------------------------------------
diff --git a/website/_dev/web_tech.md b/website/_dev/web_tech.md
deleted file mode 100644
index 0603eda..0000000
--- a/website/_dev/web_tech.md
+++ /dev/null
@@ -1,46 +0,0 @@
----
-layout: dev
-title:  "Kylin Web Summary"
-categories: development
-permalink: /development/web_tech.html
----
-
-### Project Dependencies
-* npm: used in development phase to install grunt and bower
-* grunt: build and set up kylin web
-* bower: manage kylin tech dependencies
-
-### Tech Dependencies
-* Angular JS: fundamental support of kylin web
-* ACE: sql and json editor
-* D3 JS: draw report chart and cube graph
-* Bootstrap: css lib
-
-### Supported Use Cases:
-
-###### Kylin web supports needs of various of roles in BI workflow. 
-
-* Analyst: Run query and checkout results
-* Modeler: cube design, cube/job operation and monitor
-* Admin: system operation.
-
-### Tech Overview 
-Kylin web is a one-page application build on top of restful services. Kylin web uses tools from nodejs to manage project and use AngularJS to enable one-page web app. Kylin web uses popular techs from js opensource community making it easy to catch up and contribute. 
-
-### Highlights:
-* Query utility functions:
-    * SQL auto-suggestions on table and column name
-    * Query remote/local save.
-    * Data grid supporting million level data with easy BI operations
-    * Data export
-    * Simple data visualization(line, bar, pie)
-* Cube management:
-    * Well-designed cube creation flow
-    * Visualization of cube relational structure.
-    * Well-designed cube access management
-* Job management:
-    * Job steps and log monitor
-    * Kill
-    * Resume
-* Useful admin tools.
-* Refined look&feel.


[42/52] [abbrv] kylin git commit: KYLIN-1416 remove website from coding branch

Posted by li...@apache.org.
http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_docs/gettingstarted/concepts.md
----------------------------------------------------------------------
diff --git a/website/_docs/gettingstarted/concepts.md b/website/_docs/gettingstarted/concepts.md
deleted file mode 100644
index 081ec54..0000000
--- a/website/_docs/gettingstarted/concepts.md
+++ /dev/null
@@ -1,65 +0,0 @@
----
-layout: docs
-title:  "Technical Concepts"
-categories: gettingstarted
-permalink: /docs/gettingstarted/concepts.html
-version: v1.2
-since: v1.2
----
- 
-Here are some basic technical concepts used in Apache Kylin, please check them for your reference.
-For terminology in domain, please refer to: [Terminology](terminology.md)
-
-## CUBE
-* __Table__ - This is definition of hive tables as source of cubes, which must be synced before building cubes.
-![](/images/docs/concepts/DataSource.png)
-
-* __Data Model__ - This describes a [STAR SCHEMA](https://en.wikipedia.org/wiki/Star_schema) data model, which defines fact/lookup tables and filter condition.
-![](/images/docs/concepts/DataModel.png)
-
-* __Cube Descriptor__ - This describes definition and settings for a cube instance, defining which data model to use, what dimensions and measures to have, how to partition to segments and how to handle auto-merge etc.
-![](/images/docs/concepts/CubeDesc.png)
-
-* __Cube Instance__ - This is instance of cube, built from one cube descriptor, and consist of one or more cube segments according partition settings.
-![](/images/docs/concepts/CubeInstance.png)
-
-* __Partition__ - User can define a DATE/STRING column as partition column on cube descriptor, to separate one cube into several segments with different date periods.
-![](/images/docs/concepts/Partition.png)
-
-* __Cube Segment__ - This is actual carrier of cube data, and maps to a HTable in HBase. One building job creates one new segment for the cube instance. Once data change on specified data period, we can refresh related segments to avoid rebuilding whole cube.
-![](/images/docs/concepts/CubeSegment.png)
-
-* __Aggregation Group__ - Each aggregation group is subset of dimensions, and build cuboid with combinations inside. It aims at pruning for optimization.
-![](/images/docs/concepts/AggregationGroup.png)
-
-## DIMENSION & MEASURE
-* __Mandotary__ - This dimension type is used for cuboid pruning, if a dimension is specified as “mandatory”, then those combinations without such dimension are pruned.
-* __Hierarchy__ - This dimension type is used for cuboid pruning, if dimension A,B,C forms a “hierarchy” relation, then only combinations with A, AB or ABC shall be remained. 
-* __Derived__ - On lookup tables, some dimensions could be generated from its PK, so there's specific mapping between them and FK from fact table. So those dimensions are DERIVED and don't participate in cuboid generation.
-![](/images/docs/concepts/Dimension.png)
-
-* __Count Distinct(HyperLogLog)__ - Immediate COUNT DISTINCT is hard to calculate, a approximate algorithm - [HyperLogLog](https://en.wikipedia.org/wiki/HyperLogLog) is introduced, and keep error rate in a lower level. 
-* __Count Distinct(Precise)__ - Precise COUNT DISTINCT will be pre-calculated basing on RoaringBitmap, currently only int or bigint are supported.
-* __Top N__ - (Will release in 2.x) For example, with this measure type, user can easily get specified numbers of top sellers/buyers etc. 
-![](/images/docs/concepts/Measure.png)
-
-## CUBE ACTIONS
-* __BUILD__ - Given an interval of partition column, this action is to build a new cube segment.
-* __REFRESH__ - This action will rebuilt cube segment in some partition period, which is used in case of source table increasing.
-* __MERGE__ - This action will merge multiple continuous cube segments into single one. This can be automated with auto-merge settings in cube descriptor.
-* __PURGE__ - Clear segments under a cube instance. This will only update metadata, and won't delete cube data from HBase.
-![](/images/docs/concepts/CubeAction.png)
-
-## JOB STATUS
-* __NEW__ - This denotes one job has been just created.
-* __PENDING__ - This denotes one job is paused by job scheduler and waiting for resources.
-* __RUNNING__ - This denotes one job is running in progress.
-* __FINISHED__ - This denotes one job is successfully finished.
-* __ERROR__ - This denotes one job is aborted with errors.
-* __DISCARDED__ - This denotes one job is cancelled by end users.
-![](/images/docs/concepts/Job.png)
-
-## JOB ACTION
-* __RESUME__ - Once a job in ERROR status, this action will try to restore it from latest successful point.
-* __DISCARD__ - No matter status of a job is, user can end it and release resources with DISCARD action.
-![](/images/docs/concepts/JobAction.png)
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_docs/gettingstarted/events.md
----------------------------------------------------------------------
diff --git a/website/_docs/gettingstarted/events.md b/website/_docs/gettingstarted/events.md
deleted file mode 100644
index 8605d4a..0000000
--- a/website/_docs/gettingstarted/events.md
+++ /dev/null
@@ -1,27 +0,0 @@
----
-layout: docs
-title:  "Events and Conferences"
-categories: gettingstarted
-permalink: /docs/gettingstarted/events.html
----
-
-__Coming Events__
-
-* ApacheCon EU 2015
-
-__Conferences__
-
-* [Apache Kylin - Balance Between Space and Time](http://www.chinahadoop.com/2015/July/Shanghai/agenda.php) ([slides](http://www.slideshare.net/qhzhou/apache-kylin-china-hadoop-summit-2015-shanghai)) by [Qianhao Zhou](https://github.com/qhzhou), at Hadoop Summit 2015 in Shanghai, China, 2015-07-24
-* [Apache Kylin - Balance Between Space and Time](https://www.youtube.com/watch?v=jgvZSFaXPgI), [slides](http://www.slideshare.net/DebashisSaha/apache-kylin-balance-between-space-and-time-hadop-summit-2015) ([video](https://www.youtube.com/watch?v=jgvZSFaXPgI), [slides](http://www.slideshare.net/DebashisSaha/apache-kylin-balance-between-space-and-time-hadop-summit-2015)) by [Debashis Saha](https://twitter.com/debashis_saha) & [Luke Han](https://twitter.com/lukehq), at Hadoop Summit 2015 in San Jose, US, 2015-06-09
-* [HBaseCon 2015: Apache Kylin; Extreme OLAP Engine for Hadoop](https://vimeo.com/128152444) ([video](https://vimeo.com/128152444), [slides](http://www.slideshare.net/HBaseCon/ecosystem-session-3b)) by [Seshu Adunuthula](https://twitter.com/SeshuAd) at HBaseCon 2015 in San Francisco, US, 2015-05-07
-* [Apache Kylin - Extreme OLAP Engine for Hadoop](http://strataconf.com/big-data-conference-uk-2015/public/schedule/detail/40029) ([slides](http://www.slideshare.net/lukehan/apache-kylin-extreme-olap-engine-for-big-data)) by [Luke Han](https://twitter.com/lukehq) & [Yang Li](https://github.com/liyang-gmt8), at Strata+Hadoop World in London, UK, 2015-05-06
-* [Apache Kylin Open Source Journey](http://www.infoq.com/cn/presentations/open-source-journey-of-apache-kylin) ([slides](http://www.slideshare.net/lukehan/apache-kylin-open-source-journey-for-qcon2015-beijing)) by [Luke Han](https://twitter.com/lukehq), at QCon Beijing in Beijing, China, 2015-04-23
-* [Apache Kylin - OLAP on Hadoop](http://cio.it168.com/a2015/0418/1721/000001721404.shtml) by [Yang Li](https://github.com/liyang-gmt8), at Database Technology Conference China 2015 in Beijing, China, 2015-04-18
-* [Apache Kylin – Cubes on Hadoop](https://www.youtube.com/watch?v=U0SbrVzuOe4) ([video](https://www.youtube.com/watch?v=U0SbrVzuOe4), [slides](http://www.slideshare.net/Hadoop_Summit/apache-kylin-cubes-on-hadoop)) by [Ted Dunning](https://twitter.com/ted_dunning), at Hadoop Summit 2015 Europe in Brussels, Belgium, 2015-04-16
-* [Apache Kylin - Hadoop 上的大规模联机分析平台](http://bdtc2014.hadooper.cn/m/zone/bdtc_2014/schedule3) ([slides](http://www.slideshare.net/lukehan/apache-kylin-big-data-technology-conference-2014-beijing-v2)) by [Luke Han](https://twitter.com/lukehq), at Big Data Technology Conference China in Beijing, China, 2014-12-14
-* [Apache Kylin: OLAP Engine on Hadoop - Tech Deep Dive](http://v.csdn.hudong.com/s/article.html?arcid=15820707) ([video](http://v.csdn.hudong.com/s/article.html?arcid=15820707), [slides](http://www.slideshare.net/XuJiang2/kylin-hadoop-olap-engine)) by [Jiang Xu](https://www.linkedin.com/pub/xu-jiang/4/5a8/230), at Shanghai Big Data Summit 2014 in Shanghai, China , 2014-10-25
-
-__Meetup__
-
-* [Apache Kylin Meetup @Bay Area](http://www.meetup.com/Cloud-at-ebayinc/events/218914395/), in San Jose, US, 6:00PM - 7:30PM, Thursday, 2014-12-04
-

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_docs/gettingstarted/faq.md
----------------------------------------------------------------------
diff --git a/website/_docs/gettingstarted/faq.md b/website/_docs/gettingstarted/faq.md
deleted file mode 100644
index 24294ee..0000000
--- a/website/_docs/gettingstarted/faq.md
+++ /dev/null
@@ -1,90 +0,0 @@
----
-layout: docs
-title:  "FAQ"
-categories: gettingstarted
-permalink: /docs/gettingstarted/faq.html
-version: v0.7.2
-since: v0.6.x
----
-
-### Some NPM error causes ERROR exit (中国大陆地区用户请特别注意此问题)?  
-For people from China:  
-
-* Please add proxy for your NPM (请为NPM设置代理):  
-`npm config set proxy http://YOUR_PROXY_IP`
-
-* Please update your local NPM repository to using any mirror of npmjs.org, like Taobao NPM (请更新您本地的NPM仓库以使用国内的NPM镜像,例如淘宝NPM镜像) :  
-[http://npm.taobao.org](http://npm.taobao.org)
-
-### Can't get master address from ZooKeeper" when installing Kylin on Hortonworks Sandbox
-Check out [https://github.com/KylinOLAP/Kylin/issues/9](https://github.com/KylinOLAP/Kylin/issues/9).
-
-### Map Reduce Job information can't display on sandbox deployment
-Check out [https://github.com/KylinOLAP/Kylin/issues/40](https://github.com/KylinOLAP/Kylin/issues/40)
-
-#### Install Kylin on CDH 5.2 or Hadoop 2.5.x
-Check out discussion: [https://groups.google.com/forum/?utm_medium=email&utm_source=footer#!msg/kylin-olap/X0GZfsX1jLc/nzs6xAhNpLkJ](https://groups.google.com/forum/?utm_medium=email&utm_source=footer#!msg/kylin-olap/X0GZfsX1jLc/nzs6xAhNpLkJ)
-{% highlight Groff markup %}
-I was able to deploy Kylin with following option in POM.
-<hadoop2.version>2.5.0</hadoop2.version>
-<yarn.version>2.5.0</yarn.version>
-<hbase-hadoop2.version>0.98.6-hadoop2</hbase-hadoop2.version>
-<zookeeper.version>3.4.5</zookeeper.version>
-<hive.version>0.13.1</hive.version>
-My Cluster is running on Cloudera Distribution CDH 5.2.0.
-{% endhighlight %}
-
-#### Unable to load a big cube as HTable, with java.lang.OutOfMemoryError: unable to create new native thread
-HBase (as of writing) allocates one thread per region when bulk loading a HTable. Try reduce the number of regions of your cube by setting its "capacity" to "MEDIUM" or "LARGE". Also tweaks OS & JVM can allow more threads, for example see [this article](http://blog.egilh.com/2006/06/2811aspx.html).
-
-#### Failed to run BuildCubeWithEngineTest, saying failed to connect to hbase while hbase is active
-User may get this error when first time run hbase client, please check the error trace to see whether there is an error saying couldn't access a folder like "/hadoop/hbase/local/jars"; If that folder doesn't exist, create it.
-
-#### SUM(field) returns a negtive result while all the numbers in this field are > 0
-If a column is declared as integer in Hive, the SQL engine (calcite) will use column's type (integer) as the data type for "SUM(field)", while the aggregated value on this field may exceed the scope of integer; in that case the cast will cause a negtive value be returned; The workround is, alter that column's type to BIGINT in hive, and then sync the table schema to Kylin (the cube doesn't need rebuild); Keep in mind that, always declare as BIGINT in hive for an integer column which would be used as a measure in Kylin; See hive number types: [https://cwiki.apache.org/confluence/display/Hive/LanguageManual+Types#LanguageManualTypes-NumericTypes](https://cwiki.apache.org/confluence/display/Hive/LanguageManual+Types#LanguageManualTypes-NumericTypes)
-
-#### Why Kylin need extract the distinct columns from Fact Table before building cube?
-Kylin uses dictionary to encode the values in each column, this greatly reduce the cube's storage size. To build the dictionary, Kylin need fetch the distinct values for each column.
-
-#### Why Kylin calculate the HIVE table cardinality?
-The cardinality of dimensions is an important measure of cube complexity. The higher the cardinality, the bigger the cube, and thus the longer to build and the slower to query. Cardinality > 1,000 is worth attention and > 1,000,000 should be avoided at best effort. For optimal cube performance, try reduce high cardinality by categorize values or derive features.
-
-#### How to add new user or change the default password?
-Kylin web's security is implemented with Spring security framework, where the kylinSecurity.xml is the main configuration file:
-{% highlight Groff markup %}
-${KYLIN_HOME}/tomcat/webapps/kylin/WEB-INF/classes/kylinSecurity.xml
-{% endhighlight %}
-The password hash for pre-defined test users can be found in the profile "sandbox,testing" part; To change the default password, you need generate a new hash and then update it here, please refer to the code snippet in: [https://stackoverflow.com/questions/25844419/spring-bcryptpasswordencoder-generate-different-password-for-same-input](https://stackoverflow.com/questions/25844419/spring-bcryptpasswordencoder-generate-different-password-for-same-input)
-When you deploy Kylin for more users, switch to LDAP authentication is recommended; To enable LDAP authentication, update "kylin.sandbox" in conf/kylin.properties to false, and also configure the ldap.* properties in ${KYLIN_HOME}/conf/kylin.properties
-
-#### Using sub-query for un-supported SQL
-
-{% highlight Groff markup %}
-Original SQL:
-select fact.slr_sgmt,
-sum(case when cal.RTL_WEEK_BEG_DT = '2015-09-06' then gmv else 0 end) as W36,
-sum(case when cal.RTL_WEEK_BEG_DT = '2015-08-30' then gmv else 0 end) as W35
-from ih_daily_fact fact
-inner join dw_cal_dt cal on fact.cal_dt = cal.cal_dt
-group by fact.slr_sgmt
-{% endhighlight %}
-
-{% highlight Groff markup %}
-Using sub-query
-select a.slr_sgmt,
-sum(case when a.RTL_WEEK_BEG_DT = '2015-09-06' then gmv else 0 end) as W36,
-sum(case when a.RTL_WEEK_BEG_DT = '2015-08-30' then gmv else 0 end) as W35
-from (
-    select fact.slr_sgmt as slr_sgmt,
-    cal.RTL_WEEK_BEG_DT as RTL_WEEK_BEG_DT,
-    sum(gmv) as gmv36,
-    sum(gmv) as gmv35
-    from ih_daily_fact fact
-    inner join dw_cal_dt cal on fact.cal_dt = cal.cal_dt
-    group by fact.slr_sgmt, cal.RTL_WEEK_BEG_DT
-) a
-group by a.slr_sgmt
-{% endhighlight %}
-
-
-

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_docs/gettingstarted/terminology.md
----------------------------------------------------------------------
diff --git a/website/_docs/gettingstarted/terminology.md b/website/_docs/gettingstarted/terminology.md
deleted file mode 100644
index 0f9e669..0000000
--- a/website/_docs/gettingstarted/terminology.md
+++ /dev/null
@@ -1,26 +0,0 @@
----
-layout: docs
-title:  "Terminology"
-categories: gettingstarted
-permalink: /docs/gettingstarted/terminology.html
-version: v1.0
-since: v0.5.x
----
- 
-
-Here are some domain terms we are using in Apache Kylin, please check them for your reference.   
-They are basic knowledge of Apache Kylin which also will help to well understand such concerpt, term, knowledge, theory and others about Data Warehouse, Business Intelligence for analycits. 
-
-* __Data Warehouse__: a data warehouse (DW or DWH), also known as an enterprise data warehouse (EDW), is a system used for reporting and data analysis, [wikipedia](https://en.wikipedia.org/wiki/Data_warehouse)
-* __Business Intelligence__: Business intelligence (BI) is the set of techniques and tools for the transformation of raw data into meaningful and useful information for business analysis purposes, [wikipedia](https://en.wikipedia.org/wiki/Business_intelligence)
-* __OLAP__: OLAP is an acronym for [online analytical processing](https://en.wikipedia.org/wiki/Online_analytical_processing)
-* __OLAP Cube__: an OLAP cube is an array of data understood in terms of its 0 or more dimensions, [wikipedia](http://en.wikipedia.org/wiki/OLAP_cube)
-* __Star Schema__: the star schema consists of one or more fact tables referencing any number of dimension tables, [wikipedia](https://en.wikipedia.org/wiki/Star_schema)
-* __Fact Table__: a Fact table consists of the measurements, metrics or facts of a business process, [wikipedia](https://en.wikipedia.org/wiki/Fact_table)
-* __Lookup Table__: a lookup table is an array that replaces runtime computation with a simpler array indexing operation, [wikipedia](https://en.wikipedia.org/wiki/Lookup_table)
-* __Dimension__: A dimension is a structure that categorizes facts and measures in order to enable users to answer business questions. Commonly used dimensions are people, products, place and time, [wikipedia](https://en.wikipedia.org/wiki/Dimension_(data_warehouse))
-* __Measure__: a measure is a property on which calculations (e.g., sum, count, average, minimum, maximum) can be made, [wikipedia](https://en.wikipedia.org/wiki/Measure_(data_warehouse))
-* __Join__: a SQL join clause combines records from two or more tables in a relational database, [wikipedia](https://en.wikipedia.org/wiki/Join_(SQL))
-
-
-

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_docs/howto/howto_backup_hbase.md
----------------------------------------------------------------------
diff --git a/website/_docs/howto/howto_backup_hbase.md b/website/_docs/howto/howto_backup_hbase.md
deleted file mode 100644
index 8714132..0000000
--- a/website/_docs/howto/howto_backup_hbase.md
+++ /dev/null
@@ -1,29 +0,0 @@
----
-layout: docs
-title:  How to Clean/Backup HBase Tables
-categories: howto
-permalink: /docs/howto/howto_backup_hbase.html
-version: v1.0
-since: v0.7.1
----
-
-Kylin persists all data (meta data and cube) in HBase; You may want to export the data sometimes for whatever purposes 
-(backup, migration, troubleshotting etc); This page describes the steps to do this and also there is a Java app for you to do this easily.
-
-Steps:
-
-1. Cleanup unused cubes to save storage space (be cautious on production!): run the following command in hbase CLI: 
-{% highlight Groff markup %}
-hbase org.apache.hadoop.util.RunJar /${KYLIN_HOME}/lib/kylin-job-(version).jar org.apache.kylin.job.hadoop.cube.StorageCleanupJob --delete true
-{% endhighlight %}
-2. List all HBase tables, iterate and then export each Kylin table to HDFS; 
-See [https://hbase.apache.org/book/ops_mgt.html#export](https://hbase.apache.org/book/ops_mgt.html#export)
-
-3. Copy the export folder from HDFS to local file system, and then archive it;
-
-4. (optional) Download the archive from Hadoop CLI to local;
-
-5. Cleanup the export folder from CLI HDFS and local file system;
-
-Kylin provide the "ExportHBaseData.java" (currently only exist in "minicluster" branch) for you to do the 
-step 2-5 in one run; Please ensure the correct path of "kylin.properties" has been set in the sys env; This Java uses the sandbox config by default;

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_docs/howto/howto_backup_metadata.md
----------------------------------------------------------------------
diff --git a/website/_docs/howto/howto_backup_metadata.md b/website/_docs/howto/howto_backup_metadata.md
deleted file mode 100644
index cf250b7..0000000
--- a/website/_docs/howto/howto_backup_metadata.md
+++ /dev/null
@@ -1,62 +0,0 @@
----
-layout: docs
-title:  How to Backup Metadata
-categories: howto
-permalink: /docs/howto/howto_backup_metadata.html
-version: v1.0
-since: v0.7.1
----
-
-Kylin organizes all of its metadata (including cube descriptions and instances, projects, inverted index description and instances, jobs, tables and dictionaries) as a hierarchy file system. However, Kylin uses hbase to store it, rather than normal file system. If you check your kylin configuration file(kylin.properties) you will find such a line:
-
-{% highlight Groff markup %}
-## The metadata store in hbase
-kylin.metadata.url=kylin_metadata@hbase
-{% endhighlight %}
-
-This indicates that the metadata will be saved as a htable called `kylin_metadata`. You can scan the htable in hbase shell to check it out.
-
-## Backup Metadata Store with binary package
-
-Sometimes you need to backup the Kylin's Metadata Store from hbase to your disk file system.
-In such cases, assuming you're on the hadoop CLI(or sandbox) where you deployed Kylin, you can go to KYLIN_HOME and run :
-
-{% highlight Groff markup %}
-./bin/metastore.sh backup
-{% endhighlight %}
-
-to dump your metadata to your local folder a folder under KYLIN_HOME/metadata_backps, the folder is named after current time with the syntax: KYLIN_HOME/meta_backups/meta_year_month_day_hour_minute_second
-
-## Restore Metadata Store with binary package
-
-In case you find your metadata store messed up, and you want to restore to a previous backup:
-
-Firstly, reset the metadata store (this will clean everything of the Kylin metadata store in hbase, make sure to backup):
-
-{% highlight Groff markup %}
-./bin/metastore.sh reset
-{% endhighlight %}
-
-Then upload the backup metadata to Kylin's metadata store:
-{% highlight Groff markup %}
-./bin/metastore.sh restore $KYLIN_HOME/meta_backups/meta_xxxx_xx_xx_xx_xx_xx
-{% endhighlight %}
-
-## Backup/restore metadata in development env (available since 0.7.3)
-
-When developing/debugging Kylin, typically you have a dev machine with an IDE, and a backend sandbox. Usually you'll write code and run test cases at dev machine. It would be troublesome if you always have to put a binary package in the sandbox to check the metadata. There is a helper class called SandboxMetastoreCLI to help you download/upload metadata locally at your dev machine. Follow the Usage information and run it in your IDE.
-
-## Cleanup unused resources from Metadata Store (available since 0.7.3)
-As time goes on, some resources like dictionary, table snapshots became useless (as the cube segment be dropped or merged), but they still take space there; You can run command to find and cleanup them from metadata store:
-
-Firstly, run a check, this is safe as it will not change anything:
-{% highlight Groff markup %}
-./bin/metastore.sh clean
-{% endhighlight %}
-
-The resources that will be dropped will be listed;
-
-Next, add the "--delete true" parameter to cleanup those resources; before this, make sure you have made a backup of the metadata store;
-{% highlight Groff markup %}
-./bin/metastore.sh clean --delete true
-{% endhighlight %}

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_docs/howto/howto_build_cube_with_restapi.md
----------------------------------------------------------------------
diff --git a/website/_docs/howto/howto_build_cube_with_restapi.md b/website/_docs/howto/howto_build_cube_with_restapi.md
deleted file mode 100644
index 4113700..0000000
--- a/website/_docs/howto/howto_build_cube_with_restapi.md
+++ /dev/null
@@ -1,55 +0,0 @@
----
-layout: docs
-title:  How to Build Cube with Restful API
-categories: howto
-permalink: /docs/howto/howto_build_cube_with_restapi.html
-version: v1.2
-since: v0.7.1
----
-
-### 1.	Authentication
-*   Currently, Kylin uses [basic authentication](http://en.wikipedia.org/wiki/Basic_access_authentication).
-*   Add `Authorization` header to first request for authentication
-*   Or you can do a specific request by `POST http://localhost:7070/kylin/api/user/authentication`
-*   Once authenticated, client can go subsequent requests with cookies.
-{% highlight Groff markup %}
-POST http://localhost:7070/kylin/api/user/authentication
-    
-Authorization:Basic xxxxJD124xxxGFxxxSDF
-Content-Type: application/json;charset=UTF-8
-{% endhighlight %}
-
-### 2.	Get details of cube. 
-*   `GET http://localhost:7070/kylin/api/cubes?cubeName={cube_name}&limit=15&offset=0`
-*   Client can find cube segment date ranges in returned cube detail.
-{% highlight Groff markup %}
-GET http://localhost:7070/kylin/api/cubes?cubeName=test_kylin_cube_with_slr&limit=15&offset=0
-
-Authorization:Basic xxxxJD124xxxGFxxxSDF
-Content-Type: application/json;charset=UTF-8
-{% endhighlight %}
-### 3.	Then submit a build job of the cube. 
-*   `PUT http://localhost:7070/kylin/api/cubes/{cube_name}/rebuild`
-*   For put request body detail please refer to [Build Cube API](howto_use_restapi.html#build-cube). 
-    *   `startTime` and `endTime` should be utc timestamp.
-    *   `buildType` can be `BUILD` ,`MERGE` or `REFRESH`. `BUILD` is for building a new segment, `REFRESH` for refreshing an existing segment. `MERGE` is for merging multiple existing segments into one bigger segment.
-*   This method will return a new created job instance,  whose uuid is the unique id of job to track job status.
-{% highlight Groff markup %}
-PUT http://localhost:7070/kylin/api/cubes/test_kylin_cube_with_slr/rebuild
-
-Authorization:Basic xxxxJD124xxxGFxxxSDF
-Content-Type: application/json;charset=UTF-8
-    
-{
-    "startTime": 0,
-    "endTime": 1388563200000,
-    "buildType": "BUILD"
-}
-{% endhighlight %}
-
-### 4.	Track job status. 
-*   `GET http://localhost:7070/kylin/api/jobs/{job_uuid}`
-*   Returned `job_status` represents current status of job.
-
-### 5.	If the job got errors, you can resume it. 
-*   `PUT http://localhost:7070/kylin/api/jobs/{job_uuid}/resume`

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_docs/howto/howto_cleanup_storage.md
----------------------------------------------------------------------
diff --git a/website/_docs/howto/howto_cleanup_storage.md b/website/_docs/howto/howto_cleanup_storage.md
deleted file mode 100644
index ef35d68..0000000
--- a/website/_docs/howto/howto_cleanup_storage.md
+++ /dev/null
@@ -1,23 +0,0 @@
----
-layout: docs
-title:  How to Cleanup Storage (HDFS & HBase Tables)
-categories: howto
-permalink: /docs/howto/howto_cleanup_storage.html
-version: v0.7.2
-since: v0.7.1
----
-
-Kylin will generate intermediate files in HDFS during the cube building; Besides, when purge/drop/merge cubes, some HBase tables may be left in HBase and will no longer be queried; Although Kylin has started to do some 
-automated garbage collection, it might not cover all cases; You can do an offline storage cleanup periodically:
-
-Steps:
-1. Check which resources can be cleanup, this will not remove anything:
-{% highlight Groff markup %}
-hbase org.apache.hadoop.util.RunJar ${KYLIN_HOME}/lib/kylin-job-(version).jar org.apache.kylin.job.hadoop.cube.StorageCleanupJob --delete false
-{% endhighlight %}
-Here please replace (version) with the specific Kylin jar version in your installation;
-2. You can pickup 1 or 2 resources to check whether they're no longer be referred; Then add the "--delete true" option to start the cleanup:
-{% highlight Groff markup %}
-hbase org.apache.hadoop.util.RunJar ${KYLIN_HOME}/lib/kylin-job-(version).jar org.apache.kylin.job.hadoop.cube.StorageCleanupJob --delete true
-{% endhighlight %}
-On finish, the intermediate HDFS location and HTables will be dropped;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_docs/howto/howto_jdbc.md
----------------------------------------------------------------------
diff --git a/website/_docs/howto/howto_jdbc.md b/website/_docs/howto/howto_jdbc.md
deleted file mode 100644
index 3e088fe..0000000
--- a/website/_docs/howto/howto_jdbc.md
+++ /dev/null
@@ -1,94 +0,0 @@
----
-layout: docs
-title:  How to Use kylin Remote JDBC Driver
-categories: howto
-permalink: /docs/howto/howto_jdbc.html
-version: v1.2
-since: v0.7.1
----
-
-### Authentication
-
-###### Build on kylin authentication restful service. Supported parameters:
-* user : username 
-* password : password
-* ssl: true/false. Default be false; If true, all the services call will use https.
-
-### Connection URL format:
-{% highlight Groff markup %}
-jdbc:kylin://<hostname>:<port>/<kylin_project_name>
-{% endhighlight %}
-* If "ssl" = true, the "port" should be Kylin server's HTTPS port; 
-* If "port" is not specified, the driver will use default port: HTTP 80, HTTPS 443;
-* The "kylin_project_name" must be specified and user need ensure it exists in Kylin server;
-
-### 1. Query with Statement
-{% highlight Groff markup %}
-Driver driver = (Driver) Class.forName("org.apache.kylin.jdbc.Driver").newInstance();
-
-Properties info = new Properties();
-info.put("user", "ADMIN");
-info.put("password", "KYLIN");
-Connection conn = driver.connect("jdbc:kylin://localhost:7070/kylin_project_name", info);
-Statement state = conn.createStatement();
-ResultSet resultSet = state.executeQuery("select * from test_table");
-
-while (resultSet.next()) {
-    assertEquals("foo", resultSet.getString(1));
-    assertEquals("bar", resultSet.getString(2));
-    assertEquals("tool", resultSet.getString(3));
-}
-{% endhighlight %}
-
-### 2. Query with PreparedStatement
-
-###### Supported prepared statement parameters:
-* setString
-* setInt
-* setShort
-* setLong
-* setFloat
-* setDouble
-* setBoolean
-* setByte
-* setDate
-* setTime
-* setTimestamp
-
-{% highlight Groff markup %}
-Driver driver = (Driver) Class.forName("org.apache.kylin.jdbc.Driver").newInstance();
-Properties info = new Properties();
-info.put("user", "ADMIN");
-info.put("password", "KYLIN");
-Connection conn = driver.connect("jdbc:kylin://localhost:7070/kylin_project_name", info);
-PreparedStatement state = conn.prepareStatement("select * from test_table where id=?");
-state.setInt(1, 10);
-ResultSet resultSet = state.executeQuery();
-
-while (resultSet.next()) {
-    assertEquals("foo", resultSet.getString(1));
-    assertEquals("bar", resultSet.getString(2));
-    assertEquals("tool", resultSet.getString(3));
-}
-{% endhighlight %}
-
-### 3. Get query result set metadata
-Kylin jdbc driver supports metadata list methods:
-List catalog, schema, table and column with sql pattern filters(such as %).
-
-{% highlight Groff markup %}
-Driver driver = (Driver) Class.forName("org.apache.kylin.jdbc.Driver").newInstance();
-Properties info = new Properties();
-info.put("user", "ADMIN");
-info.put("password", "KYLIN");
-Connection conn = driver.connect("jdbc:kylin://localhost:7070/kylin_project_name", info);
-Statement state = conn.createStatement();
-ResultSet resultSet = state.executeQuery("select * from test_table");
-
-ResultSet tables = conn.getMetaData().getTables(null, null, "dummy", null);
-while (tables.next()) {
-    for (int i = 0; i < 10; i++) {
-        assertEquals("dummy", tables.getString(i + 1));
-    }
-}
-{% endhighlight %}

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_docs/howto/howto_ldap_and_sso.md
----------------------------------------------------------------------
diff --git a/website/_docs/howto/howto_ldap_and_sso.md b/website/_docs/howto/howto_ldap_and_sso.md
deleted file mode 100644
index 1102559..0000000
--- a/website/_docs/howto/howto_ldap_and_sso.md
+++ /dev/null
@@ -1,124 +0,0 @@
----
-layout: docs
-title:  How to Enable Security with LDAP and SSO
-categories: howto
-permalink: /docs/howto/howto_ldap_and_sso.html
-version: v2.0
-since: v1.0
----
-
-## Enable LDAP authentication
-
-Kylin supports LDAP authentication for enterprise or production deployment; This is implemented with Spring Security framework; Before enable LDAP, please contact your LDAP administrator to get necessary information, like LDAP server URL, username/password, search patterns;
-
-#### Configure LDAP server info
-
-Firstly, provide LDAP URL, and username/password if the LDAP server is secured; The password in kylin.properties need be salted; You can Google "Generate a BCrypt Password" or run org.apache.kylin.rest.security.PasswordPlaceholderConfigurer to get a hash of your password.
-
-```
-ldap.server=ldap://<your_ldap_host>:<port>
-ldap.username=<your_user_name>
-ldap.password=<your_password_hash>
-```
-
-Secondly, provide the user search patterns, this is by LDAP design, here is just a sample:
-
-```
-ldap.user.searchBase=OU=UserAccounts,DC=mycompany,DC=com
-ldap.user.searchPattern=(&(AccountName={0})(memberOf=CN=MYCOMPANY-USERS,DC=mycompany,DC=com))
-ldap.user.groupSearchBase=OU=Group,DC=mycompany,DC=com
-```
-
-If you have service accounts (e.g, for system integration) which also need be authenticated, configure them in ldap.service.*; Otherwise, leave them be empty;
-
-### Configure the administrator group and default role
-
-To map an LDAP group to the admin group in Kylin, need set the "acl.adminRole" to "ROLE_" + GROUP_NAME. For example, in LDAP the group "KYLIN-ADMIN-GROUP" is the list of administrators, here need set it as:
-
-```
-acl.adminRole=ROLE_KYLIN-ADMIN-GROUP
-acl.defaultRole=ROLE_ANALYST,ROLE_MODELER
-```
-
-The "acl.defaultRole" is a list of the default roles that grant to everyone, keep it as-is.
-
-#### Enable LDAP
-
-For Kylin v0.x and v1.x: set "kylin.sandbox=false" in conf/kylin.properties, then restart Kylin server; 
-For Kylin since v2.0: set "kylin.security.profile=ldap" in conf/kylin.properties, then restart Kylin server; 
-
-## Enable SSO authentication
-
-From v2.0, Kylin provides SSO with SAML. The implementation is based on Spring Security SAML Extension. You can read [this reference](http://docs.spring.io/autorepo/docs/spring-security-saml/1.0.x-SNAPSHOT/reference/htmlsingle/) to get an overall understand.
-
-Before trying this, you should have successfully enabled LDAP and managed users with it, as SSO server may only do authentication, Kylin need search LDAP to get the user's detail information.
-
-### Generate IDP metadata xml
-Contact your IDP (ID provider), asking to generate the SSO metadata file; Usually you need provide three piece of info:
-
-  1. Partner entity ID, which is an unique ID of your app, e.g,: https://host-name/kylin/saml/metadata 
-  2. App callback endpoint, to which the SAML assertion be posted, it need be: https://host-name/kylin/saml/SSO
-  3. Public certificate of Kylin server, the SSO server will encrypt the message with it.
-
-### Generate JKS keystore for Kylin
-As Kylin need send encrypted message (signed with Kylin's private key) to SSO server, a keystore (JKS) need be provided. There are a couple ways to generate the keystore, below is a sample.
-
-Assume kylin.crt is the public certificate file, kylin.key is the private certificate file; firstly create a PKCS#12 file with openssl, then convert it to JKS with keytool: 
-
-```
-$ openssl pkcs12 -export -in kylin.crt -inkey kylin.key -out kylin.p12
-Enter Export Password: <export_pwd>
-Verifying - Enter Export Password: <export_pwd>
-
-
-$ keytool -importkeystore -srckeystore kylin.p12 -srcstoretype PKCS12 -srcstorepass <export_pwd> -alias 1 -destkeystore samlKeystore.jks -destalias kylin -destkeypass changeit
-
-Enter destination keystore password:  changeit
-Re-enter new password: changeit
-```
-
-It will put the keys to "samlKeystore.jks" with alias "kylin";
-
-### Enable Higher Ciphers
-
-Make sure your environment is ready to handle higher level crypto keys, you may need to download Java Cryptography Extension (JCE) Unlimited Strength Jurisdiction Policy Files, copy local_policy.jar and US_export_policy.jar to $JAVA_HOME/jre/lib/security .
-
-### Deploy IDP xml file and keystore to Kylin
-
-The IDP metadata and keystore file need be deployed in Kylin web app's classpath in $KYLIN_HOME/tomcat/webapps/kylin/WEB-INF/classes 
-	
-  1. Name the IDP file to sso_metadata.xml and then copy to Kylin's classpath;
-  2. Name the keystore as "samlKeystore.jks" and then copy to Kylin's classpath;
-  3. If you use another alias or password, remember to update that kylinSecurity.xml accordingly:
-
-```
-<!-- Central storage of cryptographic keys -->
-<bean id="keyManager" class="org.springframework.security.saml.key.JKSKeyManager">
-	<constructor-arg value="classpath:samlKeystore.jks"/>
-	<constructor-arg type="java.lang.String" value="changeit"/>
-	<constructor-arg>
-		<map>
-			<entry key="kylin" value="changeit"/>
-		</map>
-	</constructor-arg>
-	<constructor-arg type="java.lang.String" value="kylin"/>
-</bean>
-
-```
-
-### Other configurations
-In conf/kylin.properties, add the following properties with your server information:
-
-```
-saml.metadata.entityBaseURL=https://host-name/kylin
-saml.context.scheme=https
-saml.context.serverName=host-name
-saml.context.serverPort=443
-saml.context.contextPath=/kylin
-```
-
-Please note, Kylin assume in the SAML message there is a "email" attribute representing the login user, and the name before @ will be used to search LDAP. 
-
-### Enable SSO
-Set "kylin.security.profile=saml" in conf/kylin.properties, then restart Kylin server; After that, type a URL like "/kylin" or "/kylin/cubes" will redirect to SSO for login, and jump back after be authorized. While login with LDAP is still available, you can type "/kylin/login" to use original way. The Rest API (/kylin/api/*) still use LDAP + basic authentication, no impact.
-

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_docs/howto/howto_optimize_cubes.md
----------------------------------------------------------------------
diff --git a/website/_docs/howto/howto_optimize_cubes.md b/website/_docs/howto/howto_optimize_cubes.md
deleted file mode 100644
index 4ebf7be..0000000
--- a/website/_docs/howto/howto_optimize_cubes.md
+++ /dev/null
@@ -1,214 +0,0 @@
----
-layout: docs
-title:  How to Optimize Cubes
-categories: howto
-permalink: /docs/howto/howto_optimize_cubes.html
-version: v0.7.2
-since: v0.7.1
----
-
-## Hierarchies:
-
-Theoretically for N dimensions you'll end up with 2^N dimension combinations. However for some group of dimensions there are no need to create so many combinations. For example, if you have three dimensions: continent, country, city (In hierarchies, the "bigger" dimension comes first). You will only need the following three combinations of group by when you do drill down analysis:
-
-group by continent
-group by continent, country
-group by continent, country, city
-
-In such cases the combination count is reduced from 2^3=8 to 3, which is a great optimization. The same goes for the YEAR,QUATER,MONTH,DATE case.
-
-If we Donate the hierarchy dimension as H1,H2,H3, typical scenarios would be:
-
-
-A. Hierarchies on lookup table
-
-
-<table>
-  <tr>
-    <td align="center">Fact table</td>
-    <td align="center">(joins)</td>
-    <td align="center">Lookup Table</td>
-  </tr>
-  <tr>
-    <td>column1,column2,,,,,, FK</td>
-    <td></td>
-    <td>PK,,H1,H2,H3,,,,</td>
-  </tr>
-</table>
-
----
-
-B. Hierarchies on fact table
-
-
-<table>
-  <tr>
-    <td align="center">Fact table</td>
-  </tr>
-  <tr>
-    <td>column1,column2,,,H1,H2,H3,,,,,,, </td>
-  </tr>
-</table>
-
----
-
-
-There is a special case for scenario A, where PK on the lookup table is accidentally being part of the hierarchies. For example we have a calendar lookup table where cal_dt is the primary key:
-
-A*. Hierarchies on lookup table over its primary key
-
-
-<table>
-  <tr>
-    <td align="center">Lookup Table(Calendar)</td>
-  </tr>
-  <tr>
-    <td>cal_dt(PK), week_beg_dt, month_beg_dt, quarter_beg_dt,,,</td>
-  </tr>
-</table>
-
----
-
-
-For cases like A* what you need is another optimization called "Derived Columns"
-
-## Derived Columns:
-
-Derived column is used when one or more dimensions (They must be dimension on lookup table, these columns are called "Derived") can be deduced from another(Usually it is the corresponding FK, this is called the "host column")
-
-For example, suppose we have a lookup table where we join fact table and it with "where DimA = DimX". Notice in Kylin, if you choose FK into a dimension, the corresponding PK will be automatically querable, without any extra cost. The secret is that since FK and PK are always identical, Kylin can apply filters/groupby on the FK first, and transparently replace them to PK.  This indicates that if we want the DimA(FK), DimX(PK), DimB, DimC in our cube, we can safely choose DimA,DimB,DimC only.
-
-<table>
-  <tr>
-    <td align="center">Fact table</td>
-    <td align="center">(joins)</td>
-    <td align="center">Lookup Table</td>
-  </tr>
-  <tr>
-    <td>column1,column2,,,,,, DimA(FK) </td>
-    <td></td>
-    <td>DimX(PK),,DimB, DimC</td>
-  </tr>
-</table>
-
----
-
-
-Let's say that DimA(the dimension representing FK/PK) has a special mapping to DimB:
-
-
-<table>
-  <tr>
-    <th>dimA</th>
-    <th>dimB</th>
-    <th>dimC</th>
-  </tr>
-  <tr>
-    <td>1</td>
-    <td>a</td>
-    <td>?</td>
-  </tr>
-  <tr>
-    <td>2</td>
-    <td>b</td>
-    <td>?</td>
-  </tr>
-  <tr>
-    <td>3</td>
-    <td>c</td>
-    <td>?</td>
-  </tr>
-  <tr>
-    <td>4</td>
-    <td>a</td>
-    <td>?</td>
-  </tr>
-</table>
-
-
-in this case, given a value in DimA, the value of DimB is determined, so we say dimB can be derived from DimA. When we build a cube that contains both DimA and DimB, we simple include DimA, and marking DimB as derived. Derived column(DimB) does not participant in cuboids generation:
-
-original combinations:
-ABC,AB,AC,BC,A,B,C
-
-combinations when driving B from A:
-AC,A,C
-
-at Runtime, in case queries like "select count(*) from fact_table inner join looup1 group by looup1 .dimB", it is expecting cuboid containing DimB to answer the query. However, DimB will appear in NONE of the cuboids due to derived optimization. In this case, we modify the execution plan to make it group by  DimA(its host column) first, we'll get intermediate answer like:
-
-
-<table>
-  <tr>
-    <th>DimA</th>
-    <th>count(*)</th>
-  </tr>
-  <tr>
-    <td>1</td>
-    <td>1</td>
-  </tr>
-  <tr>
-    <td>2</td>
-    <td>1</td>
-  </tr>
-  <tr>
-    <td>3</td>
-    <td>1</td>
-  </tr>
-  <tr>
-    <td>4</td>
-    <td>1</td>
-  </tr>
-</table>
-
-
-Afterwards, Kylin will replace DimA values with DimB values(since both of their values are in lookup table, Kylin can load the whole lookup table into memory and build a mapping for them), and the intermediate result becomes:
-
-
-<table>
-  <tr>
-    <th>DimB</th>
-    <th>count(*)</th>
-  </tr>
-  <tr>
-    <td>a</td>
-    <td>1</td>
-  </tr>
-  <tr>
-    <td>b</td>
-    <td>1</td>
-  </tr>
-  <tr>
-    <td>c</td>
-    <td>1</td>
-  </tr>
-  <tr>
-    <td>a</td>
-    <td>1</td>
-  </tr>
-</table>
-
-
-After this, the runtime SQL engine(calcite) will further aggregate the intermediate result to:
-
-
-<table>
-  <tr>
-    <th>DimB</th>
-    <th>count(*)</th>
-  </tr>
-  <tr>
-    <td>a</td>
-    <td>2</td>
-  </tr>
-  <tr>
-    <td>b</td>
-    <td>1</td>
-  </tr>
-  <tr>
-    <td>c</td>
-    <td>1</td>
-  </tr>
-</table>
-
-
-this step happens at query runtime, this is what it means "at the cost of extra runtime aggregation"

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_docs/howto/howto_upgrade.md
----------------------------------------------------------------------
diff --git a/website/_docs/howto/howto_upgrade.md b/website/_docs/howto/howto_upgrade.md
deleted file mode 100644
index 87367ca..0000000
--- a/website/_docs/howto/howto_upgrade.md
+++ /dev/null
@@ -1,103 +0,0 @@
----
-layout: docs
-title:  How to Upgrade
-categories: howto
-permalink: /docs/howto/howto_upgrade.html
-version: v1.2
-since: v0.7.1
----
-
-## Upgrade among v0.7.x and v1.x 
-
-From v0.7.1 to latest v1.2, Kylin's metadata is backward compatible, the upgrade can be finished in couple of minutes:
-
-#### 1. Backup metadata
-Backup the Kylin metadata peridically is a good practice, and is highly suggested before upgrade; 
-
-```
-cd $KYLIN_HOME
-./bin/metastore.sh backup
-``` 
-It will print the backup folder, take it down and make sure it will not be deleted before the upgrade finished. If there is no "metastore.sh", use HBase's snapshot command to do backup:
-
-```
-hbase shell
-snapshot 'kylin_metadata', 'kylin_metadata_backup20150610'
-```
-Here 'kylin_metadata' is the default kylin metadata table name, replace it with the right table name of your Kylin;
-
-#### 2. Install new Kylin and copy back "conf"
-Download the new Kylin binary package from Kylin's download page; Extract it to a different folder other than current KYLIN_HOME; Before copy back the "conf" folder, do a compare and merge between the old and new kylin.properties to ensure newly introduced property will be kept.
-
-#### 3. Stop old and start new Kylin instance
-```
-cd $KYLIN_HOME
-./bin/kylin.sh stop
-export KYLIN_HOME="<path_of_new_installation>"
-cd $KYLIN_HOME
-./bin/kylin.sh start
-```
-
-#### 4. Back-port if the upgrade is failed
-If the new version couldn't startup and need back-port, shutdown it and then switch to the old KYLIN_HOME to start. Idealy that would return to the origin state. If the metadata is broken, restore it from the backup folder.
-
-```
-./bin/metastore.sh restore <path_of_metadata_backup>
-```
-
-## Upgrade from v0.6.x to v0.7.x 
-
-In v0.7, Kylin refactored the metadata structure, for the new features like inverted-index and streaming; If you have cube created with v0.6 and want to keep in v0.7, a migration is needed; (Please skip v0.7.1 as
-it has several compatible issues and the fix will be included in v0.7.2) Below is the steps;
-
-#### 1. Backup v0.6 metadata
-To avoid data loss in the migration, a backup at the very beginning is always suggested; You can use HBase's backup or snapshot command to achieve this; Here is a sample with snapshot:
-
-```
-hbase shell
-snapshot 'kylin_metadata', 'kylin_metadata_backup20150610'
-```
-
-'kylin_metadata' is the default kylin metadata table name, replace it with the right table name of your Kylin;
-
-#### 2. Dump v0.6 metadata to local file
-This is also a backup method; As the migration tool is only tested with local file system, this step is must; All metadata need be downloaded, including snapshot, dictionary, etc;
-
-```
-hbase  org.apache.hadoop.util.RunJar  ${KYLIN_HOME}/lib/kylin-job-x.x.x-job.jar  org.apache.kylin.common.persistence.ResourceTool  download  ./meta_dump
-```
-
-(./meta_dump is the local folder that the metadata will be downloaded, change to name you preferred)
-
-#### 3. Run CubeMetadataUpgrade to migrate the metadata
-This step is to run the migration tool to parse the v0.6 metadata and then convert to v0.7 format; A verification will be performed in the last, and report error if some cube couldn't be migrated;
-
-```
-hbase org.apache.hadoop.util.RunJar  ${KYLIN_HOME}/lib/kylin-job-x.x.x-job.jar org.apache.kylin.job.CubeMetadataUpgrade ./meta_dump
-```
-
-1. The tool will not overwrite v0.6 metadata; It will create a new folder with "_v2" suffix in the same folder, in this case the "./meta_dump_v2" will be created;
-2. By default this tool will only migrate the job history in last 30 days; If you want to keep elder job history, please tweak upgradeJobInstance() method by your own;
-3. If you see _No error or warning messages; The migration is success_ , that's good; Otherwise please check the error/warning messages carefully;
-4. For some problem you may need manually update the JSON file, to check whether the problem is gone, you can run a verify against the new metadata:
-
-```
-hbase org.apache.hadoop.util.RunJar  ${KYLIN_HOME}/lib/kylin-job-x.x.x-job.jar org.apache.kylin.job.CubeMetadataUpgrade ./meta_dump2 verify
-```
-
-#### 4. Upload the new metadata to HBase
-Now the new format of metadata will be upload to the HBase to replace the old format; Stop Kylin, and then:
-
-```
-hbase org.apache.hadoop.util.RunJar  ${KYLIN_HOME}/lib/kylin-job-x.x.x-job.jar  org.apache.kylin.common.persistence.ResourceTool  reset
-hbase org.apache.hadoop.util.RunJar  ${KYLIN_HOME}/lib/kylin-job-x.x.x-job.jar  org.apache.kylin.common.persistence.ResourceTool  upload  ./meta_dump_v2
-```
-
-#### 5. Update HTables to use new coprocessor
-Kylin uses HBase coprocessor to do server side aggregation; When Kylin instance upgrades to V0.7, the HTables that created in V0.6 should also be updated to use the new coprocessor:
-
-```
-hbase org.apache.hadoop.util.RunJar  ${KYLIN_HOME}/lib/kylin-job-x.x.x-job.jar  org.apache.kylin.job.tools.DeployCoprocessorCLI ${KYLIN_HOME}/lib/kylin-coprocessor-x.x.x.jar
-```
-
-Done; Update your v0.7 Kylin configure to point to the same metadata HBase table, then start Kylin server; Check whether all cubes and other information are kept;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_docs/howto/howto_use_restapi.md
----------------------------------------------------------------------
diff --git a/website/_docs/howto/howto_use_restapi.md b/website/_docs/howto/howto_use_restapi.md
deleted file mode 100644
index a49a2db..0000000
--- a/website/_docs/howto/howto_use_restapi.md
+++ /dev/null
@@ -1,1006 +0,0 @@
----
-layout: docs
-title:  How to Use Restful API
-categories: howto
-permalink: /docs/howto/howto_use_restapi.html
-version: v1.2
-since: v0.7.1
----
-
-This page lists all the Rest APIs provided by Kylin; The base of the URL is `/kylin/api`, so don't forget to add it before a certain API's path. For example, to get all cube instances, send HTTP GET request to "/kylin/api/cubes".
-
-* Query
-   * [Authentication](#authentication)
-   * [Query](#query)
-   * [List queryable tables](#list-queryable-tables)
-* CUBE
-   * [List cubes](#list-cubes)
-   * [Get cube](#get-cube)
-   * [Get cube descriptor (dimension, measure info, etc)](#get-cube-descriptor)
-   * [Get data model (fact and lookup table info)](#get-data-model)
-   * [Build cube](#build-cube)
-   * [Disable cube](#disable-cube)
-   * [Purge cube](#purge-cube)
-   * [Enable cube](#enable-cube)
-* JOB
-   * [Resume job](#resume-job)
-   * [Discard job](#discard-job)
-   * [Get job step output](#get-job-step-output)
-* Metadata
-   * [Get Hive Table](#get-hive-table)
-   * [Get Hive Table (Extend Info)](#get-hive-table-extend-info)
-   * [Get Hive Tables](#get-hive-tables)
-   * [Load Hive Tables](#load-hive-tables)
-* Cache
-   * [Wipe cache](#wipe-cache)
-
-## Authentication
-`POST /user/authentication`
-
-#### Request Header
-Authorization data encoded by basic auth is needed in the header, such as:
-Authorization:Basic {data}
-
-#### Response Body
-* userDetails - Defined authorities and status of current user.
-
-#### Response Sample
-
-```sh
-{  
-   "userDetails":{  
-      "password":null,
-      "username":"sample",
-      "authorities":[  
-         {  
-            "authority":"ROLE_ANALYST"
-         },
-         {  
-            "authority":"ROLE_MODELER"
-         }
-      ],
-      "accountNonExpired":true,
-      "accountNonLocked":true,
-      "credentialsNonExpired":true,
-      "enabled":true
-   }
-}
-```
-
-Example with `curl`: 
-
-```
-curl -c /path/to/cookiefile.txt -X POST -H "Authorization: Basic XXXXXXXXX" -H 'Content-Type: application/json' http://<host>:<port>/kylin/api/user/authentication
-```
-
-If login successfully, the JSESSIONID will be saved into the cookie file; In the subsequent http requests, attach the cookie, for example:
-
-```
-curl -b /path/to/cookiefile.txt -X PUT -H 'Content-Type: application/json' -d '{"startTime":'1423526400000', "endTime":'1423526400', "buildType":"BUILD"}' http://<host>:<port>/kylin/api/cubes/your_cube/rebuild
-```
-
-***
-
-## Query
-`POST /query`
-
-#### Request Body
-* sql - `required` `string` The text of sql statement.
-* offset - `optional` `int` Query offset. If offset is set in sql, curIndex will be ignored.
-* limit - `optional` `int` Query limit. If limit is set in sql, perPage will be ignored.
-* acceptPartial - `optional` `bool` Whether accept a partial result or not, default be "false". Set to "false" for production use. 
-* project - `optional` `string` Project to perform query. Default value is 'DEFAULT'.
-
-#### Request Sample
-
-```sh
-{  
-   "sql":"select * from TEST_KYLIN_FACT",
-   "offset":0,
-   "limit":50000,
-   "acceptPartial":false,
-   "project":"DEFAULT"
-}
-```
-
-#### Response Body
-* columnMetas - Column metadata information of result set.
-* results - Data set of result.
-* cube - Cube used for this query.
-* affectedRowCount - Count of affected row by this sql statement.
-* isException - Whether this response is an exception.
-* ExceptionMessage - Message content of the exception.
-* Duration - Time cost of this query
-* Partial - Whether the response is a partial result or not. Decided by `acceptPartial` of request.
-
-#### Response Sample
-
-```sh
-{  
-   "columnMetas":[  
-      {  
-         "isNullable":1,
-         "displaySize":0,
-         "label":"CAL_DT",
-         "name":"CAL_DT",
-         "schemaName":null,
-         "catelogName":null,
-         "tableName":null,
-         "precision":0,
-         "scale":0,
-         "columnType":91,
-         "columnTypeName":"DATE",
-         "readOnly":true,
-         "writable":false,
-         "caseSensitive":true,
-         "searchable":false,
-         "currency":false,
-         "signed":true,
-         "autoIncrement":false,
-         "definitelyWritable":false
-      },
-      {  
-         "isNullable":1,
-         "displaySize":10,
-         "label":"LEAF_CATEG_ID",
-         "name":"LEAF_CATEG_ID",
-         "schemaName":null,
-         "catelogName":null,
-         "tableName":null,
-         "precision":10,
-         "scale":0,
-         "columnType":4,
-         "columnTypeName":"INTEGER",
-         "readOnly":true,
-         "writable":false,
-         "caseSensitive":true,
-         "searchable":false,
-         "currency":false,
-         "signed":true,
-         "autoIncrement":false,
-         "definitelyWritable":false
-      }
-   ],
-   "results":[  
-      [  
-         "2013-08-07",
-         "32996",
-         "15",
-         "15",
-         "Auction",
-         "10000000",
-         "49.048952730908745",
-         "49.048952730908745",
-         "49.048952730908745",
-         "1"
-      ],
-      [  
-         "2013-08-07",
-         "43398",
-         "0",
-         "14",
-         "ABIN",
-         "10000633",
-         "85.78317064220418",
-         "85.78317064220418",
-         "85.78317064220418",
-         "1"
-      ]
-   ],
-   "cube":"test_kylin_cube_with_slr_desc",
-   "affectedRowCount":0,
-   "isException":false,
-   "exceptionMessage":null,
-   "duration":3451,
-   "partial":false
-}
-```
-
-## List queryable tables
-`GET /tables_and_columns`
-
-#### Request Parameters
-* project - `required` `string` The project to load tables
-
-#### Response Sample
-```sh
-[  
-   {  
-      "columns":[  
-         {  
-            "table_NAME":"TEST_CAL_DT",
-            "table_SCHEM":"EDW",
-            "column_NAME":"CAL_DT",
-            "data_TYPE":91,
-            "nullable":1,
-            "column_SIZE":-1,
-            "buffer_LENGTH":-1,
-            "decimal_DIGITS":0,
-            "num_PREC_RADIX":10,
-            "column_DEF":null,
-            "sql_DATA_TYPE":-1,
-            "sql_DATETIME_SUB":-1,
-            "char_OCTET_LENGTH":-1,
-            "ordinal_POSITION":1,
-            "is_NULLABLE":"YES",
-            "scope_CATLOG":null,
-            "scope_SCHEMA":null,
-            "scope_TABLE":null,
-            "source_DATA_TYPE":-1,
-            "iS_AUTOINCREMENT":null,
-            "table_CAT":"defaultCatalog",
-            "remarks":null,
-            "type_NAME":"DATE"
-         },
-         {  
-            "table_NAME":"TEST_CAL_DT",
-            "table_SCHEM":"EDW",
-            "column_NAME":"WEEK_BEG_DT",
-            "data_TYPE":91,
-            "nullable":1,
-            "column_SIZE":-1,
-            "buffer_LENGTH":-1,
-            "decimal_DIGITS":0,
-            "num_PREC_RADIX":10,
-            "column_DEF":null,
-            "sql_DATA_TYPE":-1,
-            "sql_DATETIME_SUB":-1,
-            "char_OCTET_LENGTH":-1,
-            "ordinal_POSITION":2,
-            "is_NULLABLE":"YES",
-            "scope_CATLOG":null,
-            "scope_SCHEMA":null,
-            "scope_TABLE":null,
-            "source_DATA_TYPE":-1,
-            "iS_AUTOINCREMENT":null,
-            "table_CAT":"defaultCatalog",
-            "remarks":null,
-            "type_NAME":"DATE"
-         }
-      ],
-      "table_NAME":"TEST_CAL_DT",
-      "table_SCHEM":"EDW",
-      "ref_GENERATION":null,
-      "self_REFERENCING_COL_NAME":null,
-      "type_SCHEM":null,
-      "table_TYPE":"TABLE",
-      "table_CAT":"defaultCatalog",
-      "remarks":null,
-      "type_CAT":null,
-      "type_NAME":null
-   }
-]
-```
-
-***
-
-## List cubes
-`GET /cubes`
-
-#### Request Parameters
-* offset - `required` `int` Offset used by pagination
-* limit - `required` `int ` Cubes per page.
-* cubeName - `optional` `string` Keyword for cube names. To find cubes whose name contains this keyword.
-* projectName - `optional` `string` Project name.
-
-#### Response Sample
-```sh
-[  
-   {  
-      "uuid":"1eaca32a-a33e-4b69-83dd-0bb8b1f8c53b",
-      "last_modified":1407831634847,
-      "name":"test_kylin_cube_with_slr_empty",
-      "owner":null,
-      "version":null,
-      "descriptor":"test_kylin_cube_with_slr_desc",
-      "cost":50,
-      "status":"DISABLED",
-      "segments":[  
-      ],
-      "create_time":null,
-      "source_records_count":0,
-      "source_records_size":0,
-      "size_kb":0
-   }
-]
-```
-
-## Get cube
-`GET /cubes/{cubeName}`
-
-#### Path Variable
-* cubeName - `required` `string` Cube name to find.
-
-## Get cube descriptor
-`GET /cube_desc/{cubeName}`
-Get descriptor for specified cube instance.
-
-#### Path Variable
-* cubeName - `required` `string` Cube name.
-
-#### Response Sample
-```sh
-[
-    {
-        "uuid": "a24ca905-1fc6-4f67-985c-38fa5aeafd92", 
-        "name": "test_kylin_cube_with_slr_desc", 
-        "description": null, 
-        "dimensions": [
-            {
-                "id": 0, 
-                "name": "CAL_DT", 
-                "table": "EDW.TEST_CAL_DT", 
-                "column": null, 
-                "derived": [
-                    "WEEK_BEG_DT"
-                ], 
-                "hierarchy": false
-            }, 
-            {
-                "id": 1, 
-                "name": "CATEGORY", 
-                "table": "DEFAULT.TEST_CATEGORY_GROUPINGS", 
-                "column": null, 
-                "derived": [
-                    "USER_DEFINED_FIELD1", 
-                    "USER_DEFINED_FIELD3", 
-                    "UPD_DATE", 
-                    "UPD_USER"
-                ], 
-                "hierarchy": false
-            }, 
-            {
-                "id": 2, 
-                "name": "CATEGORY_HIERARCHY", 
-                "table": "DEFAULT.TEST_CATEGORY_GROUPINGS", 
-                "column": [
-                    "META_CATEG_NAME", 
-                    "CATEG_LVL2_NAME", 
-                    "CATEG_LVL3_NAME"
-                ], 
-                "derived": null, 
-                "hierarchy": true
-            }, 
-            {
-                "id": 3, 
-                "name": "LSTG_FORMAT_NAME", 
-                "table": "DEFAULT.TEST_KYLIN_FACT", 
-                "column": [
-                    "LSTG_FORMAT_NAME"
-                ], 
-                "derived": null, 
-                "hierarchy": false
-            }, 
-            {
-                "id": 4, 
-                "name": "SITE_ID", 
-                "table": "EDW.TEST_SITES", 
-                "column": null, 
-                "derived": [
-                    "SITE_NAME", 
-                    "CRE_USER"
-                ], 
-                "hierarchy": false
-            }, 
-            {
-                "id": 5, 
-                "name": "SELLER_TYPE_CD", 
-                "table": "EDW.TEST_SELLER_TYPE_DIM", 
-                "column": null, 
-                "derived": [
-                    "SELLER_TYPE_DESC"
-                ], 
-                "hierarchy": false
-            }, 
-            {
-                "id": 6, 
-                "name": "SELLER_ID", 
-                "table": "DEFAULT.TEST_KYLIN_FACT", 
-                "column": [
-                    "SELLER_ID"
-                ], 
-                "derived": null, 
-                "hierarchy": false
-            }
-        ], 
-        "measures": [
-            {
-                "id": 1, 
-                "name": "GMV_SUM", 
-                "function": {
-                    "expression": "SUM", 
-                    "parameter": {
-                        "type": "column", 
-                        "value": "PRICE", 
-                        "next_parameter": null
-                    }, 
-                    "returntype": "decimal(19,4)"
-                }, 
-                "dependent_measure_ref": null
-            }, 
-            {
-                "id": 2, 
-                "name": "GMV_MIN", 
-                "function": {
-                    "expression": "MIN", 
-                    "parameter": {
-                        "type": "column", 
-                        "value": "PRICE", 
-                        "next_parameter": null
-                    }, 
-                    "returntype": "decimal(19,4)"
-                }, 
-                "dependent_measure_ref": null
-            }, 
-            {
-                "id": 3, 
-                "name": "GMV_MAX", 
-                "function": {
-                    "expression": "MAX", 
-                    "parameter": {
-                        "type": "column", 
-                        "value": "PRICE", 
-                        "next_parameter": null
-                    }, 
-                    "returntype": "decimal(19,4)"
-                }, 
-                "dependent_measure_ref": null
-            }, 
-            {
-                "id": 4, 
-                "name": "TRANS_CNT", 
-                "function": {
-                    "expression": "COUNT", 
-                    "parameter": {
-                        "type": "constant", 
-                        "value": "1", 
-                        "next_parameter": null
-                    }, 
-                    "returntype": "bigint"
-                }, 
-                "dependent_measure_ref": null
-            }, 
-            {
-                "id": 5, 
-                "name": "ITEM_COUNT_SUM", 
-                "function": {
-                    "expression": "SUM", 
-                    "parameter": {
-                        "type": "column", 
-                        "value": "ITEM_COUNT", 
-                        "next_parameter": null
-                    }, 
-                    "returntype": "bigint"
-                }, 
-                "dependent_measure_ref": null
-            }
-        ], 
-        "rowkey": {
-            "rowkey_columns": [
-                {
-                    "column": "SELLER_ID", 
-                    "length": 18, 
-                    "dictionary": null, 
-                    "mandatory": true
-                }, 
-                {
-                    "column": "CAL_DT", 
-                    "length": 0, 
-                    "dictionary": "true", 
-                    "mandatory": false
-                }, 
-                {
-                    "column": "LEAF_CATEG_ID", 
-                    "length": 0, 
-                    "dictionary": "true", 
-                    "mandatory": false
-                }, 
-                {
-                    "column": "META_CATEG_NAME", 
-                    "length": 0, 
-                    "dictionary": "true", 
-                    "mandatory": false
-                }, 
-                {
-                    "column": "CATEG_LVL2_NAME", 
-                    "length": 0, 
-                    "dictionary": "true", 
-                    "mandatory": false
-                }, 
-                {
-                    "column": "CATEG_LVL3_NAME", 
-                    "length": 0, 
-                    "dictionary": "true", 
-                    "mandatory": false
-                }, 
-                {
-                    "column": "LSTG_FORMAT_NAME", 
-                    "length": 12, 
-                    "dictionary": null, 
-                    "mandatory": false
-                }, 
-                {
-                    "column": "LSTG_SITE_ID", 
-                    "length": 0, 
-                    "dictionary": "true", 
-                    "mandatory": false
-                }, 
-                {
-                    "column": "SLR_SEGMENT_CD", 
-                    "length": 0, 
-                    "dictionary": "true", 
-                    "mandatory": false
-                }
-            ], 
-            "aggregation_groups": [
-                [
-                    "LEAF_CATEG_ID", 
-                    "META_CATEG_NAME", 
-                    "CATEG_LVL2_NAME", 
-                    "CATEG_LVL3_NAME", 
-                    "CAL_DT"
-                ]
-            ]
-        }, 
-        "signature": "lsLAl2jL62ZApmOLZqWU3g==", 
-        "last_modified": 1445850327000, 
-        "model_name": "test_kylin_with_slr_model_desc", 
-        "null_string": null, 
-        "hbase_mapping": {
-            "column_family": [
-                {
-                    "name": "F1", 
-                    "columns": [
-                        {
-                            "qualifier": "M", 
-                            "measure_refs": [
-                                "GMV_SUM", 
-                                "GMV_MIN", 
-                                "GMV_MAX", 
-                                "TRANS_CNT", 
-                                "ITEM_COUNT_SUM"
-                            ]
-                        }
-                    ]
-                }
-            ]
-        }, 
-        "notify_list": null, 
-        "auto_merge_time_ranges": null, 
-        "retention_range": 0
-    }
-]
-```
-
-## Get data model
-`GET /model/{modelName}`
-
-#### Path Variable
-* modelName - `required` `string` Data model name, by default it should be the same with cube name.
-
-#### Response Sample
-```sh
-{
-    "uuid": "ff527b94-f860-44c3-8452-93b17774c647", 
-    "name": "test_kylin_with_slr_model_desc", 
-    "lookups": [
-        {
-            "table": "EDW.TEST_CAL_DT", 
-            "join": {
-                "type": "inner", 
-                "primary_key": [
-                    "CAL_DT"
-                ], 
-                "foreign_key": [
-                    "CAL_DT"
-                ]
-            }
-        }, 
-        {
-            "table": "DEFAULT.TEST_CATEGORY_GROUPINGS", 
-            "join": {
-                "type": "inner", 
-                "primary_key": [
-                    "LEAF_CATEG_ID", 
-                    "SITE_ID"
-                ], 
-                "foreign_key": [
-                    "LEAF_CATEG_ID", 
-                    "LSTG_SITE_ID"
-                ]
-            }
-        }
-    ], 
-    "capacity": "MEDIUM", 
-    "last_modified": 1442372116000, 
-    "fact_table": "DEFAULT.TEST_KYLIN_FACT", 
-    "filter_condition": null, 
-    "partition_desc": {
-        "partition_date_column": "DEFAULT.TEST_KYLIN_FACT.CAL_DT", 
-        "partition_date_start": 0, 
-        "partition_date_format": "yyyy-MM-dd", 
-        "partition_type": "APPEND", 
-        "partition_condition_builder": "org.apache.kylin.metadata.model.PartitionDesc$DefaultPartitionConditionBuilder"
-    }
-}
-```
-
-## Build cube
-`PUT /cubes/{cubeName}/rebuild`
-
-#### Path Variable
-* cubeName - `required` `string` Cube name.
-
-#### Request Body
-* startTime - `required` `long` Start timestamp of data to build, e.g. 1388563200000 for 2014-1-1
-* endTime - `required` `long` End timestamp of data to build
-* buildType - `required` `string` Supported build type: 'BUILD', 'MERGE', 'REFRESH'
-
-#### Response Sample
-```
-{  
-   "uuid":"c143e0e4-ac5f-434d-acf3-46b0d15e3dc6",
-   "last_modified":1407908916705,
-   "name":"test_kylin_cube_with_slr_empty - 19700101000000_20140731160000 - BUILD - PDT 2014-08-12 22:48:36",
-   "type":"BUILD",
-   "duration":0,
-   "related_cube":"test_kylin_cube_with_slr_empty",
-   "related_segment":"19700101000000_20140731160000",
-   "exec_start_time":0,
-   "exec_end_time":0,
-   "mr_waiting":0,
-   "steps":[  
-      {  
-         "interruptCmd":null,
-         "name":"Create Intermediate Flat Hive Table",
-         "sequence_id":0,
-         "exec_cmd":"hive -e \"DROP TABLE IF EXISTS kylin_intermediate_test_kylin_cube_with_slr_desc_19700101000000_20140731160000_c143e0e4_ac5f_434d_acf3_46b0d15e3dc6;\nCREATE EXTERNAL TABLE IF NOT EXISTS kylin_intermediate_test_kylin_cube_with_slr_desc_19700101000000_20140731160000_c143e0e4_ac5f_434d_acf3_46b0d15e3dc6\n(\nCAL_DT date\n,LEAF_CATEG_ID int\n,LSTG_SITE_ID int\n,META_CATEG_NAME string\n,CATEG_LVL2_NAME string\n,CATEG_LVL3_NAME string\n,LSTG_FORMAT_NAME string\n,SLR_SEGMENT_CD smallint\n,SELLER_ID bigint\n,PRICE decimal\n)\nROW FORMAT DELIMITED FIELDS TERMINATED BY '\\177'\nSTORED AS SEQUENCEFILE\nLOCATION '/tmp/kylin-c143e0e4-ac5f-434d-acf3-46b0d15e3dc6/kylin_intermediate_test_kylin_cube_with_slr_desc_19700101000000_20140731160000_c143e0e4_ac5f_434d_acf3_46b0d15e3dc6';\nSET mapreduce.job.split.metainfo.maxsize=-1;\nSET mapred.compress.map.output=true;\nSET mapred.map.output.compression.codec=com.hadoop.compression.lzo.LzoCodec;\nSET mapred.output.compress=true;\nSET ma
 pred.output.compression.codec=com.hadoop.compression.lzo.LzoCodec;\nSET mapred.output.compression.type=BLOCK;\nSET mapreduce.job.max.split.locations=2000;\nSET hive.exec.compress.output=true;\nSET hive.auto.convert.join.noconditionaltask = true;\nSET hive.auto.convert.join.noconditionaltask.size = 300000000;\nINSERT OVERWRITE TABLE kylin_intermediate_test_kylin_cube_with_slr_desc_19700101000000_20140731160000_c143e0e4_ac5f_434d_acf3_46b0d15e3dc6\nSELECT\nTEST_KYLIN_FACT.CAL_DT\n,TEST_KYLIN_FACT.LEAF_CATEG_ID\n,TEST_KYLIN_FACT.LSTG_SITE_ID\n,TEST_CATEGORY_GROUPINGS.META_CATEG_NAME\n,TEST_CATEGORY_GROUPINGS.CATEG_LVL2_NAME\n,TEST_CATEGORY_GROUPINGS.CATEG_LVL3_NAME\n,TEST_KYLIN_FACT.LSTG_FORMAT_NAME\n,TEST_KYLIN_FACT.SLR_SEGMENT_CD\n,TEST_KYLIN_FACT.SELLER_ID\n,TEST_KYLIN_FACT.PRICE\nFROM TEST_KYLIN_FACT\nINNER JOIN TEST_CAL_DT\nON TEST_KYLIN_FACT.CAL_DT = TEST_CAL_DT.CAL_DT\nINNER JOIN TEST_CATEGORY_GROUPINGS\nON TEST_KYLIN_FACT.LEAF_CATEG_ID = TEST_CATEGORY_GROUPINGS.LEAF_CATEG_ID AN
 D TEST_KYLIN_FACT.LSTG_SITE_ID = TEST_CATEGORY_GROUPINGS.SITE_ID\nINNER JOIN TEST_SITES\nON TEST_KYLIN_FACT.LSTG_SITE_ID = TEST_SITES.SITE_ID\nINNER JOIN TEST_SELLER_TYPE_DIM\nON TEST_KYLIN_FACT.SLR_SEGMENT_CD = TEST_SELLER_TYPE_DIM.SELLER_TYPE_CD\nWHERE (test_kylin_fact.cal_dt < '2014-07-31 16:00:00')\n;\n\"",
-         "interrupt_cmd":null,
-         "exec_start_time":0,
-         "exec_end_time":0,
-         "exec_wait_time":0,
-         "step_status":"PENDING",
-         "cmd_type":"SHELL_CMD_HADOOP",
-         "info":null,
-         "run_async":false
-      },
-      {  
-         "interruptCmd":null,
-         "name":"Extract Fact Table Distinct Columns",
-         "sequence_id":1,
-         "exec_cmd":" -conf C:/kylin/Kylin/server/src/main/resources/hadoop_job_conf_medium.xml -cubename test_kylin_cube_with_slr_empty -input /tmp/kylin-c143e0e4-ac5f-434d-acf3-46b0d15e3dc6/kylin_intermediate_test_kylin_cube_with_slr_desc_19700101000000_20140731160000_c143e0e4_ac5f_434d_acf3_46b0d15e3dc6 -output /tmp/kylin-c143e0e4-ac5f-434d-acf3-46b0d15e3dc6/test_kylin_cube_with_slr_empty/fact_distinct_columns -jobname Kylin_Fact_Distinct_Columns_test_kylin_cube_with_slr_empty_Step_1",
-         "interrupt_cmd":null,
-         "exec_start_time":0,
-         "exec_end_time":0,
-         "exec_wait_time":0,
-         "step_status":"PENDING",
-         "cmd_type":"JAVA_CMD_HADOOP_FACTDISTINCT",
-         "info":null,
-         "run_async":true
-      },
-      {  
-         "interruptCmd":null,
-         "name":"Load HFile to HBase Table",
-         "sequence_id":12,
-         "exec_cmd":" -input /tmp/kylin-c143e0e4-ac5f-434d-acf3-46b0d15e3dc6/test_kylin_cube_with_slr_empty/hfile/ -htablename KYLIN-CUBE-TEST_KYLIN_CUBE_WITH_SLR_EMPTY-19700101000000_20140731160000_11BB4326-5975-4358-804C-70D53642E03A -cubename test_kylin_cube_with_slr_empty",
-         "interrupt_cmd":null,
-         "exec_start_time":0,
-         "exec_end_time":0,
-         "exec_wait_time":0,
-         "step_status":"PENDING",
-         "cmd_type":"JAVA_CMD_HADOOP_NO_MR_BULKLOAD",
-         "info":null,
-         "run_async":false
-      }
-   ],
-   "job_status":"PENDING",
-   "progress":0.0
-}
-```
-
-## Enable Cube
-`PUT /cubes/{cubeName}/enable`
-
-#### Path variable
-* cubeName - `required` `string` Cube name.
-
-#### Response Sample
-```sh
-{  
-   "uuid":"1eaca32a-a33e-4b69-83dd-0bb8b1f8c53b",
-   "last_modified":1407909046305,
-   "name":"test_kylin_cube_with_slr_ready",
-   "owner":null,
-   "version":null,
-   "descriptor":"test_kylin_cube_with_slr_desc",
-   "cost":50,
-   "status":"ACTIVE",
-   "segments":[  
-      {  
-         "name":"19700101000000_20140531160000",
-         "storage_location_identifier":"KYLIN-CUBE-TEST_KYLIN_CUBE_WITH_SLR_READY-19700101000000_20140531160000_BF043D2D-9A4A-45E9-AA59-5A17D3F34A50",
-         "date_range_start":0,
-         "date_range_end":1401552000000,
-         "status":"READY",
-         "size_kb":4758,
-         "source_records":6000,
-         "source_records_size":620356,
-         "last_build_time":1407832663227,
-         "last_build_job_id":"2c7a2b63-b052-4a51-8b09-0c24b5792cda",
-         "binary_signature":null,
-         "dictionaries":{  
-            "TEST_CATEGORY_GROUPINGS/CATEG_LVL2_NAME":"/dict/TEST_CATEGORY_GROUPINGS/CATEG_LVL2_NAME/16d8185c-ee6b-4f8c-a919-756d9809f937.dict",
-            "TEST_KYLIN_FACT/LSTG_SITE_ID":"/dict/TEST_SITES/SITE_ID/0bec6bb3-1b0d-469c-8289-b8c4ca5d5001.dict",
-            "TEST_KYLIN_FACT/SLR_SEGMENT_CD":"/dict/TEST_SELLER_TYPE_DIM/SELLER_TYPE_CD/0c5d77ec-316b-47e0-ba9a-0616be890ad6.dict",
-            "TEST_KYLIN_FACT/CAL_DT":"/dict/PREDEFINED/date(yyyy-mm-dd)/64ac4f82-f2af-476e-85b9-f0805001014e.dict",
-            "TEST_CATEGORY_GROUPINGS/CATEG_LVL3_NAME":"/dict/TEST_CATEGORY_GROUPINGS/CATEG_LVL3_NAME/270fbfb0-281c-4602-8413-2970a7439c47.dict",
-            "TEST_KYLIN_FACT/LEAF_CATEG_ID":"/dict/TEST_CATEGORY_GROUPINGS/LEAF_CATEG_ID/2602386c-debb-4968-8d2f-b52b8215e385.dict",
-            "TEST_CATEGORY_GROUPINGS/META_CATEG_NAME":"/dict/TEST_CATEGORY_GROUPINGS/META_CATEG_NAME/0410d2c4-4686-40bc-ba14-170042a2de94.dict"
-         },
-         "snapshots":{  
-            "TEST_CAL_DT":"/table_snapshot/TEST_CAL_DT.csv/8f7cfc8a-020d-4019-b419-3c6deb0ffaa0.snapshot",
-            "TEST_SELLER_TYPE_DIM":"/table_snapshot/TEST_SELLER_TYPE_DIM.csv/c60fd05e-ac94-4016-9255-96521b273b81.snapshot",
-            "TEST_CATEGORY_GROUPINGS":"/table_snapshot/TEST_CATEGORY_GROUPINGS.csv/363f4a59-b725-4459-826d-3188bde6a971.snapshot",
-            "TEST_SITES":"/table_snapshot/TEST_SITES.csv/78e0aecc-3ec6-4406-b86e-bac4b10ea63b.snapshot"
-         }
-      }
-   ],
-   "create_time":null,
-   "source_records_count":6000,
-   "source_records_size":0,
-   "size_kb":4758
-}
-```
-
-## Disable Cube
-`PUT /cubes/{cubeName}/disable`
-
-#### Path variable
-* cubeName - `required` `string` Cube name.
-
-#### Response Sample
-(Same as "Enable Cube")
-
-## Purge Cube
-`PUT /cubes/{cubeName}/purge`
-
-#### Path variable
-* cubeName - `required` `string` Cube name.
-
-#### Response Sample
-(Same as "Enable Cube")
-
-***
-
-## Resume Job
-`PUT /jobs/{jobId}/resume`
-
-#### Path variable
-* jobId - `required` `string` Job id.
-
-#### Response Sample
-```
-{  
-   "uuid":"c143e0e4-ac5f-434d-acf3-46b0d15e3dc6",
-   "last_modified":1407908916705,
-   "name":"test_kylin_cube_with_slr_empty - 19700101000000_20140731160000 - BUILD - PDT 2014-08-12 22:48:36",
-   "type":"BUILD",
-   "duration":0,
-   "related_cube":"test_kylin_cube_with_slr_empty",
-   "related_segment":"19700101000000_20140731160000",
-   "exec_start_time":0,
-   "exec_end_time":0,
-   "mr_waiting":0,
-   "steps":[  
-      {  
-         "interruptCmd":null,
-         "name":"Create Intermediate Flat Hive Table",
-         "sequence_id":0,
-         "exec_cmd":"hive -e \"DROP TABLE IF EXISTS kylin_intermediate_test_kylin_cube_with_slr_desc_19700101000000_20140731160000_c143e0e4_ac5f_434d_acf3_46b0d15e3dc6;\nCREATE EXTERNAL TABLE IF NOT EXISTS kylin_intermediate_test_kylin_cube_with_slr_desc_19700101000000_20140731160000_c143e0e4_ac5f_434d_acf3_46b0d15e3dc6\n(\nCAL_DT date\n,LEAF_CATEG_ID int\n,LSTG_SITE_ID int\n,META_CATEG_NAME string\n,CATEG_LVL2_NAME string\n,CATEG_LVL3_NAME string\n,LSTG_FORMAT_NAME string\n,SLR_SEGMENT_CD smallint\n,SELLER_ID bigint\n,PRICE decimal\n)\nROW FORMAT DELIMITED FIELDS TERMINATED BY '\\177'\nSTORED AS SEQUENCEFILE\nLOCATION '/tmp/kylin-c143e0e4-ac5f-434d-acf3-46b0d15e3dc6/kylin_intermediate_test_kylin_cube_with_slr_desc_19700101000000_20140731160000_c143e0e4_ac5f_434d_acf3_46b0d15e3dc6';\nSET mapreduce.job.split.metainfo.maxsize=-1;\nSET mapred.compress.map.output=true;\nSET mapred.map.output.compression.codec=com.hadoop.compression.lzo.LzoCodec;\nSET mapred.output.compress=true;\nSET ma
 pred.output.compression.codec=com.hadoop.compression.lzo.LzoCodec;\nSET mapred.output.compression.type=BLOCK;\nSET mapreduce.job.max.split.locations=2000;\nSET hive.exec.compress.output=true;\nSET hive.auto.convert.join.noconditionaltask = true;\nSET hive.auto.convert.join.noconditionaltask.size = 300000000;\nINSERT OVERWRITE TABLE kylin_intermediate_test_kylin_cube_with_slr_desc_19700101000000_20140731160000_c143e0e4_ac5f_434d_acf3_46b0d15e3dc6\nSELECT\nTEST_KYLIN_FACT.CAL_DT\n,TEST_KYLIN_FACT.LEAF_CATEG_ID\n,TEST_KYLIN_FACT.LSTG_SITE_ID\n,TEST_CATEGORY_GROUPINGS.META_CATEG_NAME\n,TEST_CATEGORY_GROUPINGS.CATEG_LVL2_NAME\n,TEST_CATEGORY_GROUPINGS.CATEG_LVL3_NAME\n,TEST_KYLIN_FACT.LSTG_FORMAT_NAME\n,TEST_KYLIN_FACT.SLR_SEGMENT_CD\n,TEST_KYLIN_FACT.SELLER_ID\n,TEST_KYLIN_FACT.PRICE\nFROM TEST_KYLIN_FACT\nINNER JOIN TEST_CAL_DT\nON TEST_KYLIN_FACT.CAL_DT = TEST_CAL_DT.CAL_DT\nINNER JOIN TEST_CATEGORY_GROUPINGS\nON TEST_KYLIN_FACT.LEAF_CATEG_ID = TEST_CATEGORY_GROUPINGS.LEAF_CATEG_ID AN
 D TEST_KYLIN_FACT.LSTG_SITE_ID = TEST_CATEGORY_GROUPINGS.SITE_ID\nINNER JOIN TEST_SITES\nON TEST_KYLIN_FACT.LSTG_SITE_ID = TEST_SITES.SITE_ID\nINNER JOIN TEST_SELLER_TYPE_DIM\nON TEST_KYLIN_FACT.SLR_SEGMENT_CD = TEST_SELLER_TYPE_DIM.SELLER_TYPE_CD\nWHERE (test_kylin_fact.cal_dt < '2014-07-31 16:00:00')\n;\n\"",
-         "interrupt_cmd":null,
-         "exec_start_time":0,
-         "exec_end_time":0,
-         "exec_wait_time":0,
-         "step_status":"PENDING",
-         "cmd_type":"SHELL_CMD_HADOOP",
-         "info":null,
-         "run_async":false
-      },
-      {  
-         "interruptCmd":null,
-         "name":"Extract Fact Table Distinct Columns",
-         "sequence_id":1,
-         "exec_cmd":" -conf C:/kylin/Kylin/server/src/main/resources/hadoop_job_conf_medium.xml -cubename test_kylin_cube_with_slr_empty -input /tmp/kylin-c143e0e4-ac5f-434d-acf3-46b0d15e3dc6/kylin_intermediate_test_kylin_cube_with_slr_desc_19700101000000_20140731160000_c143e0e4_ac5f_434d_acf3_46b0d15e3dc6 -output /tmp/kylin-c143e0e4-ac5f-434d-acf3-46b0d15e3dc6/test_kylin_cube_with_slr_empty/fact_distinct_columns -jobname Kylin_Fact_Distinct_Columns_test_kylin_cube_with_slr_empty_Step_1",
-         "interrupt_cmd":null,
-         "exec_start_time":0,
-         "exec_end_time":0,
-         "exec_wait_time":0,
-         "step_status":"PENDING",
-         "cmd_type":"JAVA_CMD_HADOOP_FACTDISTINCT",
-         "info":null,
-         "run_async":true
-      },
-      {  
-         "interruptCmd":null,
-         "name":"Load HFile to HBase Table",
-         "sequence_id":12,
-         "exec_cmd":" -input /tmp/kylin-c143e0e4-ac5f-434d-acf3-46b0d15e3dc6/test_kylin_cube_with_slr_empty/hfile/ -htablename KYLIN-CUBE-TEST_KYLIN_CUBE_WITH_SLR_EMPTY-19700101000000_20140731160000_11BB4326-5975-4358-804C-70D53642E03A -cubename test_kylin_cube_with_slr_empty",
-         "interrupt_cmd":null,
-         "exec_start_time":0,
-         "exec_end_time":0,
-         "exec_wait_time":0,
-         "step_status":"PENDING",
-         "cmd_type":"JAVA_CMD_HADOOP_NO_MR_BULKLOAD",
-         "info":null,
-         "run_async":false
-      }
-   ],
-   "job_status":"PENDING",
-   "progress":0.0
-}
-```
-
-## Discard Job
-`PUT /jobs/{jobId}/cancel`
-
-#### Path variable
-* jobId - `required` `string` Job id.
-
-#### Response Sample
-(Same as "Resume job")
-
-## Get job step output
-`GET /{jobId}/steps/{stepId}/output`
-
-#### Path Variable
-* jobId - `required` `string` Job id.
-* stepId - `required` `string` Step id; the step id is composed by jobId with step sequence id; for example, the jobId is "fb479e54-837f-49a2-b457-651fc50be110", its 3rd step id is "fb479e54-837f-49a2-b457-651fc50be110-3", 
-
-#### Response Sample
-```
-{  
-   "cmd_output":"log string"
-}
-```
-
-***
-
-## Get Hive Table
-`GET /tables/{tableName}`
-
-#### Request Parameters
-* tableName - `required` `string` table name to find.
-
-#### Response Sample
-```sh
-{
-    uuid: "69cc92c0-fc42-4bb9-893f-bd1141c91dbe",
-    name: "SAMPLE_07",
-    columns: [{
-        id: "1",
-        name: "CODE",
-        datatype: "string"
-    }, {
-        id: "2",
-        name: "DESCRIPTION",
-        datatype: "string"
-    }, {
-        id: "3",
-        name: "TOTAL_EMP",
-        datatype: "int"
-    }, {
-        id: "4",
-        name: "SALARY",
-        datatype: "int"
-    }],
-    database: "DEFAULT",
-    last_modified: 1419330476755
-}
-```
-
-## Get Hive Table (Extend Info)
-`GET /tables/{tableName}/exd-map`
-
-#### Request Parameters
-* tableName - `optional` `string` table name to find.
-
-#### Response Sample
-```
-{
-    "minFileSize": "46055",
-    "totalNumberFiles": "1",
-    "location": "hdfs://sandbox.hortonworks.com:8020/apps/hive/warehouse/sample_07",
-    "lastAccessTime": "1418374103365",
-    "lastUpdateTime": "1398176493340",
-    "columns": "struct columns { string code, string description, i32 total_emp, i32 salary}",
-    "partitionColumns": "",
-    "EXD_STATUS": "true",
-    "maxFileSize": "46055",
-    "inputformat": "org.apache.hadoop.mapred.TextInputFormat",
-    "partitioned": "false",
-    "tableName": "sample_07",
-    "owner": "hue",
-    "totalFileSize": "46055",
-    "outputformat": "org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"
-}
-```
-
-## Get Hive Tables
-`GET /tables`
-
-#### Request Parameters
-* project- `required` `string` will list all tables in the project.
-* ext- `optional` `boolean`  set true to get extend info of table.
-
-#### Response Sample
-```sh
-[
- {
-    uuid: "53856c96-fe4d-459e-a9dc-c339b1bc3310",
-    name: "SAMPLE_08",
-    columns: [{
-        id: "1",
-        name: "CODE",
-        datatype: "string"
-    }, {
-        id: "2",
-        name: "DESCRIPTION",
-        datatype: "string"
-    }, {
-        id: "3",
-        name: "TOTAL_EMP",
-        datatype: "int"
-    }, {
-        id: "4",
-        name: "SALARY",
-        datatype: "int"
-    }],
-    database: "DEFAULT",
-    cardinality: {},
-    last_modified: 0,
-    exd: {
-        minFileSize: "46069",
-        totalNumberFiles: "1",
-        location: "hdfs://sandbox.hortonworks.com:8020/apps/hive/warehouse/sample_08",
-        lastAccessTime: "1398176495945",
-        lastUpdateTime: "1398176495981",
-        columns: "struct columns { string code, string description, i32 total_emp, i32 salary}",
-        partitionColumns: "",
-        EXD_STATUS: "true",
-        maxFileSize: "46069",
-        inputformat: "org.apache.hadoop.mapred.TextInputFormat",
-        partitioned: "false",
-        tableName: "sample_08",
-        owner: "hue",
-        totalFileSize: "46069",
-        outputformat: "org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"
-    }
-  }
-]
-```
-
-## Load Hive Tables
-`POST /tables/{tables}/{project}`
-
-#### Request Parameters
-* tables - `required` `string` table names you want to load from hive, separated with comma.
-* project - `required` `String`  the project which the tables will be loaded into.
-
-#### Response Sample
-```
-{
-    "result.loaded": ["DEFAULT.SAMPLE_07"],
-    "result.unloaded": ["sapmle_08"]
-}
-```
-
-***
-
-## Wipe cache
-`GET /cache/{type}/{name}/{action}`
-
-#### Path variable
-* type - `required` `string` 'METADATA' or 'CUBE'
-* name - `required` `string` Cache key, e.g the cube name.
-* action - `required` `string` 'create', 'update' or 'drop'
-

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_docs/howto/howto_use_restapi_in_js.md
----------------------------------------------------------------------
diff --git a/website/_docs/howto/howto_use_restapi_in_js.md b/website/_docs/howto/howto_use_restapi_in_js.md
deleted file mode 100644
index f41b3c5..0000000
--- a/website/_docs/howto/howto_use_restapi_in_js.md
+++ /dev/null
@@ -1,48 +0,0 @@
----
-layout: docs
-title:  How to Use Restful API in Javascript
-categories: howto
-permalink: /docs/howto/howto_use_restapi_in_js.html
-version: v1.2
-since: v0.7.1
----
-Kylin security is based on basic access authorization, if you want to use API in your javascript, you need to add authorization info in http headers.
-
-## Example on Query API.
-```
-$.ajaxSetup({
-      headers: { 'Authorization': "Basic eWFu**********X***ZA==", 'Content-Type': 'application/json;charset=utf-8' } // use your own authorization code here
-    });
-    var request = $.ajax({
-       url: "http://hostname/kylin/api/query",
-       type: "POST",
-       data: '{"sql":"select count(*) from SUMMARY;","offset":0,"limit":50000,"acceptPartial":true,"project":"test"}',
-       dataType: "json"
-    });
-    request.done(function( msg ) {
-       alert(msg);
-    }); 
-    request.fail(function( jqXHR, textStatus ) {
-       alert( "Request failed: " + textStatus );
-  });
-
-```
-
-## Keypoints
-1. add basic access authorization info in http headers.
-2. use right ajax type and data synax.
-
-## Basic access authorization
-For what is basic access authorization, refer to [Wikipedia Page](http://en.wikipedia.org/wiki/Basic_access_authentication).
-How to generate your authorization code (download and import "jquery.base64.js" from [https://github.com/yckart/jquery.base64.js](https://github.com/yckart/jquery.base64.js)).
-
-```
-var authorizationCode = $.base64('encode', 'NT_USERNAME' + ":" + 'NT_PASSWORD');
- 
-$.ajaxSetup({
-   headers: { 
-    'Authorization': "Basic " + authorizationCode, 
-    'Content-Type': 'application/json;charset=utf-8' 
-   }
-});
-```

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_docs/index.cn.md
----------------------------------------------------------------------
diff --git a/website/_docs/index.cn.md b/website/_docs/index.cn.md
deleted file mode 100644
index 053524c..0000000
--- a/website/_docs/index.cn.md
+++ /dev/null
@@ -1,22 +0,0 @@
----
-layout: docs-cn
-title: 概述
-categories: docs
-permalink: /cn/docs/index.html
----
-
-欢迎来到 Apache Kylin™
-------------  
-> Extreme OLAP Engine for Big Data
-
-Apache Kylin™是一个开源的分布式分析引擎,提供Hadoop之上的SQL查询接口及多维分析(OLAP)能力以支持超大规模数据,最初由eBay Inc.开发并贡献至开源社区。
-
-安装 
-------------  
-请参考安装文档以安装Apache Kylin: [安装向导](/cn/docs/install/)
-
-
-
-
-
-

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_docs/index.md
----------------------------------------------------------------------
diff --git a/website/_docs/index.md b/website/_docs/index.md
deleted file mode 100644
index 89b5024..0000000
--- a/website/_docs/index.md
+++ /dev/null
@@ -1,52 +0,0 @@
----
-layout: docs
-title: Overview
-categories: docs
-permalink: /docs/index.html
----
-
-Welcome to Apache Kylin™
-------------  
-> Extreme OLAP Engine for Big Data
-
-Apache Kylin™ is an open source Distributed Analytics Engine designed to provide SQL interface and multi-dimensional analysis (OLAP) on Hadoop supporting extremely large datasets, original contributed from eBay Inc.
-
-Installation & Setup
-------------  
-
-Please follow installation & tutorial in the navigation panel.
-
-Advanced Topics
--------  
-
-#### Connectivity
-
-1. [How to use Kylin remote JDBC driver](howto/howto_jdbc.html)
-2. [SQL reference](http://calcite.apache.org/)
-
----
-
-#### REST APIs
-
-1. [Kylin Restful API list](howto/howto_use_restapi.html)
-2. [Build cube with Restful API](howto/howto_build_cube_with_restapi.html)
-3. [How to consume Kylin REST API in javascript](howto/howto_use_restapi_in_js.html)
-
----
-
-#### Operations
-
-1. [Backup/restore Kylin metadata store](howto/howto_backup_metadata.html)
-2. [Cleanup storage (HDFS & HBase tables)](howto/howto_cleanup_storage.html)
-3. [Advanced env configurations](install/advance_settings.html)
-3. [How to upgrade](howto/howto_upgrade.html)
-
----
-
-#### Technical Details
-
-1. [New meta data model structure](/development/new_metadata.html)
-
-
-
-


[15/52] [abbrv] kylin git commit: KYLIN-1024 Load hive tables with selecting mode

Posted by li...@apache.org.
KYLIN-1024 Load hive tables with selecting mode


Project: http://git-wip-us.apache.org/repos/asf/kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/75970c3b
Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/75970c3b
Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/75970c3b

Branch: refs/heads/1.x-HBase1.1.3
Commit: 75970c3b1b576fdf1624e71eb70373e164541bc1
Parents: 0c13797
Author: janzhongi <ji...@ebay.com>
Authored: Sun Jan 31 01:58:12 2016 +0800
Committer: janzhongi <ji...@ebay.com>
Committed: Sun Jan 31 01:58:12 2016 +0800

----------------------------------------------------------------------
 .../apache/kylin/common/util/HiveClient.java    |   8 +
 conf/kylin.properties                           |   3 +-
 .../test_case_data/sandbox/kylin.properties     |   3 +
 .../kylin/rest/controller/TableController.java  |  44 ++
 webapp/app/js/controllers/sourceMeta.js         | 176 ++++++-
 .../app/js/directives/angular-tree-control.js   | 524 ++++++++++++-------
 webapp/app/js/model/tableModel.js               |   6 -
 webapp/app/js/services/kylinProperties.js       |   7 +
 webapp/app/js/services/tables.js                |   7 +-
 .../app/partials/tables/source_table_tree.html  |  27 +
 webapp/bower.json                               |   2 +-
 11 files changed, 601 insertions(+), 206 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/75970c3b/common/src/main/java/org/apache/kylin/common/util/HiveClient.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/kylin/common/util/HiveClient.java b/common/src/main/java/org/apache/kylin/common/util/HiveClient.java
index 41f0a9a..c56cb6c 100644
--- a/common/src/main/java/org/apache/kylin/common/util/HiveClient.java
+++ b/common/src/main/java/org/apache/kylin/common/util/HiveClient.java
@@ -133,6 +133,14 @@ public class HiveClient {
         return getBasicStatForTable(new org.apache.hadoop.hive.ql.metadata.Table(table), StatsSetupConst.NUM_FILES);
     }
 
+    public List<String> getHiveDbNames() throws Exception {
+        return getMetaStoreClient().getAllDatabases();
+    }
+
+    public List<String> getHiveTableNames(String database) throws Exception {
+        return getMetaStoreClient().getAllTables(database);
+    }
+
     /**
      * COPIED FROM org.apache.hadoop.hive.ql.stats.StatsUtil for backward compatibility
      * 

http://git-wip-us.apache.org/repos/asf/kylin/blob/75970c3b/conf/kylin.properties
----------------------------------------------------------------------
diff --git a/conf/kylin.properties b/conf/kylin.properties
index e196d7a..e0727ed 100644
--- a/conf/kylin.properties
+++ b/conf/kylin.properties
@@ -125,7 +125,8 @@ kylin.web.hadoop=
 kylin.web.diagnostic=
 #contact mail on web page ,optional
 kylin.web.contact_mail=
-
+#set the number of databases and tables of each database shown on the hive tree
+kylin.web.hive.limit=20
 ###########################config info for front#######################
 
 #env DEV|QA|PROD

http://git-wip-us.apache.org/repos/asf/kylin/blob/75970c3b/examples/test_case_data/sandbox/kylin.properties
----------------------------------------------------------------------
diff --git a/examples/test_case_data/sandbox/kylin.properties b/examples/test_case_data/sandbox/kylin.properties
index efdd6d4..3a7db5e 100644
--- a/examples/test_case_data/sandbox/kylin.properties
+++ b/examples/test_case_data/sandbox/kylin.properties
@@ -120,6 +120,9 @@ kylin.web.diagnostic=
 #contact mail on web page ,optional
 kylin.web.contact_mail=
 
+#set the number of databases and tables of each database shown on the hive tree
+kylin.web.hive.limit=20
+
 ###########################config info for front#######################
 
 #env DEV|QA|PROD

http://git-wip-us.apache.org/repos/asf/kylin/blob/75970c3b/server/src/main/java/org/apache/kylin/rest/controller/TableController.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/kylin/rest/controller/TableController.java b/server/src/main/java/org/apache/kylin/rest/controller/TableController.java
index 5e817e3..f899d99 100644
--- a/server/src/main/java/org/apache/kylin/rest/controller/TableController.java
+++ b/server/src/main/java/org/apache/kylin/rest/controller/TableController.java
@@ -27,6 +27,7 @@ import java.util.List;
 import java.util.Map;
 
 import org.apache.commons.lang.StringUtils;
+import org.apache.kylin.common.util.HiveClient;
 import org.apache.kylin.metadata.MetadataConstants;
 import org.apache.kylin.metadata.model.ColumnDesc;
 import org.apache.kylin.metadata.model.TableDesc;
@@ -190,6 +191,49 @@ public class TableController extends BasicController {
         return descs;
     }
 
+
+    /**
+     * Show all databases in Hive
+     *
+     * @return Hive databases list
+     * @throws IOException
+     */
+    @RequestMapping(value = "/hive", method = { RequestMethod.GET })
+    @ResponseBody
+    private static List<String> showHiveDatabases() throws IOException {
+        HiveClient hiveClient = new HiveClient();
+        List<String> results = null;
+
+        try {
+            results = hiveClient.getHiveDbNames();
+        } catch (Exception e) {
+            e.printStackTrace();
+            throw new IOException(e);
+        }
+        return results;
+    }
+
+    /**
+     * Show all tables in a Hive database
+     *
+     * @return Hive table list
+     * @throws IOException
+     */
+    @RequestMapping(value = "/hive/{database}", method = { RequestMethod.GET })
+    @ResponseBody
+    private static List<String> showHiveTables(@PathVariable String database) throws IOException {
+        HiveClient hiveClient = new HiveClient();
+        List<String> results = null;
+
+        try {
+            results = hiveClient.getHiveTableNames(database);
+        } catch (Exception e) {
+            e.printStackTrace();
+            throw new IOException(e);
+        }
+        return results;
+    }
+
     public void setCubeService(CubeService cubeService) {
         this.cubeMgmtService = cubeService;
     }

http://git-wip-us.apache.org/repos/asf/kylin/blob/75970c3b/webapp/app/js/controllers/sourceMeta.js
----------------------------------------------------------------------
diff --git a/webapp/app/js/controllers/sourceMeta.js b/webapp/app/js/controllers/sourceMeta.js
index d05dc7f..5c32f87 100755
--- a/webapp/app/js/controllers/sourceMeta.js
+++ b/webapp/app/js/controllers/sourceMeta.js
@@ -101,13 +101,187 @@ KylinApp
       });
     };
 
-    var ModalInstanceCtrl = function ($scope, $location, $modalInstance, tableNames, MessageService, projectName, scope) {
+    $scope.openTreeModal = function () {
+      $modal.open({
+        templateUrl: 'addHiveTableFromTree.html',
+        controller: ModalInstanceCtrl,
+        resolve: {
+          tableNames: function () {
+            return $scope.tableNames;
+          },
+          projectName:function(){
+            return  $scope.projectModel.selectedProject;
+          },
+          scope: function () {
+            return $scope;
+          }
+        }
+      });
+    };
+
+    var ModalInstanceCtrl = function ($scope, $location, $modalInstance, kylinConfig,tableNames, MessageService, projectName, scope) {
       $scope.tableNames = "";
       $scope.projectName = projectName;
       $scope.cancel = function () {
         $modalInstance.dismiss('cancel');
       };
+
+      $scope.treeOptions = {multiSelection: true};
+      $scope.selectedNodes = [];
+      $scope.hiveLimit =  kylinConfig.getHiveLimit();
+
+      $scope.loadHive = function () {
+        if($scope.hiveLoaded)
+          return;
+        TableService.showHiveDatabases({}, function (databases) {
+          $scope.dbNum = databases.length;
+          if (databases.length > 0) {
+            $scope.hiveMap = {};
+            for (var i = 0; i < databases.length; i++) {
+              var dbName = databases[i];
+              var hiveData = {"dbname":dbName,"tables":[],"expanded":false};
+              $scope.hive.push(hiveData);
+              $scope.hiveMap[dbName] = i;
+            }
+          }
+          $scope.hiveLoaded = true;
+          $scope.showMoreDatabases();
+        });
+      }
+
+      $scope.showMoreTables = function(hiveTables, node){
+        var shownTimes = parseInt(node.children.length / $scope.hiveLimit);
+        var from = $scope.hiveLimit * shownTimes;
+        var to = 0;
+        var hasMore = false;
+        if(from + $scope.hiveLimit > hiveTables.length) {
+          to = hiveTables.length - 1;
+        } else {
+          to = from + $scope.hiveLimit - 1;
+          hasMore = true;
+        }
+        if(!angular.isUndefined(node.children[from])){
+          node.children.pop();
+        }
+
+        for(var idx = from; idx <= to; idx++){
+          node.children.push({"label":node.label+'.'+hiveTables[idx],"id":idx-from+1,"children":[]});
+        }
+
+        if(hasMore){
+          var loading = {"label":"","id":65535,"children":[]};
+          node.children.push(loading);
+        }
+      }
+
+      $scope.showAllTables = function(hiveTables, node){
+        var shownTimes = parseInt(node.children.length / $scope.hiveLimit);
+        var from = $scope.hiveLimit * shownTimes;
+        var to = hiveTables.length - 1;
+        if(!angular.isUndefined(node.children[from])){
+          node.children.pop();
+        }
+        for(var idx = from; idx <= to; idx++){
+          node.children.push({"label":node.label+'.'+hiveTables[idx],"id":idx-from+1,"children":[]});
+        }
+      }
+
+      $scope.showMoreDatabases = function(){
+        var shownTimes = parseInt($scope.treedata.length / $scope.hiveLimit);
+        var from = $scope.hiveLimit * shownTimes;
+        var to = 0;
+        var hasMore = false;
+        if(from + $scope.hiveLimit > $scope.hive.length) {
+          to = $scope.hive.length - 1;
+        } else {
+          to = from + $scope.hiveLimit - 1;
+          hasMore = true;
+        }
+        if(!angular.isUndefined($scope.treedata[from])){
+          $scope.treedata.pop();
+        }
+
+        for(var idx = from; idx <= to; idx++){
+          var children = [];
+          var loading = {"label":"","id":0,"children":[]};
+          children.push(loading);
+          $scope.treedata.push({"label":$scope.hive[idx].dbname,"id":idx+1,"children":children,"expanded":false});
+        }
+
+        if(hasMore){
+          var loading = {"label":"","id":65535,"children":[0]};
+          $scope.treedata.push(loading);
+        }
+      }
+
+      $scope.showAllDatabases = function(){
+        var shownTimes = parseInt($scope.treedata.length / $scope.hiveLimit);
+        var from = $scope.hiveLimit * shownTimes;
+        var to = $scope.hive.length - 1;
+
+        if(!angular.isUndefined($scope.treedata[from])){
+          $scope.treedata.pop();
+        }
+
+        for(var idx = from; idx <= to; idx++){
+          var children = [];
+          var loading = {"label":"","id":0,"children":[]};
+          children.push(loading);
+          $scope.treedata.push({"label":$scope.hive[idx].dbname,"id":idx+1,"children":children,"expanded":false});
+        }
+      }
+
+      $scope.showMoreClicked = function($parentNode){
+        if($parentNode == null){
+          $scope.showMoreDatabases();
+        } else {
+          $scope.showMoreTables($scope.hive[$scope.hiveMap[$parentNode.label]].tables,$parentNode);
+        }
+      }
+
+      $scope.showAllClicked = function($parentNode){
+        if($parentNode == null){
+          $scope.showAllDatabases();
+        } else {
+          $scope.showAllTables($scope.hive[$scope.hiveMap[$parentNode.label]].tables,$parentNode);
+        }
+      }
+
+      $scope.showToggle = function(node) {
+        if(node.expanded == false){
+          TableService.showHiveTables({"database": node.label},function (hive_tables){
+            var tables = [];
+            for (var i = 0; i < hive_tables.length; i++) {
+              tables.push(hive_tables[i]);
+            }
+            $scope.hive[$scope.hiveMap[node.label]].tables = tables;
+            $scope.showMoreTables(tables,node);
+            node.expanded = true;
+          });
+        }
+      }
+
+      $scope.showSelected = function(node) {
+
+      }
+
+      if(angular.isUndefined($scope.hive) || angular.isUndefined($scope.hiveLoaded) || angular.isUndefined($scope.treedata) ){
+        $scope.hive = [];
+        $scope.hiveLoaded = false;
+        $scope.treedata = [];
+        $scope.loadHive();
+      }
+
       $scope.add = function () {
+
+        if($scope.tableNames.length === 0 && $scope.selectedNodes.length > 0) {
+          for(var i = 0; i <  $scope.selectedNodes.length; i++){
+            if($scope.selectedNodes[i].label.indexOf(".") >= 0){
+              $scope.tableNames += ($scope.selectedNodes[i].label) += ',';
+            }
+          }
+        }
+
         if ($scope.tableNames.trim() === "") {
           SweetAlert.swal('', 'Please input table(s) you want to synchronize.', 'info');
           return;

http://git-wip-us.apache.org/repos/asf/kylin/blob/75970c3b/webapp/app/js/directives/angular-tree-control.js
----------------------------------------------------------------------
diff --git a/webapp/app/js/directives/angular-tree-control.js b/webapp/app/js/directives/angular-tree-control.js
index 6a7d26c..6fca987 100644
--- a/webapp/app/js/directives/angular-tree-control.js
+++ b/webapp/app/js/directives/angular-tree-control.js
@@ -20,208 +20,344 @@
  * IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN
  * CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
  */
-  (function ( angular ) {
-    'use strict';
-
-    angular.module( 'treeControl', [] )
-        .directive( 'treecontrol', ['$compile', function( $compile ) {
-            /**
-             * @param cssClass - the css class
-             * @param addClassProperty - should we wrap the class name with class=""
-             */
-            function classIfDefined(cssClass, addClassProperty) {
-                if (cssClass) {
-                    if (addClassProperty)
-                        return 'class="' + cssClass + '"';
-                    else
-                        return cssClass;
+
+(function ( angular ) {
+  'use strict';
+
+  angular.module( 'treeControl', [] )
+    .directive( 'treecontrol', ['$compile', function( $compile ) {
+      /**
+       * @param cssClass - the css class
+       * @param addClassProperty - should we wrap the class name with class=""
+       */
+      function classIfDefined(cssClass, addClassProperty) {
+        if (cssClass) {
+          if (addClassProperty)
+            return 'class="' + cssClass + '"';
+          else
+            return cssClass;
+        }
+        else
+          return "";
+      }
+
+      function ensureDefault(obj, prop, value) {
+        if (!obj.hasOwnProperty(prop))
+          obj[prop] = value;
+      }
+
+      return {
+        restrict: 'EA',
+        require: "treecontrol",
+        transclude: true,
+        scope: {
+          treeModel: "=",
+          selectedNode: "=?",
+          selectedNodes: "=?",
+          expandedNodes: "=?",
+          onSelection: "&",
+          onNodeToggle: "&",
+          options: "=?",
+          orderBy: "@",
+          reverseOrder: "@",
+          filterExpression: "=?",
+          filterComparator: "=?",
+          onDblclick: "&"
+        },
+        controller: ['$scope', function( $scope ) {
+
+          function defaultIsLeaf(node) {
+            return !node[$scope.options.nodeChildren] || node[$scope.options.nodeChildren].length === 0;
+          }
+
+          function shallowCopy(src, dst) {
+            if (angular.isArray(src)) {
+              dst = dst || [];
+
+              for ( var i = 0; i < src.length; i++) {
+                dst[i] = src[i];
+              }
+            } else if (angular.isObject(src)) {
+              dst = dst || {};
+
+              for (var key in src) {
+                if (hasOwnProperty.call(src, key) && !(key.charAt(0) === '$' && key.charAt(1) === '$')) {
+                  dst[key] = src[key];
                 }
-                else
-                    return "";
+              }
             }
 
-            function ensureDefault(obj, prop, value) {
-                if (!obj.hasOwnProperty(prop))
-                    obj[prop] = value;
+            return dst || src;
+          }
+          function defaultEquality(a, b) {
+            if (a === undefined || b === undefined)
+              return false;
+            a = shallowCopy(a);
+            a[$scope.options.nodeChildren] = [];
+            b = shallowCopy(b);
+            b[$scope.options.nodeChildren] = [];
+            return angular.equals(a, b);
+          }
+
+          $scope.options = $scope.options || {};
+          ensureDefault($scope.options, "multiSelection", false);
+          ensureDefault($scope.options, "nodeChildren", "children");
+          ensureDefault($scope.options, "dirSelectable", "true");
+          ensureDefault($scope.options, "injectClasses", {});
+          ensureDefault($scope.options.injectClasses, "ul", "");
+          ensureDefault($scope.options.injectClasses, "li", "");
+          ensureDefault($scope.options.injectClasses, "liSelected", "");
+          ensureDefault($scope.options.injectClasses, "iExpanded", "");
+          ensureDefault($scope.options.injectClasses, "iCollapsed", "");
+          ensureDefault($scope.options.injectClasses, "iLeaf", "");
+          ensureDefault($scope.options.injectClasses, "label", "");
+          ensureDefault($scope.options.injectClasses, "labelSelected", "");
+          ensureDefault($scope.options, "equality", defaultEquality);
+          ensureDefault($scope.options, "isLeaf", defaultIsLeaf);
+
+          $scope.selectedNodes = $scope.selectedNodes || [];
+          $scope.expandedNodes = $scope.expandedNodes || [];
+          $scope.expandedNodesMap = {};
+          for (var i=0; i < $scope.expandedNodes.length; i++) {
+            $scope.expandedNodesMap[""+i] = $scope.expandedNodes[i];
+          }
+          $scope.parentScopeOfTree = $scope.$parent;
+
+
+          function isSelectedNode(node) {
+            if (!$scope.options.multiSelection && ($scope.options.equality(node, $scope.selectedNode)))
+              return true;
+            else if ($scope.options.multiSelection && $scope.selectedNodes) {
+              for (var i = 0; (i < $scope.selectedNodes.length); i++) {
+                if ($scope.options.equality(node, $scope.selectedNodes[i])) {
+                  return true;
+                }
+              }
+              return false;
             }
+          }
+
+          $scope.headClass = function(node) {
+            var liSelectionClass = classIfDefined($scope.options.injectClasses.liSelected, false);
+            var injectSelectionClass = "";
+            if (liSelectionClass && isSelectedNode(node))
+              injectSelectionClass = " " + liSelectionClass;
+            if ($scope.options.isLeaf(node))
+              return "tree-leaf" + injectSelectionClass;
+            if ($scope.expandedNodesMap[this.$id])
+              return "tree-expanded" + injectSelectionClass;
+            else
+              return "tree-collapsed" + injectSelectionClass;
+          };
 
-            return {
-                restrict: 'EA',
-                require: "treecontrol",
-                transclude: true,
-                scope: {
-                    treeModel: "=",
-                    selectedNode: "=?",
-                    onSelection: "&",
-                    options: "=?",
-                    onDblclick: "&"
-                },
-                controller: function( $scope ) {
-                    function defaultIsSelectable() {
-                      return true;
-                    }
-
-                    $scope.options = $scope.options || {};
-                    ensureDefault($scope.options, "nodeChildren", "children");
-                    ensureDefault($scope.options, "dirSelectable", "true");
-                    ensureDefault($scope.options, "injectClasses", {});
-                    ensureDefault($scope.options.injectClasses, "ul", "");
-                    ensureDefault($scope.options.injectClasses, "li", "");
-                    ensureDefault($scope.options.injectClasses, "liSelected", "");
-                    ensureDefault($scope.options.injectClasses, "iExpanded", "");
-                    ensureDefault($scope.options.injectClasses, "iCollapsed", "");
-                    ensureDefault($scope.options.injectClasses, "iLeaf", "");
-                    ensureDefault($scope.options.injectClasses, "label", "");
-                    ensureDefault($scope.options.injectClasses, "labelSelected", "");
-                    ensureDefault($scope.options, "isSelectable", defaultIsSelectable);
-                  ensureDefault($scope.options, "equality", function (a, b) {
-                        if (a === undefined || b === undefined)
-                            return false;
-                        a = angular.copy(a); a[$scope.options.nodeChildren] = [];
-                        b = angular.copy(b); b[$scope.options.nodeChildren] = [];
-                        return angular.equals(a, b);
-                    });
-
-                    $scope.expandedNodes = {};
-                    $scope.parentScopeOfTree = $scope.$parent;
-
-                    $scope.headClass = function(node) {
-                        var liSelectionClass = classIfDefined($scope.options.injectClasses.liSelected, false);
-                        var injectSelectionClass = "";
-                        if (liSelectionClass && (this.$id == $scope.selectedScope))
-                            injectSelectionClass = " " + liSelectionClass;
-                        if (!node[$scope.options.nodeChildren] || node[$scope.options.nodeChildren].length === 0)
-                            return "tree-leaf" + injectSelectionClass
-                        if ($scope.expandedNodes[this.$id])
-                            return "tree-expanded" + injectSelectionClass;
-                        else
-                            return "tree-collapsed" + injectSelectionClass;
-                    };
-
-                    $scope.iBranchClass = function() {
-                        if ($scope.expandedNodes[this.$id])
-                            return classIfDefined($scope.options.injectClasses.iExpanded);
-                        else
-                            return classIfDefined($scope.options.injectClasses.iCollapsed);
-                    };
-
-                    $scope.nodeExpanded = function() {
-                        return !!$scope.expandedNodes[this.$id];
-                    };
-
-                    $scope.selectNodeHead = function() {
-                        $scope.expandedNodes[this.$id] = ($scope.expandedNodes[this.$id] === undefined ? this.node : undefined);
-                    };
-
-                    $scope.selectNodeLabel = function( selectedNode ){
-                        if (selectedNode[$scope.options.nodeChildren] && selectedNode[$scope.options.nodeChildren].length > 0 &&
-                          (!$scope.options.dirSelectable||!$scope.options.isSelectable(selectedNode))) {
-                            this.selectNodeHead();
-                        }
-                        else {
-                            $scope.selectedScope = this.$id;
-                            $scope.selectedNode = selectedNode;
-                            if ($scope.onSelection)
-                                $scope.onSelection({node: selectedNode});
-                        }
-                    };
-
-                    $scope.dblClickNode = function(selectedNode){
-                        if($scope.onDblclick!=null){
-                           $scope.onDblclick({node:selectedNode});
-                        }
-                    }
-
-                    $scope.selectedClass = function() {
-                        var labelSelectionClass = classIfDefined($scope.options.injectClasses.labelSelected, false);
-                        var injectSelectionClass = "";
-                        if (labelSelectionClass && (this.$id == $scope.selectedScope))
-                            injectSelectionClass = " " + labelSelectionClass;
-
-                        return (this.$id == $scope.selectedScope)?"tree-selected" + injectSelectionClass:"";
-                    };
-
-                    //tree template
-                    var template =
-                        '<ul '+classIfDefined($scope.options.injectClasses.ul, true)+'>' +
-                            '<li ng-repeat="node in node.' + $scope.options.nodeChildren+'" ng-class="headClass(node)" '+classIfDefined($scope.options.injectClasses.li, true)+'>' +
-                            '<i class="tree-branch-head" ng-class="iBranchClass()" ng-click="selectNodeHead(node)"></i>' +
-                            '<i class="tree-leaf-head '+classIfDefined($scope.options.injectClasses.iLeaf, false)+'"></i>' +
-                            '<div class="tree-label '+classIfDefined($scope.options.injectClasses.label, false)+'" ng-class="selectedClass()" ng-click="selectNodeLabel(node)" ng-dblclick="dblClickNode(node)" tree-transclude></div>' +
-                            '<treeitem ng-if="nodeExpanded()"></treeitem>' +
-                            '</li>' +
-                            '</ul>';
-
-                    return {
-                        template: $compile(template)
-                    }
-                },
-                compile: function(element, attrs, childTranscludeFn) {
-                    return function ( scope, element, attrs, treemodelCntr ) {
-
-                        scope.$watch("treeModel", function updateNodeOnRootScope(newValue) {
-                            if (angular.isArray(newValue)) {
-                                if (angular.isDefined(scope.node) && angular.equals(scope.node[scope.options.nodeChildren], newValue))
-                                    return;
-                                scope.node = {};
-                                scope.node[scope.options.nodeChildren] = newValue;
-                            }
-                            else {
-                                if (angular.equals(scope.node, newValue))
-                                    return;
-                                scope.node = newValue;
-                            }
-                        });
-
-                        //Rendering template for a root node
-                        treemodelCntr.template( scope, function(clone) {
-                            element.html('').append( clone );
-                        });
-                        // save the transclude function from compile (which is not bound to a scope as apposed to the one from link)
-                        // we can fix this to work with the link transclude function with angular 1.2.6. as for angular 1.2.0 we need
-                        // to keep using the compile function
-                        scope.$treeTransclude = childTranscludeFn;
-                    }
+          $scope.iBranchClass = function() {
+            if ($scope.expandedNodesMap[this.$id])
+              return classIfDefined($scope.options.injectClasses.iExpanded);
+            else
+              return classIfDefined($scope.options.injectClasses.iCollapsed);
+          };
+
+          $scope.nodeExpanded = function() {
+            return !!$scope.expandedNodesMap[this.$id];
+          };
+
+          $scope.selectNodeHead = function() {
+            var expanding = $scope.expandedNodesMap[this.$id] === undefined;
+            $scope.expandedNodesMap[this.$id] = (expanding ? this.node : undefined);
+            if (expanding) {
+              $scope.expandedNodes.push(this.node);
+            }
+            else {
+              var index;
+              for (var i=0; (i < $scope.expandedNodes.length) && !index; i++) {
+                if ($scope.options.equality($scope.expandedNodes[i], this.node)) {
+                  index = i;
                 }
-            };
-        }])
-        .directive("treeitem", function() {
-            return {
-                restrict: 'E',
-                require: "^treecontrol",
-                link: function( scope, element, attrs, treemodelCntr) {
-                    // Rendering template for the current node
-                    treemodelCntr.template(scope, function(clone) {
-                        element.html('').append(clone);
-                    });
+              }
+              if (index != undefined)
+                $scope.expandedNodes.splice(index, 1);
+            }
+            if ($scope.onNodeToggle)
+              $scope.onNodeToggle({node: this.node, expanded: expanding});
+          };
+
+          $scope.selectNodeLabel = function( selectedNode ){
+            if(selectedNode[$scope.options.nodeChildren] && selectedNode[$scope.options.nodeChildren].length > 0){
+              this.selectNodeHead();
+            }
+            if($scope.options.dirSelectable || !(selectedNode[$scope.options.nodeChildren] && selectedNode[$scope.options.nodeChildren].length > 0) )
+             {
+              var selected = false;
+              if ($scope.options.multiSelection) {
+                var pos = $scope.selectedNodes.indexOf(selectedNode);
+                if (pos === -1) {
+                  $scope.selectedNodes.push(selectedNode);
+                  selected = true;
+                } else {
+                  $scope.selectedNodes.splice(pos, 1);
+                }
+              } else {
+                if ($scope.selectedNode != selectedNode) {
+                  $scope.selectedNode = selectedNode;
+                  selected = true;
                 }
+                else {
+                  $scope.selectedNode = undefined;
+                }
+              }
+              if ($scope.onSelection)
+                $scope.onSelection({node: selectedNode, selected: selected});
             }
-        })
-        .directive("treeTransclude", function() {
-            return {
-                link: function(scope, element, attrs, controller) {
-                    angular.forEach(scope.expandedNodes, function (node, id) {
-                        if (scope.options.equality(node, scope.node)) {
-                            scope.expandedNodes[scope.$id] = scope.node;
-                            scope.expandedNodes[id] = undefined;
-                        }
-                    });
-                    if (scope.options.equality(scope.node, scope.selectedNode)) {
-                        scope.selectNodeLabel(scope.node);
-                    }
-
-                    // create a scope for the transclusion, whos parent is the parent of the tree control
-                    scope.transcludeScope = scope.parentScopeOfTree.$new();
-                    scope.transcludeScope.node = scope.node;
-                    scope.$on('$destroy', function() {
-                        scope.transcludeScope.$destroy();
-                    });
-
-                    scope.$treeTransclude(scope.transcludeScope, function(clone) {
-                        element.empty();
-                        element.append(clone);
-                    });
+          };
+
+
+          $scope.dblClickNode = function(selectedNode){
+            if($scope.onDblclick!=null){
+              $scope.onDblclick({node:selectedNode});
+            }
+          }
+
+          $scope.selectedClass = function() {
+            var isThisNodeSelected = isSelectedNode(this.node);
+            var labelSelectionClass = classIfDefined($scope.options.injectClasses.labelSelected, false);
+            var injectSelectionClass = "";
+            if (labelSelectionClass && isThisNodeSelected)
+              injectSelectionClass = " " + labelSelectionClass;
+
+            return isThisNodeSelected?"tree-selected" + injectSelectionClass:"";
+          };
+
+          //tree template
+          var orderBy = $scope.orderBy ? ' | orderBy:orderBy:reverseOrder' : '';
+          var template =
+            '<ul '+classIfDefined($scope.options.injectClasses.ul, true)+'>' +
+            '<li ng-repeat="node in node.' + $scope.options.nodeChildren + ' | filter:filterExpression:filterComparator ' + orderBy + '" ng-class="headClass(node)" '+classIfDefined($scope.options.injectClasses.li, true)+'>' +
+            '<i class="tree-branch-head" ng-class="iBranchClass()" ng-click="selectNodeHead(node)"></i>' +
+            '<i class="tree-leaf-head '+classIfDefined($scope.options.injectClasses.iLeaf, false)+'"></i>' +
+            '<div class="tree-label '+classIfDefined($scope.options.injectClasses.label, false)+'" ng-class="selectedClass()" ng-click="selectNodeLabel(node)" ng-dblclick="dblClickNode(node)" tree-transclude></div>' +
+            '<treeitem ng-if="nodeExpanded()"></treeitem>' +
+            '</li>' +
+            '</ul>';
+
+          this.template = $compile(template);
+        }],
+        compile: function(element, attrs, childTranscludeFn) {
+          return function ( scope, element, attrs, treemodelCntr ) {
+
+            scope.$watch("treeModel", function updateNodeOnRootScope(newValue) {
+              if (angular.isArray(newValue)) {
+                if (angular.isDefined(scope.node) && angular.equals(scope.node[scope.options.nodeChildren], newValue))
+                  return;
+                scope.node = {};
+                scope.synteticRoot = scope.node;
+                scope.node[scope.options.nodeChildren] = newValue;
+              }
+              else {
+                if (angular.equals(scope.node, newValue))
+                  return;
+                scope.node = newValue;
+              }
+            });
+
+            scope.$watchCollection('expandedNodes', function(newValue) {
+              var notFoundIds = 0;
+              var newExpandedNodesMap = {};
+              var $liElements = element.find('li');
+              var existingScopes = [];
+              // find all nodes visible on the tree and the scope $id of the scopes including them
+              angular.forEach($liElements, function(liElement) {
+                var $liElement = angular.element(liElement);
+                var liScope = $liElement.scope();
+                existingScopes.push(liScope);
+              });
+              // iterate over the newValue, the new expanded nodes, and for each find it in the existingNodesAndScopes
+              // if found, add the mapping $id -> node into newExpandedNodesMap
+              // if not found, add the mapping num -> node into newExpandedNodesMap
+              angular.forEach(newValue, function(newExNode) {
+                var found = false;
+                for (var i=0; (i < existingScopes.length) && !found; i++) {
+                  var existingScope = existingScopes[i];
+                  if (scope.options.equality(newExNode, existingScope.node)) {
+                    newExpandedNodesMap[existingScope.$id] = existingScope.node;
+                    found = true;
+                  }
                 }
+                if (!found)
+                  newExpandedNodesMap[notFoundIds++] = newExNode;
+              });
+              scope.expandedNodesMap = newExpandedNodesMap;
+            });
+
+//                        scope.$watch('expandedNodesMap', function(newValue) {
+//
+//                        });
+
+            //Rendering template for a root node
+            treemodelCntr.template( scope, function(clone) {
+              element.html('').append( clone );
+            });
+            // save the transclude function from compile (which is not bound to a scope as apposed to the one from link)
+            // we can fix this to work with the link transclude function with angular 1.2.6. as for angular 1.2.0 we need
+            // to keep using the compile function
+            scope.$treeTransclude = childTranscludeFn;
+          }
+        }
+      };
+    }])
+    .directive("treeitem", function() {
+      return {
+        restrict: 'E',
+        require: "^treecontrol",
+        link: function( scope, element, attrs, treemodelCntr) {
+          // Rendering template for the current node
+          treemodelCntr.template(scope, function(clone) {
+            element.html('').append(clone);
+          });
+        }
+      }
+    })
+    .directive("treeTransclude", function() {
+      return {
+        link: function(scope, element, attrs, controller) {
+          if (!scope.options.isLeaf(scope.node)) {
+            angular.forEach(scope.expandedNodesMap, function (node, id) {
+              if (scope.options.equality(node, scope.node)) {
+                scope.expandedNodesMap[scope.$id] = scope.node;
+                scope.expandedNodesMap[id] = undefined;
+              }
+            });
+          }
+          if (!scope.options.multiSelection && scope.options.equality(scope.node, scope.selectedNode)) {
+            scope.selectedNode = scope.node;
+          } else if (scope.options.multiSelection) {
+            var newSelectedNodes = [];
+            for (var i = 0; (i < scope.selectedNodes.length); i++) {
+              if (scope.options.equality(scope.node, scope.selectedNodes[i])) {
+                newSelectedNodes.push(scope.node);
+              }
             }
-        });
+            scope.selectedNodes = newSelectedNodes;
+          }
+
+          // create a scope for the transclusion, whos parent is the parent of the tree control
+          scope.transcludeScope = scope.parentScopeOfTree.$new();
+          scope.transcludeScope.node = scope.node;
+          scope.transcludeScope.$parentNode = (scope.$parent.node === scope.synteticRoot)?null:scope.$parent.node;
+          scope.transcludeScope.$index = scope.$index;
+          scope.transcludeScope.$first = scope.$first;
+          scope.transcludeScope.$middle = scope.$middle;
+          scope.transcludeScope.$last = scope.$last;
+          scope.transcludeScope.$odd = scope.$odd;
+          scope.transcludeScope.$even = scope.$even;
+          scope.$on('$destroy', function() {
+            scope.transcludeScope.$destroy();
+          });
+
+          scope.$treeTransclude(scope.transcludeScope, function(clone) {
+            element.empty();
+            element.append(clone);
+          });
+        }
+      }
+    });
 })( angular );

http://git-wip-us.apache.org/repos/asf/kylin/blob/75970c3b/webapp/app/js/model/tableModel.js
----------------------------------------------------------------------
diff --git a/webapp/app/js/model/tableModel.js b/webapp/app/js/model/tableModel.js
index 2b9c8ea..ca9c47c 100755
--- a/webapp/app/js/model/tableModel.js
+++ b/webapp/app/js/model/tableModel.js
@@ -48,12 +48,6 @@ KylinApp.service('TableModel', function (ProjectModel, $q, TableService) {
 
   this.treeOptions = {
     nodeChildren: "columns",
-    isSelectable:function(node){
-      if(node.id||node.uuid){//db
-        return true;
-      }
-      return false;
-    },
     injectClasses: {
       ul: "a1",
       li: "a2",

http://git-wip-us.apache.org/repos/asf/kylin/blob/75970c3b/webapp/app/js/services/kylinProperties.js
----------------------------------------------------------------------
diff --git a/webapp/app/js/services/kylinProperties.js b/webapp/app/js/services/kylinProperties.js
index a03403b..82158ec 100644
--- a/webapp/app/js/services/kylinProperties.js
+++ b/webapp/app/js/services/kylinProperties.js
@@ -20,6 +20,7 @@ KylinApp.service('kylinConfig', function (AdminService, $log) {
   var _config;
   var timezone;
   var deployEnv;
+  var deployEnv = 20;
 
 
   this.init = function () {
@@ -61,6 +62,12 @@ KylinApp.service('kylinConfig', function (AdminService, $log) {
     }
     return this.deployEnv.toUpperCase();
   }
+  this.getHiveLimit = function () {
+    if (!this.hiveLimit) {
+      this.hiveLimit = this.getProperty("kylin.web.hive.limit");
+    }
+    return this.hiveLimit;
+  }
 
   //fill config info for Config from backend
   this.initWebConfigInfo = function () {

http://git-wip-us.apache.org/repos/asf/kylin/blob/75970c3b/webapp/app/js/services/tables.js
----------------------------------------------------------------------
diff --git a/webapp/app/js/services/tables.js b/webapp/app/js/services/tables.js
index 1f1f15a..e98603b 100755
--- a/webapp/app/js/services/tables.js
+++ b/webapp/app/js/services/tables.js
@@ -17,12 +17,13 @@
  */
 
 KylinApp.factory('TableService', ['$resource', function ($resource, config) {
-  return $resource(Config.service.url + 'tables/:tableName/:action', {}, {
+  return $resource(Config.service.url + 'tables/:tableName/:action/:database', {}, {
     list: {method: 'GET', params: {}, cache: true, isArray: true},
     get: {method: 'GET', params: {}, isArray: false},
     getExd: {method: 'GET', params: {action: 'exd-map'}, isArray: false},
     reload: {method: 'PUT', params: {action: 'reload'}, isArray: false},
     loadHiveTable: {method: 'POST', params: {}, isArray: false},
-    genCardinality: {method: 'PUT', params: {action: 'cardinality'}, isArray: false}
-  });
+    genCardinality: {method: 'PUT', params: {action: 'cardinality'}, isArray: false},
+    showHiveDatabases: {method: 'GET', params: {action:'hive'}, cache: true, isArray: true},
+    showHiveTables: {method: 'GET', params: {action:'hive'}, cache: true, isArray: true}  });
 }]);

http://git-wip-us.apache.org/repos/asf/kylin/blob/75970c3b/webapp/app/partials/tables/source_table_tree.html
----------------------------------------------------------------------
diff --git a/webapp/app/partials/tables/source_table_tree.html b/webapp/app/partials/tables/source_table_tree.html
index bdbe079..c048685 100755
--- a/webapp/app/partials/tables/source_table_tree.html
+++ b/webapp/app/partials/tables/source_table_tree.html
@@ -26,6 +26,8 @@
         <div class="col-xs-5" style="padding-left: 0px;margin-top: 20px;">
             <div class="pull-right">
                 <a class="btn btn-xs btn-primary" tooltip="Load Hive Table"  ng-if="userService.hasRole('ROLE_ADMIN')"  ng-click="openModal()"><i class="fa fa-download"></i></a>
+                <a class="btn btn-xs btn-info" tooltip="Load Hive Table From Tree"  ng-if="userService.hasRole('ROLE_ADMIN')"  ng-click="openTreeModal()"><i class="fa fa-download"></i></a>
+
                 <a class="btn btn-xs btn-success" tooltip="Refresh Tables" ng-click="aceSrcTbChanged()"><i class="fa fa-refresh"></i></a>
             </div>
         </div>
@@ -60,3 +62,28 @@
     <button class="btn btn-primary" ng-click="cancel()">Cancel</button>
   </div>
 </script>
+
+<script type="text/ng-template" id="addHiveTableFromTree.html">
+  <div class="modal-header"><button class="close" type="button" data-dismiss="modal" ng-click="cancel()">×</button>
+    <h4>Load Hive Table Metadata From Tree</h4>
+  </div>
+  <div class="modal-body">
+    <span><strong>Project: </strong>{{ $parent.projectName!=null?$parent.projectName:'NULL'}}</span>
+    <div class="form-group searchBox">
+      <input type="text" placeholder="Filter ..." class="nav-search-input" ng-model="predicate" />
+    </div>
+    <loading ng-if="!hiveLoaded" text="Loading Databases..."></loading>
+    <treecontrol class="tree-light check" tree-model="treedata" selected-nodes="selectedNodes" filter-expression="predicate" on-selection="showSelected(node)" on-node-toggle="showToggle(node)" options="treeOptions">
+      <div ng-if="node.label==''&&node.id==0"><img src="image/ajax-loader.gif">Loading Tables...</div>
+      <button class="btn btn-xs btn-primary" ng-if="node.label==''&&node.id==65535" ng-click="showMoreClicked($parentNode)">Show More</button>
+      <button class="btn btn-xs btn-primary" ng-if="node.label==''&&node.id==65535" ng-click="showAllClicked($parentNode)">Show All</button>
+      {{node.label}}
+    </treecontrol>
+  </div>
+
+  <div class="modal-footer">
+    <button class="btn btn-primary" ng-click="add()">Sync</button>
+    <button class="btn btn-primary" ng-click="cancel()">Cancel</button>
+  </div>
+
+</script>

http://git-wip-us.apache.org/repos/asf/kylin/blob/75970c3b/webapp/bower.json
----------------------------------------------------------------------
diff --git a/webapp/bower.json b/webapp/bower.json
index ff769b5..2c1ac44 100755
--- a/webapp/bower.json
+++ b/webapp/bower.json
@@ -13,7 +13,7 @@
     "font-awesome": "4.3.0",
     "angular-ui-ace": "0.1.0",
     "angular-base64": "2.0.1",
-    "angular-tree-control": "0.1.4",
+    "angular-tree-control": "0.2.8",
     "angularLocalStorage": "~0.3.0",
     "messenger": "1.4.1",
     "moment": "2.5.1",


[26/52] [abbrv] kylin git commit: KYLIN-1416 remove website from coding branch

Posted by li...@apache.org.
http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/index.cn.md
----------------------------------------------------------------------
diff --git a/website/index.cn.md b/website/index.cn.md
deleted file mode 100644
index a4637fc..0000000
--- a/website/index.cn.md
+++ /dev/null
@@ -1,90 +0,0 @@
----
-layout: default-cn
-title: 首页
----
-
-
-<main id="main" >
-  <div class="container" >
-    <div id="zero" class=" main" >
-      <header style=" padding:2em 0 4em 0">
-        <div class="container" >
-
-          <h4 class="section-title"><span>Apache Kylin™ 概览</span></h4>
-          <div class="row" style="margin-top:-20px;">
-            <div class="col-sm-12 col-md-12">              
-              <p class="title_text"> Apache Kylin™是一个开源的分布式分析引擎,提供Hadoop之上的SQL查询接口及多维分析(OLAP)能力以支持超大规模数据,最初由eBay Inc. 开发并贡献至开源社区。</p>
-              <img id="diagram" src="{{ "/assets/images/kylin_diagram.png"| prepend: site.baseurl }}"> </div>
-          </div>
-        </div>
-        <!-- /container --> 
-        
-      </header>
-    </div>
-    <!-- / section --> 
-  </div>
-  <!-- /container -->
-  
-  <section id="second" class="main">
-    <header style="background-color:#efefef;">
-      <div class="container"  >
-        <h4 class="section-title"><span> Kylin是什么? </span></h4>
-        <!-- second-->
-        <div class="row">
-          <div class="col-sm-12 col-md-12">
-            <div align="left">
-              <p> <b>- 可扩展超快OLAP引擎: </b><br/>
-              <div class="indent">Kylin是为减少在Hadoop上百亿规模数据查询延迟而设计</div>
-              </p>
-              <p> <b>- Hadoop ANSI SQL 接口: </b><br/>
-              <div class="indent">Kylin为Hadoop提供标准SQL支持大部分查询功能</div>
-              </p>
-              <p> <b>- 交互式查询能力: </b><br/>
-              <div class="indent">通过Kylin,用户可以与Hadoop数据进行亚秒级交互,在同样的数据集上提供比Hive更好的性能</div>
-              </p>
-              <p> <b>- 多维立方体(MOLAP Cube):</b><br/>
-              <div class="indent">用户能够在Kylin里为百亿以上数据集定义数据模型并构建立方体</div>
-              </p>
-              <p> <b>- 与BI工具无缝整合:</b><br/>
-              <div class="indent">Kylin提供与BI工具,如Tableau,的整合能力,即将提供对其他工具的整合</div>
-              </p>
-              <p> <b>- 其他特性:</b> <br/>
-              <div class="indent">- Job管理与监控 <br/>
-                - 压缩与编码 <br/>
-                - 增量更新 <br/>
-                - 利用HBase Coprocessor<br/>
-                - 基于HyperLogLog的Dinstinc Count近似算法 <br/>
-                - 友好的web界面以管理,监控和使用立方体 <br/>
-                - 项目及立方体级别的访问控制安全<br/>
-                - 支持LDAP </div>
-              </p>
-            </div>
-          </div>
-        </div>
-      </div>
-      <!-- /container --> 
-      
-    </header>
-  </section>
-  
-  <!-- second -->
-  <section id="first" class="main">
-    <header>
-      <div class="container" >
-        <h4 class="section-title"><span>Kylin 生态圈</span></h4>
-        <div class="row">
-          <div class="col-sm-7 col-md-7">
-            <p> </p>
-            <p><b>Kylin 核心:</b> Kylin OLAP引擎基础框架,包括元数据(Metadata)引擎,查询引擎,Job引擎及存储引擎等,同时包括REST服务器以响应客户端请求</p>
-            <p><b>扩展:</b> 支持额外功能和特性的插件</p>
-            <p><b>整合:</b> 与调度系统,ETL,监控等生命周期管理系统的整合</p>
-            <p><b>用户界面:</b> 在Kylin核心之上扩展的第三方用户界面</p>
-            <p><b>驱动:</b> ODBC 和 JDBC 驱动以支持不同的工具和产品,比如Tableau</p>
-          </div>
-          <div class="col-sm-5 col-md-5">    </div>
-        </div>
-        <!-- /container --> 
-      </div>
-    </header>
-  </section>  
-</main>

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/index.md
----------------------------------------------------------------------
diff --git a/website/index.md b/website/index.md
deleted file mode 100644
index 9b64f17..0000000
--- a/website/index.md
+++ /dev/null
@@ -1,90 +0,0 @@
----
-layout: default
-title: Home
----
-
-
-<main id="main" >
-  <div class="container" >
-    <div id="zero" class=" main" >
-      <header style=" padding:2em 0 4em 0">
-        <div class="container" >
-
-          <h4 class="section-title"><span>Apache Kylin™ Overview</span></h4>
-          <div class="row" style="margin-top:-20px;">
-            <div class="col-sm-12 col-md-12">              
-              <p class="title_text"> Apache Kylin™ is an open source Distributed Analytics Engine designed to provide SQL interface and multi-dimensional analysis (OLAP) on Hadoop supporting extremely large datasets, original contributed from eBay Inc.</p>
-              <img id="diagram" src="assets/images/kylin_diagram.png"> </div>
-          </div>
-        </div>
-        <!-- /container --> 
-        
-      </header>
-    </div>
-    <!-- / section --> 
-  </div>
-  <!-- /container -->
-  
-  <section id="second" class="main">
-    <header style="background-color:#efefef;">
-      <div class="container"  >
-        <h4 class="section-title"><span> What is Kylin? </span></h4>
-        <!-- second-->
-        <div class="row">
-          <div class="col-sm-12 col-md-12">
-            <div align="left">
-              <p> <b>- Extremely Fast OLAP Engine at Scale: </b><br/>
-              <div class="indent">Kylin is designed to reduce query latency on Hadoop for 10+ billions of rows of data</div>
-              </p>
-              <p> <b>- ANSI SQL Interface on Hadoop: </b><br/>
-              <div class="indent">Kylin offers ANSI SQL on Hadoop and supports most ANSI SQL query functions</div>
-              </p>
-              <p> <b>- Interactive Query Capability: </b><br/>
-              <div class="indent">Users can interact with Hadoop data via Kylin at sub-second latency, better than Hive queries for the same dataset</div>
-              </p>
-              <p> <b>- MOLAP Cube:</b><br/>
-              <div class="indent">User can define a data model and pre-build in Kylin with more than 10+ billions of raw data records</div>
-              </p>
-              <p> <b>- Seamless Integration with BI Tools:</b><br/>
-              <div class="indent">Kylin currently offers integration capability with BI Tools like Tableau.  Integration with Microstrategy and Excel is coming soon</div>
-              </p>
-              <p> <b>- Other Highlights:</b> <br/>
-              <div class="indent">- Job Management and Monitoring <br/>
-                - Compression and Encoding Support <br/>
-                - Incremental Refresh of Cubes <br/>
-                - Leverage HBase Coprocessor for query latency <br/>
-                - Approximate Query Capability for distinct Count (HyperLogLog) <br/>
-                - Easy Web interface to manage, build, monitor and query cubes <br/>
-                - Security capability to set ACL at Cube/Project Level <br/>
-                - Support LDAP Integration </div>
-              </p>
-            </div>
-          </div>
-        </div>
-      </div>
-      <!-- /container --> 
-      
-    </header>
-  </section>
-  
-  <!-- second -->
-  <section id="first" class="main">
-    <header>
-      <div class="container" >
-        <h4 class="section-title"><span>Kylin Ecosystem</span></h4>
-        <div class="row">
-          <div class="col-sm-7 col-md-7">
-            <p> </p>
-            <p><b>Kylin Core:</b> Fundamental framework of Kylin OLAP Engine comprises of Metadata Engine, Query Engine, Job Engine and Storage Engine to run the entire stack. It also includes a REST Server to service client requests</p>
-            <p><b>Extensions:</b> Plugins to support additional functions and features </p>
-            <p><b>Integration:</b> Lifecycle Management Support to integrate with Job Scheduler,  ETL, Monitoring and Alerting Systems </p>
-            <p><b>User Interface:</b> Allows third party users to build customized user-interface atop Kylin core</p>
-            <p><b>Drivers:</b> ODBC and JDBC drivers to support different tools and products, such as Tableau</p>
-          </div>
-          <div class="col-sm-5 col-md-5"> <img id="core" src="assets/images/core.png"> </div>
-        </div>
-        <!-- /container --> 
-      </div>
-    </header>
-  </section>  
-</main>


[36/52] [abbrv] kylin git commit: KYLIN-1416 remove website from coding branch

Posted by li...@apache.org.
http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/assets/css/docs.css
----------------------------------------------------------------------
diff --git a/website/assets/css/docs.css b/website/assets/css/docs.css
deleted file mode 100755
index eeb7314..0000000
--- a/website/assets/css/docs.css
+++ /dev/null
@@ -1,126 +0,0 @@
-.aside {
-  height: 100%;
-  background-color:#fff;
-}
-
-.aside1 {
-  background-color: #fff;
-  height: 100%;
-  padding-left: 0px;
-  padding-right: 0px;
-  text-align: left;
-}
-
-.aside1 > .nav > li > a {
-  border-radius: 0px;
-  color: #454545;
-  font-weight: 600;
-  font-family: 'Open sans';
-  font-size: 15px;
-  padding: 0px 30px 5px 0px;
-}
-
-.aside1> .nav > li > a:hover {
-  color: #000000;
-  background-color: #ffffff;
-}
-
-.aside1 > .nav-pills>li.active>a ,.nav-pills>li.active>a:hover,.nav-pills>li.active>a:focus {
-  color: #000000;
-  background-color: #ffffff;
-}
-
-.list-group {
-  font-family: 'Open sans';
-}
-
-
-.aside2 {
-  height: 100%;
-  overflow-y: auto;
-  background-color: #fff;
-  overflow-x: hidden;
-  padding: 8px 0px 0px 60px;
-}
-
-.post-title {
-  font-family: 'Open sans';
-  font-size: 30px;
-}
-
-.post-meta {
-  font-family: 'Open sans';
-}
-
-.post-content {
-  font-family: 'Open sans';
-}
-
-.post-content> h1  {
-  font-family: 'Open sans';
-  font-size: 28px;
-}
-
-.post-content> h2  {
-  font-family: 'Open sans';
-  font-size: 24px;
-}
-
-.post-content> h3  {
-  font-family: 'Open sans';
-  font-size: 20px;
-}
-
-.post-content> h4  {
-  font-family: 'Open sans';
-  font-size: 18px;
-}
-
-.post-content> h5  {
-  font-family: 'Open sans';
-  font-size: 16px;
-}
-
-.post-content> h6  {
-  font-family: 'Open sans';
-  font-size: 16px;
-  color: #666666;
-}
-
-.post-link {
-  font-family: 'Open sans';
-  color: #454545;
-  font-size: 20px;
-}
-
-.nside {
-  color: #fb8d08 !important;
-}
-
-.nnav {
-  font-weight: 600;
-}
-
-.nnav-home {
-  font-weight: 200;
-}
-
-#navtitle {
-  color:#454545;
-}
-
-#navtitle:hover, #navtitle:focus{
-  color:#000000;
-}
-
-#navlist {
-  color:#454545;
-}
-
-#navlist:hover, #navlist:focus{
-  color:#000000;
-}
-
-
-
-

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/assets/css/font-awesome.min.css
----------------------------------------------------------------------
diff --git a/website/assets/css/font-awesome.min.css b/website/assets/css/font-awesome.min.css
deleted file mode 100644
index ec53d4d..0000000
--- a/website/assets/css/font-awesome.min.css
+++ /dev/null
@@ -1,4 +0,0 @@
-/*!
- *  Font Awesome 4.2.0 by @davegandy - http://fontawesome.io - @fontawesome
- *  License - http://fontawesome.io/license (Font: SIL OFL 1.1, CSS: MIT License)
- */@font-face{font-family:'FontAwesome';src:url('../fonts/fontawesome-webfont.eot?v=4.2.0');src:url('../fonts/fontawesome-webfont.eot?#iefix&v=4.2.0') format('embedded-opentype'),url('../fonts/fontawesome-webfont.woff?v=4.2.0') format('woff'),url('../fonts/fontawesome-webfont.ttf?v=4.2.0') format('truetype'),url('../fonts/fontawesome-webfont.svg?v=4.2.0#fontawesomeregular') format('svg');font-weight:normal;font-style:normal}.fa{display:inline-block;font:normal normal normal 14px/1 FontAwesome;font-size:inherit;text-rendering:auto;-webkit-font-smoothing:antialiased;-moz-osx-font-smoothing:grayscale}.fa-lg{font-size:1.33333333em;line-height:.75em;vertical-align:-15%}.fa-2x{font-size:2em}.fa-3x{font-size:3em}.fa-4x{font-size:4em}.fa-5x{font-size:5em}.fa-fw{width:1.28571429em;text-align:center}.fa-ul{padding-left:0;margin-left:2.14285714em;list-style-type:none}.fa-ul>li{position:relative}.fa-li{position:absolute;left:-2.14285714em;width:2.14285714em;top:.14285714em;text-align:center}.fa
 -li.fa-lg{left:-1.85714286em}.fa-border{padding:.2em .25em .15em;border:solid .08em #eee;border-radius:.1em}.pull-right{float:right}.pull-left{float:left}.fa.pull-left{margin-right:.3em}.fa.pull-right{margin-left:.3em}.fa-spin{-webkit-animation:fa-spin 2s infinite linear;animation:fa-spin 2s infinite linear}@-webkit-keyframes fa-spin{0%{-webkit-transform:rotate(0deg);transform:rotate(0deg)}100%{-webkit-transform:rotate(359deg);transform:rotate(359deg)}}@keyframes fa-spin{0%{-webkit-transform:rotate(0deg);transform:rotate(0deg)}100%{-webkit-transform:rotate(359deg);transform:rotate(359deg)}}.fa-rotate-90{filter:progid:DXImageTransform.Microsoft.BasicImage(rotation=1);-webkit-transform:rotate(90deg);-ms-transform:rotate(90deg);transform:rotate(90deg)}.fa-rotate-180{filter:progid:DXImageTransform.Microsoft.BasicImage(rotation=2);-webkit-transform:rotate(180deg);-ms-transform:rotate(180deg);transform:rotate(180deg)}.fa-rotate-270{filter:progid:DXImageTransform.Microsoft.BasicImage(rotat
 ion=3);-webkit-transform:rotate(270deg);-ms-transform:rotate(270deg);transform:rotate(270deg)}.fa-flip-horizontal{filter:progid:DXImageTransform.Microsoft.BasicImage(rotation=0, mirror=1);-webkit-transform:scale(-1, 1);-ms-transform:scale(-1, 1);transform:scale(-1, 1)}.fa-flip-vertical{filter:progid:DXImageTransform.Microsoft.BasicImage(rotation=2, mirror=1);-webkit-transform:scale(1, -1);-ms-transform:scale(1, -1);transform:scale(1, -1)}:root .fa-rotate-90,:root .fa-rotate-180,:root .fa-rotate-270,:root .fa-flip-horizontal,:root .fa-flip-vertical{filter:none}.fa-stack{position:relative;display:inline-block;width:2em;height:2em;line-height:2em;vertical-align:middle}.fa-stack-1x,.fa-stack-2x{position:absolute;left:0;width:100%;text-align:center}.fa-stack-1x{line-height:inherit}.fa-stack-2x{font-size:2em}.fa-inverse{color:#fff}.fa-glass:before{content:"\f000"}.fa-music:before{content:"\f001"}.fa-search:before{content:"\f002"}.fa-envelope-o:before{content:"\f003"}.fa-heart:before{conte
 nt:"\f004"}.fa-star:before{content:"\f005"}.fa-star-o:before{content:"\f006"}.fa-user:before{content:"\f007"}.fa-film:before{content:"\f008"}.fa-th-large:before{content:"\f009"}.fa-th:before{content:"\f00a"}.fa-th-list:before{content:"\f00b"}.fa-check:before{content:"\f00c"}.fa-remove:before,.fa-close:before,.fa-times:before{content:"\f00d"}.fa-search-plus:before{content:"\f00e"}.fa-search-minus:before{content:"\f010"}.fa-power-off:before{content:"\f011"}.fa-signal:before{content:"\f012"}.fa-gear:before,.fa-cog:before{content:"\f013"}.fa-trash-o:before{content:"\f014"}.fa-home:before{content:"\f015"}.fa-file-o:before{content:"\f016"}.fa-clock-o:before{content:"\f017"}.fa-road:before{content:"\f018"}.fa-download:before{content:"\f019"}.fa-arrow-circle-o-down:before{content:"\f01a"}.fa-arrow-circle-o-up:before{content:"\f01b"}.fa-inbox:before{content:"\f01c"}.fa-play-circle-o:before{content:"\f01d"}.fa-rotate-right:before,.fa-repeat:before{content:"\f01e"}.fa-refresh:before{content:"\
 f021"}.fa-list-alt:before{content:"\f022"}.fa-lock:before{content:"\f023"}.fa-flag:before{content:"\f024"}.fa-headphones:before{content:"\f025"}.fa-volume-off:before{content:"\f026"}.fa-volume-down:before{content:"\f027"}.fa-volume-up:before{content:"\f028"}.fa-qrcode:before{content:"\f029"}.fa-barcode:before{content:"\f02a"}.fa-tag:before{content:"\f02b"}.fa-tags:before{content:"\f02c"}.fa-book:before{content:"\f02d"}.fa-bookmark:before{content:"\f02e"}.fa-print:before{content:"\f02f"}.fa-camera:before{content:"\f030"}.fa-font:before{content:"\f031"}.fa-bold:before{content:"\f032"}.fa-italic:before{content:"\f033"}.fa-text-height:before{content:"\f034"}.fa-text-width:before{content:"\f035"}.fa-align-left:before{content:"\f036"}.fa-align-center:before{content:"\f037"}.fa-align-right:before{content:"\f038"}.fa-align-justify:before{content:"\f039"}.fa-list:before{content:"\f03a"}.fa-dedent:before,.fa-outdent:before{content:"\f03b"}.fa-indent:before{content:"\f03c"}.fa-video-camera:bef
 ore{content:"\f03d"}.fa-photo:before,.fa-image:before,.fa-picture-o:before{content:"\f03e"}.fa-pencil:before{content:"\f040"}.fa-map-marker:before{content:"\f041"}.fa-adjust:before{content:"\f042"}.fa-tint:before{content:"\f043"}.fa-edit:before,.fa-pencil-square-o:before{content:"\f044"}.fa-share-square-o:before{content:"\f045"}.fa-check-square-o:before{content:"\f046"}.fa-arrows:before{content:"\f047"}.fa-step-backward:before{content:"\f048"}.fa-fast-backward:before{content:"\f049"}.fa-backward:before{content:"\f04a"}.fa-play:before{content:"\f04b"}.fa-pause:before{content:"\f04c"}.fa-stop:before{content:"\f04d"}.fa-forward:before{content:"\f04e"}.fa-fast-forward:before{content:"\f050"}.fa-step-forward:before{content:"\f051"}.fa-eject:before{content:"\f052"}.fa-chevron-left:before{content:"\f053"}.fa-chevron-right:before{content:"\f054"}.fa-plus-circle:before{content:"\f055"}.fa-minus-circle:before{content:"\f056"}.fa-times-circle:before{content:"\f057"}.fa-check-circle:before{cont
 ent:"\f058"}.fa-question-circle:before{content:"\f059"}.fa-info-circle:before{content:"\f05a"}.fa-crosshairs:before{content:"\f05b"}.fa-times-circle-o:before{content:"\f05c"}.fa-check-circle-o:before{content:"\f05d"}.fa-ban:before{content:"\f05e"}.fa-arrow-left:before{content:"\f060"}.fa-arrow-right:before{content:"\f061"}.fa-arrow-up:before{content:"\f062"}.fa-arrow-down:before{content:"\f063"}.fa-mail-forward:before,.fa-share:before{content:"\f064"}.fa-expand:before{content:"\f065"}.fa-compress:before{content:"\f066"}.fa-plus:before{content:"\f067"}.fa-minus:before{content:"\f068"}.fa-asterisk:before{content:"\f069"}.fa-exclamation-circle:before{content:"\f06a"}.fa-gift:before{content:"\f06b"}.fa-leaf:before{content:"\f06c"}.fa-fire:before{content:"\f06d"}.fa-eye:before{content:"\f06e"}.fa-eye-slash:before{content:"\f070"}.fa-warning:before,.fa-exclamation-triangle:before{content:"\f071"}.fa-plane:before{content:"\f072"}.fa-calendar:before{content:"\f073"}.fa-random:before{content
 :"\f074"}.fa-comment:before{content:"\f075"}.fa-magnet:before{content:"\f076"}.fa-chevron-up:before{content:"\f077"}.fa-chevron-down:before{content:"\f078"}.fa-retweet:before{content:"\f079"}.fa-shopping-cart:before{content:"\f07a"}.fa-folder:before{content:"\f07b"}.fa-folder-open:before{content:"\f07c"}.fa-arrows-v:before{content:"\f07d"}.fa-arrows-h:before{content:"\f07e"}.fa-bar-chart-o:before,.fa-bar-chart:before{content:"\f080"}.fa-twitter-square:before{content:"\f081"}.fa-facebook-square:before{content:"\f082"}.fa-camera-retro:before{content:"\f083"}.fa-key:before{content:"\f084"}.fa-gears:before,.fa-cogs:before{content:"\f085"}.fa-comments:before{content:"\f086"}.fa-thumbs-o-up:before{content:"\f087"}.fa-thumbs-o-down:before{content:"\f088"}.fa-star-half:before{content:"\f089"}.fa-heart-o:before{content:"\f08a"}.fa-sign-out:before{content:"\f08b"}.fa-linkedin-square:before{content:"\f08c"}.fa-thumb-tack:before{content:"\f08d"}.fa-external-link:before{content:"\f08e"}.fa-sign-
 in:before{content:"\f090"}.fa-trophy:before{content:"\f091"}.fa-github-square:before{content:"\f092"}.fa-upload:before{content:"\f093"}.fa-lemon-o:before{content:"\f094"}.fa-phone:before{content:"\f095"}.fa-square-o:before{content:"\f096"}.fa-bookmark-o:before{content:"\f097"}.fa-phone-square:before{content:"\f098"}.fa-twitter:before{content:"\f099"}.fa-facebook:before{content:"\f09a"}.fa-github:before{content:"\f09b"}.fa-unlock:before{content:"\f09c"}.fa-credit-card:before{content:"\f09d"}.fa-rss:before{content:"\f09e"}.fa-hdd-o:before{content:"\f0a0"}.fa-bullhorn:before{content:"\f0a1"}.fa-bell:before{content:"\f0f3"}.fa-certificate:before{content:"\f0a3"}.fa-hand-o-right:before{content:"\f0a4"}.fa-hand-o-left:before{content:"\f0a5"}.fa-hand-o-up:before{content:"\f0a6"}.fa-hand-o-down:before{content:"\f0a7"}.fa-arrow-circle-left:before{content:"\f0a8"}.fa-arrow-circle-right:before{content:"\f0a9"}.fa-arrow-circle-up:before{content:"\f0aa"}.fa-arrow-circle-down:before{content:"\f0a
 b"}.fa-globe:before{content:"\f0ac"}.fa-wrench:before{content:"\f0ad"}.fa-tasks:before{content:"\f0ae"}.fa-filter:before{content:"\f0b0"}.fa-briefcase:before{content:"\f0b1"}.fa-arrows-alt:before{content:"\f0b2"}.fa-group:before,.fa-users:before{content:"\f0c0"}.fa-chain:before,.fa-link:before{content:"\f0c1"}.fa-cloud:before{content:"\f0c2"}.fa-flask:before{content:"\f0c3"}.fa-cut:before,.fa-scissors:before{content:"\f0c4"}.fa-copy:before,.fa-files-o:before{content:"\f0c5"}.fa-paperclip:before{content:"\f0c6"}.fa-save:before,.fa-floppy-o:before{content:"\f0c7"}.fa-square:before{content:"\f0c8"}.fa-navicon:before,.fa-reorder:before,.fa-bars:before{content:"\f0c9"}.fa-list-ul:before{content:"\f0ca"}.fa-list-ol:before{content:"\f0cb"}.fa-strikethrough:before{content:"\f0cc"}.fa-underline:before{content:"\f0cd"}.fa-table:before{content:"\f0ce"}.fa-magic:before{content:"\f0d0"}.fa-truck:before{content:"\f0d1"}.fa-pinterest:before{content:"\f0d2"}.fa-pinterest-square:before{content:"\f0d
 3"}.fa-google-plus-square:before{content:"\f0d4"}.fa-google-plus:before{content:"\f0d5"}.fa-money:before{content:"\f0d6"}.fa-caret-down:before{content:"\f0d7"}.fa-caret-up:before{content:"\f0d8"}.fa-caret-left:before{content:"\f0d9"}.fa-caret-right:before{content:"\f0da"}.fa-columns:before{content:"\f0db"}.fa-unsorted:before,.fa-sort:before{content:"\f0dc"}.fa-sort-down:before,.fa-sort-desc:before{content:"\f0dd"}.fa-sort-up:before,.fa-sort-asc:before{content:"\f0de"}.fa-envelope:before{content:"\f0e0"}.fa-linkedin:before{content:"\f0e1"}.fa-rotate-left:before,.fa-undo:before{content:"\f0e2"}.fa-legal:before,.fa-gavel:before{content:"\f0e3"}.fa-dashboard:before,.fa-tachometer:before{content:"\f0e4"}.fa-comment-o:before{content:"\f0e5"}.fa-comments-o:before{content:"\f0e6"}.fa-flash:before,.fa-bolt:before{content:"\f0e7"}.fa-sitemap:before{content:"\f0e8"}.fa-umbrella:before{content:"\f0e9"}.fa-paste:before,.fa-clipboard:before{content:"\f0ea"}.fa-lightbulb-o:before{content:"\f0eb"}.
 fa-exchange:before{content:"\f0ec"}.fa-cloud-download:before{content:"\f0ed"}.fa-cloud-upload:before{content:"\f0ee"}.fa-user-md:before{content:"\f0f0"}.fa-stethoscope:before{content:"\f0f1"}.fa-suitcase:before{content:"\f0f2"}.fa-bell-o:before{content:"\f0a2"}.fa-coffee:before{content:"\f0f4"}.fa-cutlery:before{content:"\f0f5"}.fa-file-text-o:before{content:"\f0f6"}.fa-building-o:before{content:"\f0f7"}.fa-hospital-o:before{content:"\f0f8"}.fa-ambulance:before{content:"\f0f9"}.fa-medkit:before{content:"\f0fa"}.fa-fighter-jet:before{content:"\f0fb"}.fa-beer:before{content:"\f0fc"}.fa-h-square:before{content:"\f0fd"}.fa-plus-square:before{content:"\f0fe"}.fa-angle-double-left:before{content:"\f100"}.fa-angle-double-right:before{content:"\f101"}.fa-angle-double-up:before{content:"\f102"}.fa-angle-double-down:before{content:"\f103"}.fa-angle-left:before{content:"\f104"}.fa-angle-right:before{content:"\f105"}.fa-angle-up:before{content:"\f106"}.fa-angle-down:before{content:"\f107"}.fa-d
 esktop:before{content:"\f108"}.fa-laptop:before{content:"\f109"}.fa-tablet:before{content:"\f10a"}.fa-mobile-phone:before,.fa-mobile:before{content:"\f10b"}.fa-circle-o:before{content:"\f10c"}.fa-quote-left:before{content:"\f10d"}.fa-quote-right:before{content:"\f10e"}.fa-spinner:before{content:"\f110"}.fa-circle:before{content:"\f111"}.fa-mail-reply:before,.fa-reply:before{content:"\f112"}.fa-github-alt:before{content:"\f113"}.fa-folder-o:before{content:"\f114"}.fa-folder-open-o:before{content:"\f115"}.fa-smile-o:before{content:"\f118"}.fa-frown-o:before{content:"\f119"}.fa-meh-o:before{content:"\f11a"}.fa-gamepad:before{content:"\f11b"}.fa-keyboard-o:before{content:"\f11c"}.fa-flag-o:before{content:"\f11d"}.fa-flag-checkered:before{content:"\f11e"}.fa-terminal:before{content:"\f120"}.fa-code:before{content:"\f121"}.fa-mail-reply-all:before,.fa-reply-all:before{content:"\f122"}.fa-star-half-empty:before,.fa-star-half-full:before,.fa-star-half-o:before{content:"\f123"}.fa-location-a
 rrow:before{content:"\f124"}.fa-crop:before{content:"\f125"}.fa-code-fork:before{content:"\f126"}.fa-unlink:before,.fa-chain-broken:before{content:"\f127"}.fa-question:before{content:"\f128"}.fa-info:before{content:"\f129"}.fa-exclamation:before{content:"\f12a"}.fa-superscript:before{content:"\f12b"}.fa-subscript:before{content:"\f12c"}.fa-eraser:before{content:"\f12d"}.fa-puzzle-piece:before{content:"\f12e"}.fa-microphone:before{content:"\f130"}.fa-microphone-slash:before{content:"\f131"}.fa-shield:before{content:"\f132"}.fa-calendar-o:before{content:"\f133"}.fa-fire-extinguisher:before{content:"\f134"}.fa-rocket:before{content:"\f135"}.fa-maxcdn:before{content:"\f136"}.fa-chevron-circle-left:before{content:"\f137"}.fa-chevron-circle-right:before{content:"\f138"}.fa-chevron-circle-up:before{content:"\f139"}.fa-chevron-circle-down:before{content:"\f13a"}.fa-html5:before{content:"\f13b"}.fa-css3:before{content:"\f13c"}.fa-anchor:before{content:"\f13d"}.fa-unlock-alt:before{content:"\
 f13e"}.fa-bullseye:before{content:"\f140"}.fa-ellipsis-h:before{content:"\f141"}.fa-ellipsis-v:before{content:"\f142"}.fa-rss-square:before{content:"\f143"}.fa-play-circle:before{content:"\f144"}.fa-ticket:before{content:"\f145"}.fa-minus-square:before{content:"\f146"}.fa-minus-square-o:before{content:"\f147"}.fa-level-up:before{content:"\f148"}.fa-level-down:before{content:"\f149"}.fa-check-square:before{content:"\f14a"}.fa-pencil-square:before{content:"\f14b"}.fa-external-link-square:before{content:"\f14c"}.fa-share-square:before{content:"\f14d"}.fa-compass:before{content:"\f14e"}.fa-toggle-down:before,.fa-caret-square-o-down:before{content:"\f150"}.fa-toggle-up:before,.fa-caret-square-o-up:before{content:"\f151"}.fa-toggle-right:before,.fa-caret-square-o-right:before{content:"\f152"}.fa-euro:before,.fa-eur:before{content:"\f153"}.fa-gbp:before{content:"\f154"}.fa-dollar:before,.fa-usd:before{content:"\f155"}.fa-rupee:before,.fa-inr:before{content:"\f156"}.fa-cny:before,.fa-rmb:be
 fore,.fa-yen:before,.fa-jpy:before{content:"\f157"}.fa-ruble:before,.fa-rouble:before,.fa-rub:before{content:"\f158"}.fa-won:before,.fa-krw:before{content:"\f159"}.fa-bitcoin:before,.fa-btc:before{content:"\f15a"}.fa-file:before{content:"\f15b"}.fa-file-text:before{content:"\f15c"}.fa-sort-alpha-asc:before{content:"\f15d"}.fa-sort-alpha-desc:before{content:"\f15e"}.fa-sort-amount-asc:before{content:"\f160"}.fa-sort-amount-desc:before{content:"\f161"}.fa-sort-numeric-asc:before{content:"\f162"}.fa-sort-numeric-desc:before{content:"\f163"}.fa-thumbs-up:before{content:"\f164"}.fa-thumbs-down:before{content:"\f165"}.fa-youtube-square:before{content:"\f166"}.fa-youtube:before{content:"\f167"}.fa-xing:before{content:"\f168"}.fa-xing-square:before{content:"\f169"}.fa-youtube-play:before{content:"\f16a"}.fa-dropbox:before{content:"\f16b"}.fa-stack-overflow:before{content:"\f16c"}.fa-instagram:before{content:"\f16d"}.fa-flickr:before{content:"\f16e"}.fa-adn:before{content:"\f170"}.fa-bitbuck
 et:before{content:"\f171"}.fa-bitbucket-square:before{content:"\f172"}.fa-tumblr:before{content:"\f173"}.fa-tumblr-square:before{content:"\f174"}.fa-long-arrow-down:before{content:"\f175"}.fa-long-arrow-up:before{content:"\f176"}.fa-long-arrow-left:before{content:"\f177"}.fa-long-arrow-right:before{content:"\f178"}.fa-apple:before{content:"\f179"}.fa-windows:before{content:"\f17a"}.fa-android:before{content:"\f17b"}.fa-linux:before{content:"\f17c"}.fa-dribbble:before{content:"\f17d"}.fa-skype:before{content:"\f17e"}.fa-foursquare:before{content:"\f180"}.fa-trello:before{content:"\f181"}.fa-female:before{content:"\f182"}.fa-male:before{content:"\f183"}.fa-gittip:before{content:"\f184"}.fa-sun-o:before{content:"\f185"}.fa-moon-o:before{content:"\f186"}.fa-archive:before{content:"\f187"}.fa-bug:before{content:"\f188"}.fa-vk:before{content:"\f189"}.fa-weibo:before{content:"\f18a"}.fa-renren:before{content:"\f18b"}.fa-pagelines:before{content:"\f18c"}.fa-stack-exchange:before{content:"\f
 18d"}.fa-arrow-circle-o-right:before{content:"\f18e"}.fa-arrow-circle-o-left:before{content:"\f190"}.fa-toggle-left:before,.fa-caret-square-o-left:before{content:"\f191"}.fa-dot-circle-o:before{content:"\f192"}.fa-wheelchair:before{content:"\f193"}.fa-vimeo-square:before{content:"\f194"}.fa-turkish-lira:before,.fa-try:before{content:"\f195"}.fa-plus-square-o:before{content:"\f196"}.fa-space-shuttle:before{content:"\f197"}.fa-slack:before{content:"\f198"}.fa-envelope-square:before{content:"\f199"}.fa-wordpress:before{content:"\f19a"}.fa-openid:before{content:"\f19b"}.fa-institution:before,.fa-bank:before,.fa-university:before{content:"\f19c"}.fa-mortar-board:before,.fa-graduation-cap:before{content:"\f19d"}.fa-yahoo:before{content:"\f19e"}.fa-google:before{content:"\f1a0"}.fa-reddit:before{content:"\f1a1"}.fa-reddit-square:before{content:"\f1a2"}.fa-stumbleupon-circle:before{content:"\f1a3"}.fa-stumbleupon:before{content:"\f1a4"}.fa-delicious:before{content:"\f1a5"}.fa-digg:before{co
 ntent:"\f1a6"}.fa-pied-piper:before{content:"\f1a7"}.fa-pied-piper-alt:before{content:"\f1a8"}.fa-drupal:before{content:"\f1a9"}.fa-joomla:before{content:"\f1aa"}.fa-language:before{content:"\f1ab"}.fa-fax:before{content:"\f1ac"}.fa-building:before{content:"\f1ad"}.fa-child:before{content:"\f1ae"}.fa-paw:before{content:"\f1b0"}.fa-spoon:before{content:"\f1b1"}.fa-cube:before{content:"\f1b2"}.fa-cubes:before{content:"\f1b3"}.fa-behance:before{content:"\f1b4"}.fa-behance-square:before{content:"\f1b5"}.fa-steam:before{content:"\f1b6"}.fa-steam-square:before{content:"\f1b7"}.fa-recycle:before{content:"\f1b8"}.fa-automobile:before,.fa-car:before{content:"\f1b9"}.fa-cab:before,.fa-taxi:before{content:"\f1ba"}.fa-tree:before{content:"\f1bb"}.fa-spotify:before{content:"\f1bc"}.fa-deviantart:before{content:"\f1bd"}.fa-soundcloud:before{content:"\f1be"}.fa-database:before{content:"\f1c0"}.fa-file-pdf-o:before{content:"\f1c1"}.fa-file-word-o:before{content:"\f1c2"}.fa-file-excel-o:before{conte
 nt:"\f1c3"}.fa-file-powerpoint-o:before{content:"\f1c4"}.fa-file-photo-o:before,.fa-file-picture-o:before,.fa-file-image-o:before{content:"\f1c5"}.fa-file-zip-o:before,.fa-file-archive-o:before{content:"\f1c6"}.fa-file-sound-o:before,.fa-file-audio-o:before{content:"\f1c7"}.fa-file-movie-o:before,.fa-file-video-o:before{content:"\f1c8"}.fa-file-code-o:before{content:"\f1c9"}.fa-vine:before{content:"\f1ca"}.fa-codepen:before{content:"\f1cb"}.fa-jsfiddle:before{content:"\f1cc"}.fa-life-bouy:before,.fa-life-buoy:before,.fa-life-saver:before,.fa-support:before,.fa-life-ring:before{content:"\f1cd"}.fa-circle-o-notch:before{content:"\f1ce"}.fa-ra:before,.fa-rebel:before{content:"\f1d0"}.fa-ge:before,.fa-empire:before{content:"\f1d1"}.fa-git-square:before{content:"\f1d2"}.fa-git:before{content:"\f1d3"}.fa-hacker-news:before{content:"\f1d4"}.fa-tencent-weibo:before{content:"\f1d5"}.fa-qq:before{content:"\f1d6"}.fa-wechat:before,.fa-weixin:before{content:"\f1d7"}.fa-send:before,.fa-paper-pla
 ne:before{content:"\f1d8"}.fa-send-o:before,.fa-paper-plane-o:before{content:"\f1d9"}.fa-history:before{content:"\f1da"}.fa-circle-thin:before{content:"\f1db"}.fa-header:before{content:"\f1dc"}.fa-paragraph:before{content:"\f1dd"}.fa-sliders:before{content:"\f1de"}.fa-share-alt:before{content:"\f1e0"}.fa-share-alt-square:before{content:"\f1e1"}.fa-bomb:before{content:"\f1e2"}.fa-soccer-ball-o:before,.fa-futbol-o:before{content:"\f1e3"}.fa-tty:before{content:"\f1e4"}.fa-binoculars:before{content:"\f1e5"}.fa-plug:before{content:"\f1e6"}.fa-slideshare:before{content:"\f1e7"}.fa-twitch:before{content:"\f1e8"}.fa-yelp:before{content:"\f1e9"}.fa-newspaper-o:before{content:"\f1ea"}.fa-wifi:before{content:"\f1eb"}.fa-calculator:before{content:"\f1ec"}.fa-paypal:before{content:"\f1ed"}.fa-google-wallet:before{content:"\f1ee"}.fa-cc-visa:before{content:"\f1f0"}.fa-cc-mastercard:before{content:"\f1f1"}.fa-cc-discover:before{content:"\f1f2"}.fa-cc-amex:before{content:"\f1f3"}.fa-cc-paypal:befor
 e{content:"\f1f4"}.fa-cc-stripe:before{content:"\f1f5"}.fa-bell-slash:before{content:"\f1f6"}.fa-bell-slash-o:before{content:"\f1f7"}.fa-trash:before{content:"\f1f8"}.fa-copyright:before{content:"\f1f9"}.fa-at:before{content:"\f1fa"}.fa-eyedropper:before{content:"\f1fb"}.fa-paint-brush:before{content:"\f1fc"}.fa-birthday-cake:before{content:"\f1fd"}.fa-area-chart:before{content:"\f1fe"}.fa-pie-chart:before{content:"\f200"}.fa-line-chart:before{content:"\f201"}.fa-lastfm:before{content:"\f202"}.fa-lastfm-square:before{content:"\f203"}.fa-toggle-off:before{content:"\f204"}.fa-toggle-on:before{content:"\f205"}.fa-bicycle:before{content:"\f206"}.fa-bus:before{content:"\f207"}.fa-ioxhost:before{content:"\f208"}.fa-angellist:before{content:"\f209"}.fa-cc:before{content:"\f20a"}.fa-shekel:before,.fa-sheqel:before,.fa-ils:before{content:"\f20b"}.fa-meanpath:before{content:"\f20c"}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/assets/css/pygments.css
----------------------------------------------------------------------
diff --git a/website/assets/css/pygments.css b/website/assets/css/pygments.css
deleted file mode 100644
index 122b429..0000000
--- a/website/assets/css/pygments.css
+++ /dev/null
@@ -1,61 +0,0 @@
-.hll { background-color: #ffffcc }
-.c { color: #408080; font-style: italic } /* Comment */
-.err { border: 1px solid #FF0000 } /* Error */
-.k { color: #008000; font-weight: bold } /* Keyword */
-.o { color: #666666 } /* Operator */
-.cm { color: #408080; font-style: italic } /* Comment.Multiline */
-.cp { color: #BC7A00 } /* Comment.Preproc */
-.c1 { color: #408080; font-style: italic } /* Comment.Single */
-.cs { color: #408080; font-style: italic } /* Comment.Special */
-.gd { color: #A00000 } /* Generic.Deleted */
-.ge { font-style: italic } /* Generic.Emph */
-.gr { color: #FF0000 } /* Generic.Error */
-.gh { color: #000080; font-weight: bold } /* Generic.Heading */
-.gi { color: #00A000 } /* Generic.Inserted */
-.go { color: #808080 } /* Generic.Output */
-.gp { color: #000080; font-weight: bold } /* Generic.Prompt */
-.gs { font-weight: bold } /* Generic.Strong */
-.gu { color: #800080; font-weight: bold } /* Generic.Subheading */
-.gt { color: #0040D0 } /* Generic.Traceback */
-.kc { color: #008000; font-weight: bold } /* Keyword.Constant */
-.kd { color: #008000; font-weight: bold } /* Keyword.Declaration */
-.kn { color: #008000; font-weight: bold } /* Keyword.Namespace */
-.kp { color: #008000 } /* Keyword.Pseudo */
-.kr { color: #008000; font-weight: bold } /* Keyword.Reserved */
-.kt { color: #B00040 } /* Keyword.Type */
-.m { color: #666666 } /* Literal.Number */
-.s { color: #BA2121 } /* Literal.String */
-.na { color: #7D9029 } /* Name.Attribute */
-.nb { color: #008000 } /* Name.Builtin */
-.nc { color: #0000FF; font-weight: bold } /* Name.Class */
-.no { color: #880000 } /* Name.Constant */
-.nd { color: #AA22FF } /* Name.Decorator */
-.ni { color: #999999; font-weight: bold } /* Name.Entity */
-.ne { color: #D2413A; font-weight: bold } /* Name.Exception */
-.nf { color: #0000FF } /* Name.Function */
-.nl { color: #A0A000 } /* Name.Label */
-.nn { color: #0000FF; font-weight: bold } /* Name.Namespace */
-.nt { color: #008000; font-weight: bold } /* Name.Tag */
-.nv { color: #19177C } /* Name.Variable */
-.ow { color: #AA22FF; font-weight: bold } /* Operator.Word */
-.w { color: #bbbbbb } /* Text.Whitespace */
-.mf { color: #666666 } /* Literal.Number.Float */
-.mh { color: #666666 } /* Literal.Number.Hex */
-.mi { color: #666666 } /* Literal.Number.Integer */
-.mo { color: #666666 } /* Literal.Number.Oct */
-.sb { color: #BA2121 } /* Literal.String.Backtick */
-.sc { color: #BA2121 } /* Literal.String.Char */
-.sd { color: #BA2121; font-style: italic } /* Literal.String.Doc */
-.s2 { color: #BA2121 } /* Literal.String.Double */
-.se { color: #BB6622; font-weight: bold } /* Literal.String.Escape */
-.sh { color: #BA2121 } /* Literal.String.Heredoc */
-.si { color: #BB6688; font-weight: bold } /* Literal.String.Interpol */
-.sx { color: #008000 } /* Literal.String.Other */
-.sr { color: #BB6688 } /* Literal.String.Regex */
-.s1 { color: #BA2121 } /* Literal.String.Single */
-.ss { color: #19177C } /* Literal.String.Symbol */
-.bp { color: #008000 } /* Name.Builtin.Pseudo */
-.vc { color: #19177C } /* Name.Variable.Class */
-.vg { color: #19177C } /* Name.Variable.Global */
-.vi { color: #19177C } /* Name.Variable.Instance */
-.il { color: #666666 } /* Literal.Number.Integer.Long */

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/assets/css/styles.css
----------------------------------------------------------------------
diff --git a/website/assets/css/styles.css b/website/assets/css/styles.css
deleted file mode 100644
index 8d7b498..0000000
--- a/website/assets/css/styles.css
+++ /dev/null
@@ -1,656 +0,0 @@
-@font-face {
-	font-family: 'HelveticaNeueLight';
-	src: url('../fonts/helveticaneue_lt.eot');
-	src: url('../fonts/helveticaneue_lt.eot') format('embedded-opentype'),  url('../fonts/helveticaneue_lt.woff') format('woff'),  url('../fonts/helveticaneue_lt.ttf') format('truetype'),  url('../fonts/helveticaneue_lt.svg#HelveticaNeueLight') format('svg');
-}
-@font-face {
-	font-family: 'HelveticaNeueRegular';
-	src: url('../fonts/helveticaneue-webfont.eot');
-	src: url('../fonts/helveticaneue-webfont.eot') format('embedded-opentype'),  url('../fonts/helveticaneue-webfont.woff') format('woff'),  url('../fonts/helveticaneue-webfont.ttf') format('truetype'),  url('../fonts/helveticaneue-webfont.svg#HelveticaNeueRegular') format('svg');
-}
-/*@font-face {
-    font-family: 'open_sanslight';
-    src: url('../fonts/opensans-light-webfont.eot');
-    src: url('../fonts/opensans-light-webfont.eot?#iefix') format('embedded-opentype'),
-         url('../fonts/opensans-light-webfont.woff') format('woff'),
-         url('../fonts/opensans-light-webfont.ttf') format('truetype'),
-         url('../fonts/opensans-light-webfont.svg#open_sanslight') format('svg');
-    font-weight: normal;
-    font-style: normal;
-
-}
-
-@font-face {
-    font-family: 'open_sansregular';
-    src: url('../fonts/opensans-regular-webfont.eot');
-    src: url('../fonts/opensans-regular-webfont.eot?#iefix') format('embedded-opentype'),
-         url('../fonts/opensans-regular-webfont.woff') format('woff'),
-         url('../fonts/opensans-regular-webfont.ttf') format('truetype'),
-         url('../fonts/opensans-regular-webfont.svg#open_sansregular') format('svg');
-    font-weight: normal;
-    font-style: normal;
-
-}
-*/
-
-
-
-html {
-	font-size: 100%;
-}
-body {
-	font-size: 14px;
-	font-size: 0.9375rem;
-	font-family: 'Open sans', 'HelveticaNeueRegular', Arial, sans-serif;
-	color: #666666;
-	font-weight: 400;
-	padding: 0;
-}
-.topspace {
-	margin-top: 40px;
-}
-p {
-	line-height: 1.6em;
-	margin: 0 0 30px 0;
-}
-.title_text {
-	font-size: 1.1em !important;
-}
-.big_text {
-	font-size: 1.7em !important;
-	font-family: 'Open sans';
-	font-weight: 300;
-}
-.normal_text {
-	font-size: 1.3em !important;
-	font-family: 'Open sans';
-	font-weight: 600;
-}
-ul, ol {
-	line-height: 1.6em;
-	margin: 0 0 30px 0;
-}
-h1, h2, h3, h4, h5, h6, .h1, .h2, .h3, .h4, .h5, .h6 {
-	font-family: 'Open sans', 'HelveticaNeueRegular', Arial, sans-serif;
-	font-weight: 400;
-	color: #333333;
-}
-h1 {
-	font-size: 40px;
-	font-size: 2.5rem;
-}
-h2 {
-	font-size: 36px;
-	font-size: 2.25rem;
-}
-h3 {
-	font-size: 30px;
-	font-size: 1.875rem;
-}
-h4 {
-	font-size: 24px;
-	font-size: 1.5rem;
-}
-h5 {
-	font-size: 20px;
-	font-size: 1.25rem;
-}
-a {
-	color: #2679f4;
-	-webkit-transition: 0.25s;
-	-moz-transition: 0.25s;
-	-o-transition: 0.25s;
-	transition: 0.25s;
-}
-a:hover, a:focus {
-	cursor: pointer;
-	color: #2062c4;
-	text-decoration: none;
-}
-/*********************************************************************
- 
-
-                                HEADER
-
-
-**********************************************************************/
-#head {
-	background: #f4f4f4 url(../images/data.png) top center;
-	background-size: cover;
-	color: #7C7C7C;
-	height: 330px;	
-}
-#head .title {
-	font-family: 'Open sans';
-	font-weight: 400;
-	font-size: 2.2rem;
-	font-size: 36px;
-	color: #fb8d08;/*	background-image: -webkit-gradient( linear, left bottom, right bottom, color-stop(0, #C6419E), color-stop(1, #6382FF) );
-	background-image: -o-linear-gradient(right, #C6419E 0%, #6382FF 100%);
-	background-image: -moz-linear-gradient(right, #C6419E 0%, #6382FF 100%);
-	background-image: -webkit-linear-gradient(right, #C6419E 0%, #6382FF 100%);
-	background-image: -ms-linear-gradient(right, #C6419E 0%, #6382FF 100%);
-	background-image: linear-gradient(to right, #C6419E 0%, #6382FF 100%);
-	-webkit-background-clip: text;
-	-webkit-text-fill-color: transparent;*/
-  
-}
-#head img.img-circle {
-	display: block;
-	width: 140px;
-	height: 140px;
-	overflow: hidden;
-	border: 9px solid rgba(0, 0, 0, 0.05);
-	margin: 0 auto;
-}
-
-#head .title a {
-	text-decoration: none;
-	color: #333333;
-}
-#head .tagline {
-	display: block;
-	font-size: 14px;
-	font-size: 1.1rem;
-	line-height: 1.2em;
-	color: #7C7C7C;
-	margin: 5px 0 0;
-}
-#head .tagline b {
-	font-weight: normal;
-}
-#head .tagline a {
-	color: #5E5E5E;
-}
-#logo {
-	padding-top: 50px;
-}
-.home #head .title {
-	font-size: 18px;
-	color: #fb8d08;/*color:#146df3;*/
-	
-}
-#download {
-	text-align: center;
-	padding: 20px;
-	font-size: 16px;
-	font-weight: 500;
-	width: 150px;
-	margin: 30px auto;
-	padding: 0.6em 0.8em;
-	color: #333;
-	/*background: rgba(38,121,244,0.8);*/
-	border: 1px solid #333;
-	-webkit-animation-delay: 1s;
-	-ms-animation-delay: 1s;
-	-moz-animation-delay: 1s;
-	-o-animation-delay: 1s;
-	animation-delay: 1s;
-	text-shadow: none;
-	text-transform: uppercase;
- transparent!important
-}
-#download a {
-	color: #333;
-}
-#download:hover {
-	border: 1px solid #5c5c5c;
-	opacity: 0.8;
-}
-/*********************************************************************
- 
-
-                            NAVBAR 
-
-
-**********************************************************************/
-.navbar {
-	/*
-	background: #f4f4f4 url(../images/data.png) top center;
-	background-size: cover;
-	color: #7C7C7C;
-	height: 100px;
-*/
-	 border-width: 1px 0;
-	-webkit-border-radius: 0;
-	-webkit-background-clip: padding-box;
-	-moz-border-radius: 0;
-	-moz-background-clip: padding;
-	border-radius: 0;
-	background-clip: padding-box;
-	width: 100%;
-
-}
-.navbar.stick {
-	position: fixed;
-	top: 0;
-	left: 0;
-	opacity: .95;
-}
-.navbar-collapse {
-	-webkit-border-radius: 0;
-	-webkit-background-clip: padding-box;
-	-moz-border-radius: 0;
-	-moz-background-clip: padding;
-	border-radius: 0;
-	background-clip: padding-box;
-	font-family: 'Open sans';
-	font-weight: 400;
-	/* text-transform: uppercase;*/
-}
-.navbar-nav {
-	float: none;
-	margin: 0 auto;
-	text-align: center;
-}
-.navbar-nav > li {
-	float: none;
-	display: inline-block;
-}
-.navbar-nav > li > a {
-	padding: 2px 12px 16px 12px;
-}
-.navbar-default {
-	background-color: #ffffff;
-	border-color: #cccccc;
-}
-.navbar-default .navbar-nav > li > a {
-	color: #454545;
-}
-.navbar-default .navbar-nav > li > a:hover, .navbar-default .navbar-nav > li > a:focus {
-	color: #000000;
-	background-color: #ffffff;
-}
-.navbar-default .navbar-nav > .active > a, .navbar-default .navbar-nav > .active > a:hover, .navbar-default .navbar-nav > .active > a:focus {
-	color: #000000;
-	background-color: #ffffff;
-}
-.navbar-default .dropdown ul.dropdown-menu > li > a {
-	color: #454545;
-}
-.navbar-default .dropdown ul.dropdown-menu > li > a:hover {
-	background-color: #eeeeee;
-	color: #000000;
-}
-/*.top-logo {
-	background-image: -webkit-gradient(  linear,  left bottom,  right bottom,  color-stop(0, #C6419E),  color-stop(1, #6382FF) );
-	background-image: -o-linear-gradient(right, #C6419E 0%, #6382FF 100%);
-	background-image: -moz-linear-gradient(right, #C6419E 0%, #6382FF 100%);
-	background-image: -webkit-linear-gradient(right, #C6419E 0%, #6382FF 100%);
-	background-image: -ms-linear-gradient(right, #C6419E 0%, #6382FF 100%);
-	background-image: linear-gradient(to right, #C6419E 0%, #6382FF 100%);
-	-webkit-background-clip: text;
-	-webkit-text-fill-color: transparent;
-}*/
-.top-logo-border {
-	background-color: #efefef;
-	width: 3em;
-	height: 3em;
-	border-radius: 1.5em;
-}
-/*********************************************************************************/
-/* Main Sections                                                                 */
-/*********************************************************************************/
-
-.main {
-	position: relative;
-	margin: 0;
-	font-family: 'Open sans';
-	font-weight: 300;
-}
-.indent {
-	margin-left: 40px;
-	line-height: 1.8em
-}
-.main p {
-	margin-top: 1.25em;
-	font-size: 1.04em;
-	font-weight: 100;
-	padding: 0;
-	line-height: 2em;
-}
-.main p a {
-	color: #2679f4;
-	-webkit-transition: 0.25s;
-	transition: 0.25s;
-}
-.main p a:hover, a:focus {
-	cursor: pointer;
-	color: #2062c4;
-	text-decoration: none;
-}
-.main > header {
-	background: #fff;
-	text-align: center;
-	padding: 4em 0 4em 0;
-	margin: 0;
-}
-.main > header h2 {
-	font-size: 2.25em;
-	font-weight: 300;
-	margin-bottom: 0;
-}
-.main > header p {
-	margin: 2em 0 0 0;
-	padding: 0;
-	text-align: left;
-	line-height: 1.5em
-}
-.less-margin {
-	margin-top: 1em !important;
-}
-.main > .content {
-	padding: 6em 0 6em 0;
-}
-.main > .content > .container {
-}
-.main > .content h3 {
-	font-size: 1.5em;
-}
-/*********************************************************************
- 
-
-                                CONTENT
-
-
-**********************************************************************/
-
-.circle {
-	width: 8em;
-	height: 8em;
-	border-radius: 4em;
-	text-align: center;
-	margin: 0 auto;
-	border: 1px solid #333;
-	padding-top: 2em;
-}
-.circle-icon {
-	font-size: 4.3em;
-	text-align: center;
-	vertical-align: middle;
-	padding-top: 0.5em;
-	color: #666666;
-}
-.version:hover {
-	opacity: .7;
-	-webkit-transition: 0.25s;
-	-moz-transition: 0.25s;
-	-o-transition: 0.25s;
-	transition: 0.25s;
-}
-.biggericon {
-	font-size: 58px;
-}
-#lessmargin {
-	margin-top: 1.4em;
-}
-img {
-	max-width: 100%;
-}
-.section-title, .entry-title {
-	display: block;
-	width: 100%;
-	overflow: hidden;
-	margin: 0px 0 25px;
-	text-align: center;
-	font-weight: 300;
-	text-transform: uppercase;
-	font-size: 36px;
-	font-size: 1.8rem;
-	letter-spacing: 1px;
-}
-.section-title a, .entry-title a {
-	color: #333333;
-}
-.section-title span, .entry-title span {
-	display: inline-block;
-	position: relative;
-}
-.section-title span:before, .entry-title span:before, .section-title span:after, .entry-title span:after {
-	content: "";
-	position: absolute;
-	height: 4px;
-	top: .53em;
-	width: 400%;
-	border-bottom: 1px solid #ccc;
-	border-top: 1px solid #ccc;
-}
-.section-title span:before, .entry-title span:before {
-	right: 100%;
-	margin-right: 45px;
-}
-.section-title span:after, .entry-title span:after {
-	left: 100%;
-	margin-left: 45px;
-}
-.entry-header .entry-meta {
-	text-align: center;
-	font-family: Georgia, serif;
-	font-size: 18px;
-	font-size: 1.125rem;
-	font-style: italic;
-	font-weight: normal;
-	color: #a5a5a5;
-	margin: 0 0 15px 0;
-}
-.entry-header .entry-meta a {
-	color: #a5a5a5;
-}
-.entry-content {
-	margin: 0 0 60px 0;
-}
-.code-lan {
-	font-family: "Open sans", Helvetica, Arial, sans-serif;
-	font-size: 1.1em !important;
-	text-align: center !important;
-}
-.code-lan>span {
-	padding-left: 10px;
-	padding-right: 10px;
-}
-#left-menu {
-	line-height: 2.5em !important;
-	list-style: none;
-	text-align: left;
-	margin-top: 10px;
-}
-#left-menu li {
-	border-bottom: 1px solid #e9e9e9;
-	cursor: pointer;
-}
-a.section-menu-title {
-	color: #666;
-	font-weight: 400;
-	-webkit-transition: 0.25s;
-	-moz-transition: 0.25s;
-	-o-transition: 0.25s;
-	transition: 0.25s;
-	padding-left: 1em
-}
-a.section-menu-title:hover {
-	color: #404040 !important;
-}
-.maintains {
-	padding-left: 2em;
-	line-height: 2em !important;
-}
-.maintain {
-	line-height: 2em !important;
-}
-.people_intro {
-/*border-bottom:1px dashed #CCC;
-	line-height:2em;
-	padding-bottom:0.3em*/
-}
-#contributors {
-	padding-bottom: 1.4em
-}
-#contributors2 {
-	border-bottom: 1px dashed #cfcfcf;
-	padding-bottom: 1.4em
-}
-.core_team {
-	text-align: center !important;
-	font-size: 1.2em !important;
-	padding-bottom: 2em !important;
-	border-bottom: 1px dashed #cfcfcf;
-}
-.participant {
-	margin-top: 0px !important
-}
-.content-p {
-	margin-top: 1em !important;
-}
-.content-header {
-	text-align: left;
-	font-weight: bold !important;
-}
-/*********************************************************************
- 
-
-                                FOOTER
-
-
-**********************************************************************/
-
-
-ul.icons {
-	position: relative;
-	background: rgba(128,128,128,0.05);
-	border-radius: 4em;
-	display: inline-block;
-	padding: 0.35em 0.75em 0.35em 0.75em;
-	font-size: 1.45em;
-	cursor: default;
-}
-ul.icons li {
-	display: inline-block;
-}
-ul.icons li a {
-	display: inline-block;
-	background: none;
-	width: 2.5em;
-	height: 2.5em;
-	line-height: 2.5em;
-	text-align: center;
-	border-radius: 100%;
-	border: 0;
-	color: inherit;
-}
-ul.icons li a:hover {
-	color: #ef8376;
-}
-ol.style1 {
-}
-#footer {
-	background: #232323;
-	padding: 30px 0 0 0;
-	font-size: 12px;
-	color: #999;
-}
-#footer a {
-	color: #ccc;
-}
-#footer a:hover {
-	color: #fff;
-}
-#footer h3.widget-title {
-	font-size: 15px;
-	font-size: 0.9375rem;
-	text-transform: uppercase;
-	color: #ccc;
-	margin: 0 0 20px;
-}
-#underfooter {
-	background: #696969;
-	padding: 15px 0;
-	color: #EEEEEE;
-	font-size: 12px;
-}
-#underfooter a {
-	color: #aaa;
-}
-#underfooter a:hover {
-	color: #fff;
-}
-#underfooter p {
-	margin: 0;
-}
-.navbar-default .navbar-collapse, .navbar-default .navbar-form {
-	border-color: transparent;
-}
-#nav.affix {
-	position: fixed;
-	top: 0;
-	width: 100%
-}
-#nav {
-	border-radius: 0;
-	background-clip: padding-box;
-	font-family: 'Open sans';
-	font-weight: 400;
-	/* text-transform: uppercase;*/
-	max-height: 340px;
-	padding-right: 15px;
-	padding-left: 15px;
-	border-bottom: 1px solid #d5d4d4;
-	background-color: #fff;
-	opacity: .95;
-}
-.navbar-nav>li>a {
-	color: #454545;
-}
-.nav>li>a:hover, .nav>li>a:focus {
-	text-decoration: none;
-	background-color: transparent;
-}
-#drawer-icon {
-	font-size: 20px;
-	padding: 4px 6px;
-	border: 1px solid #2679f4;
-	border-radius: 3px;
-	margin: 10px;
-}
-
-/* Max page width
-/////////////////////////////////////////////////////////////////////*/
-@media (min-width: 1200px) {
-.container {
-	max-width: 1080px;
-}
-}
-
-@media only screen and (max-width: 767px) {
-/*
-#nav-wrapper{
-	position:fixed;
-	top:0px;
-	
-	}*/
-	
-#nav {
-	background-color: transparent !important;
-}
-#head {
-	margin-top: 40px;
-}
-#nav-wrapper {
-	position: fixed;
-	top: 0px;
-}
-.navbar-nav {
-	background-color: #fff;
-	border: 1px solid #d5d4d4;
-}
-.navbar-nav>li {
-	display: list-item;
-	border-bottom: 1px solid #d5d4d4;
-}
-.navbar-default .navbar-toggle:hover, .navbar-default .navbar-toggle:focus {
-background-color: transparent !important;
-}
-}
-
-.center {
-  text-align: center;
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/assets/fonts/FontAwesome.otf
----------------------------------------------------------------------
diff --git a/website/assets/fonts/FontAwesome.otf b/website/assets/fonts/FontAwesome.otf
deleted file mode 100644
index 81c9ad9..0000000
Binary files a/website/assets/fonts/FontAwesome.otf and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/assets/fonts/fontawesome-webfont.eot
----------------------------------------------------------------------
diff --git a/website/assets/fonts/fontawesome-webfont.eot b/website/assets/fonts/fontawesome-webfont.eot
deleted file mode 100644
index 84677bc..0000000
Binary files a/website/assets/fonts/fontawesome-webfont.eot and /dev/null differ


[11/52] [abbrv] kylin git commit: KYLIN-1368 Make JDBC Driver more generic to restAPI json result

Posted by li...@apache.org.
KYLIN-1368 Make JDBC Driver more generic to restAPI json result


Project: http://git-wip-us.apache.org/repos/asf/kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/4a5d0599
Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/4a5d0599
Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/4a5d0599

Branch: refs/heads/1.x-HBase1.1.3
Commit: 4a5d0599286fab01969d7f5356bd5233de6ecacb
Parents: 247394f
Author: lidongsjtu <li...@apache.org>
Authored: Tue Jan 26 17:56:00 2016 +0800
Committer: lidongsjtu <li...@apache.org>
Committed: Tue Jan 26 17:56:00 2016 +0800

----------------------------------------------------------------------
 .../apache/kylin/jdbc/json/SQLResponseStub.java | 12 ++--
 .../apache/kylin/jdbc/SQLResonseStubTest.java   | 62 ++++++++++++++++++++
 2 files changed, 70 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/4a5d0599/jdbc/src/main/java/org/apache/kylin/jdbc/json/SQLResponseStub.java
----------------------------------------------------------------------
diff --git a/jdbc/src/main/java/org/apache/kylin/jdbc/json/SQLResponseStub.java b/jdbc/src/main/java/org/apache/kylin/jdbc/json/SQLResponseStub.java
index e810cfd..ee28438 100644
--- a/jdbc/src/main/java/org/apache/kylin/jdbc/json/SQLResponseStub.java
+++ b/jdbc/src/main/java/org/apache/kylin/jdbc/json/SQLResponseStub.java
@@ -21,8 +21,11 @@ package org.apache.kylin.jdbc.json;
 import java.io.Serializable;
 import java.util.List;
 
+import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
+
 /**
  */
+@JsonIgnoreProperties(ignoreUnknown = true)
 public class SQLResponseStub implements Serializable {
     private static final long serialVersionUID = 1L;
 
@@ -56,6 +59,10 @@ public class SQLResponseStub implements Serializable {
     public SQLResponseStub() {
     }
 
+    public static long getSerialversionuid() {
+        return serialVersionUID;
+    }
+
     public List<ColumnMetaStub> getColumnMetas() {
         return columnMetas;
     }
@@ -136,10 +143,7 @@ public class SQLResponseStub implements Serializable {
         this.hitCache = hitCache;
     }
 
-    public static long getSerialversionuid() {
-        return serialVersionUID;
-    }
-
+    @JsonIgnoreProperties(ignoreUnknown = true)
     public static class ColumnMetaStub {
 
         private boolean isAutoIncrement;

http://git-wip-us.apache.org/repos/asf/kylin/blob/4a5d0599/jdbc/src/test/java/org/apache/kylin/jdbc/SQLResonseStubTest.java
----------------------------------------------------------------------
diff --git a/jdbc/src/test/java/org/apache/kylin/jdbc/SQLResonseStubTest.java b/jdbc/src/test/java/org/apache/kylin/jdbc/SQLResonseStubTest.java
new file mode 100644
index 0000000..9e4b3f6
--- /dev/null
+++ b/jdbc/src/test/java/org/apache/kylin/jdbc/SQLResonseStubTest.java
@@ -0,0 +1,62 @@
+/*
+ * 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.jdbc;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNull;
+
+import java.io.IOException;
+
+import org.apache.kylin.jdbc.json.SQLResponseStub;
+import org.junit.Test;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+
+/**
+ * Created by dongli on 1/25/16.
+ */
+public class SQLResonseStubTest {
+
+    @Test
+    public void testReadValuePartRecognizedField() throws IOException {
+        final String payload = "{ \"columnMetas\":[ { \"isNullable\":1, \"displaySize\":0, \"schemaName\":null, \"catelogName\":null, \"tableName\":null, \"precision\":0, \"scale\":0, \"columnType\":91, \"columnTypeName\":\"DATE\", \"readOnly\":true, \"writable\":false, \"caseSensitive\":true, \"searchable\":false, \"currency\":false, \"signed\":true, \"autoIncrement\":false, \"definitelyWritable\":false }, { \"isNullable\":1, \"displaySize\":10, \"label\":\"LEAF_CATEG_ID\", \"name\":\"LEAF_CATEG_ID\", \"schemaName\":null, \"catelogName\":null, \"tableName\":null, \"precision\":10, \"scale\":0, \"columnType\":4, \"columnTypeName\":\"INTEGER\", \"readOnly\":true, \"writable\":false, \"caseSensitive\":true, \"searchable\":false, \"currency\":false, \"signed\":true, \"autoIncrement\":false, \"definitelyWritable\":false } ], \"results\":[ [ \"2013-08-07\", \"32996\", \"15\", \"15\", \"Auction\", \"10000000\", \"49.048952730908745\", \"49.048952730908745\", \"49.048952730908745\", \"1\" 
 ], [ \"2013-08-07\", \"43398\", \"0\", \"14\", \"ABIN\", \"10000633\", \"85.78317064220418\", \"85.78317064220418\", \"85.78317064220418\", \"1\" ] ], \"cube\":\"test_kylin_cube_with_slr_desc\", \"affectedRowCount\":0, \"isException\":false, \"exceptionMessage\":null, \"duration\":3451, \"partial\":false }";
+        final SQLResponseStub stub = new ObjectMapper().readValue(payload, SQLResponseStub.class);
+        assertEquals("test_kylin_cube_with_slr_desc", stub.getCube());
+        assertEquals(3451, stub.getDuration());
+        assertFalse(stub.getColumnMetas().isEmpty());
+        assertEquals(91, stub.getColumnMetas().get(0).getColumnType());
+        assertNull(stub.getColumnMetas().get(0).getLabel());
+        assertFalse(stub.getResults().isEmpty());
+        assertNull(stub.getExceptionMessage());
+    }
+
+    @Test
+    public void testReadValueWithUnrecognizedField() throws IOException {
+        final String payload = "{ \"columnMetas\":[ { \"Unrecognized\":0, \"isNullable\":1, \"displaySize\":0, \"label\":\"CAL_DT\", \"name\":\"CAL_DT\", \"schemaName\":null, \"catelogName\":null, \"tableName\":null, \"precision\":0, \"scale\":0, \"columnType\":91, \"columnTypeName\":\"DATE\", \"readOnly\":true, \"writable\":false, \"caseSensitive\":true, \"searchable\":false, \"currency\":false, \"signed\":true, \"autoIncrement\":false, \"definitelyWritable\":false }, { \"isNullable\":1, \"displaySize\":10, \"label\":\"LEAF_CATEG_ID\", \"name\":\"LEAF_CATEG_ID\", \"schemaName\":null, \"catelogName\":null, \"tableName\":null, \"precision\":10, \"scale\":0, \"columnType\":4, \"columnTypeName\":\"INTEGER\", \"readOnly\":true, \"writable\":false, \"caseSensitive\":true, \"searchable\":false, \"currency\":false, \"signed\":true, \"autoIncrement\":false, \"definitelyWritable\":false } ], \"results\":[ [ \"2013-08-07\", \"32996\", \"15\", \"15\", \"Auction\", \"10000000\", \"49.0489527309
 08745\", \"49.048952730908745\", \"49.048952730908745\", \"1\" ], [ \"2013-08-07\", \"43398\", \"0\", \"14\", \"ABIN\", \"10000633\", \"85.78317064220418\", \"85.78317064220418\", \"85.78317064220418\", \"1\" ] ], \"cube\":\"test_kylin_cube_with_slr_desc\", \"affectedRowCount\":0, \"isException\":false, \"exceptionMessage\":null, \"duration\":3451, \"partial\":false, \"hitCache\":false }";
+        final SQLResponseStub stub = new ObjectMapper().readValue(payload, SQLResponseStub.class);
+        assertEquals("test_kylin_cube_with_slr_desc", stub.getCube());
+        assertEquals(3451, stub.getDuration());
+        assertFalse(stub.getColumnMetas().isEmpty());
+        assertEquals(91, stub.getColumnMetas().get(0).getColumnType());
+        assertEquals("CAL_DT", stub.getColumnMetas().get(0).getLabel());
+        assertFalse(stub.getResults().isEmpty());
+        assertNull(stub.getExceptionMessage());
+    }
+}


[32/52] [abbrv] kylin git commit: KYLIN-1416 remove website from coding branch

Posted by li...@apache.org.
http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/assets/fonts/opensans-light-webfont.svg
----------------------------------------------------------------------
diff --git a/website/assets/fonts/opensans-light-webfont.svg b/website/assets/fonts/opensans-light-webfont.svg
deleted file mode 100644
index a36a5b7..0000000
--- a/website/assets/fonts/opensans-light-webfont.svg
+++ /dev/null
@@ -1,1824 +0,0 @@
-<?xml version="1.0" standalone="no"?>
-<!DOCTYPE svg PUBLIC "-//W3C//DTD SVG 1.1//EN" "http://www.w3.org/Graphics/SVG/1.1/DTD/svg11.dtd" >
-<svg xmlns="http://www.w3.org/2000/svg">
-<metadata></metadata>
-<defs>
-<font id="open_sanslight" horiz-adv-x="1169" >
-<font-face units-per-em="2048" ascent="1638" descent="-410" />
-<missing-glyph horiz-adv-x="532" />
-<glyph unicode="&#xfb01;" horiz-adv-x="1077" d="M29 1001v58l202 37v84q0 200 73.5 293.5t240.5 93.5q90 0 180 -27l-23 -86q-80 25 -159 25q-116 0 -164.5 -68.5t-48.5 -222.5v-101h256v-86h-256v-1001h-99v1001h-202zM782 1389q0 96 63 96q31 0 48.5 -25t17.5 -71q0 -45 -17.5 -71t-48.5 -26q-63 0 -63 97zM796 0v1087 h99v-1087h-99z" />
-<glyph unicode="&#xfb02;" horiz-adv-x="1077" d="M29 1001v58l202 37v84q0 200 73.5 293.5t240.5 93.5q90 0 180 -27l-23 -86q-80 25 -159 25q-116 0 -164.5 -68.5t-48.5 -222.5v-101h256v-86h-256v-1001h-99v1001h-202zM796 0v1556h99v-1556h-99z" />
-<glyph unicode="&#xfb03;" horiz-adv-x="1692" d="M29 1001v58l202 37v84q0 200 73.5 293.5t240.5 93.5q90 0 180 -27l-23 -86q-80 25 -159 25q-116 0 -164.5 -68.5t-48.5 -222.5v-101h256v-86h-256v-1001h-99v1001h-202zM643 1001v58l202 37v84q0 200 73.5 293.5t240.5 93.5q90 0 180 -27l-23 -86q-80 25 -159 25 q-116 0 -164.5 -68.5t-48.5 -222.5v-101h256v-86h-256v-1001h-99v1001h-202zM1397 1389q0 96 63 96q31 0 48.5 -25t17.5 -71q0 -45 -17.5 -71t-48.5 -26q-63 0 -63 97zM1411 0v1087h99v-1087h-99z" />
-<glyph unicode="&#xfb04;" horiz-adv-x="1692" d="M29 1001v58l202 37v84q0 200 73.5 293.5t240.5 93.5q90 0 180 -27l-23 -86q-80 25 -159 25q-116 0 -164.5 -68.5t-48.5 -222.5v-101h256v-86h-256v-1001h-99v1001h-202zM643 1001v58l202 37v84q0 200 73.5 293.5t240.5 93.5q90 0 180 -27l-23 -86q-80 25 -159 25 q-116 0 -164.5 -68.5t-48.5 -222.5v-101h256v-86h-256v-1001h-99v1001h-202zM1411 0v1556h99v-1556h-99z" />
-<glyph horiz-adv-x="0" />
-<glyph unicode="&#xd;" horiz-adv-x="1044" />
-<glyph unicode=" "  horiz-adv-x="532" />
-<glyph unicode="&#x09;" horiz-adv-x="532" />
-<glyph unicode="&#xa0;" horiz-adv-x="532" />
-<glyph unicode="!" horiz-adv-x="492" d="M164 78q0 98 80 98q82 0 82 -98t-82 -98q-80 0 -80 98zM186 1462h119l-29 -1085h-61z" />
-<glyph unicode="&#x22;" horiz-adv-x="723" d="M133 1462h127l-33 -528h-61zM463 1462h127l-33 -528h-61z" />
-<glyph unicode="#" horiz-adv-x="1323" d="M55 451v79h299l76 398h-297v80h311l86 454h91l-89 -454h365l88 454h86l-88 -454h285v-80h-301l-76 -398h303v-79h-320l-86 -451h-90l88 451h-360l-86 -451h-88l86 451h-283zM440 530h363l78 398h-363z" />
-<glyph unicode="$" d="M164 186v103q75 -36 179.5 -61t193.5 -25v508q-145 44 -215 88t-102 104t-32 146q0 124 94.5 208.5t254.5 104.5v192h81v-190q197 -9 351 -72l-33 -90q-141 62 -318 72v-486q213 -66 293 -144t80 -204q0 -133 -99 -217t-274 -106v-236h-81v232q-92 2 -200.5 22.5 t-172.5 50.5zM297 1049q0 -86 57 -141t183 -93v453q-119 -16 -179.5 -76t-60.5 -143zM618 209q122 13 192.5 75t70.5 160q0 85 -63 140.5t-200 95.5v-471z" />
-<glyph unicode="%" horiz-adv-x="1653" d="M113 1026q0 223 72 340t212 117q139 0 215 -120.5t76 -336.5q0 -226 -75 -343.5t-216 -117.5q-133 0 -208.5 120.5t-75.5 340.5zM211 1026q0 -186 45 -279.5t141 -93.5q193 0 193 373q0 184 -49.5 276.5t-143.5 92.5q-96 0 -141 -92.5t-45 -276.5zM373 0l811 1462h96 l-811 -1462h-96zM965 438q0 225 73.5 341t212.5 116q137 0 213 -120t76 -337q0 -226 -74 -343.5t-215 -117.5q-136 0 -211 121.5t-75 339.5zM1063 438q0 -185 45 -277.5t141 -92.5q193 0 193 370q0 369 -193 369q-96 0 -141 -91.5t-45 -277.5z" />
-<glyph unicode="&#x26;" horiz-adv-x="1460" d="M123 371q0 138 73.5 235t274.5 205l-75 82q-66 71 -98 139t-32 142q0 143 95.5 227t256.5 84q155 0 245.5 -81t90.5 -224q0 -105 -70 -192.5t-253 -194.5l452 -457q61 72 104 157t75 201h96q-63 -246 -209 -426l266 -268h-135l-193 197q-92 -90 -164 -131.5t-157.5 -63.5 t-194.5 -22q-209 0 -328.5 103t-119.5 288zM227 375q0 -143 93 -224t258 -81q128 0 234.5 43.5t209.5 146.5l-483 485q-136 -72 -196.5 -122.5t-88 -109.5t-27.5 -138zM373 1176q0 -79 40 -146t152 -174q159 85 221 159t62 169q0 94 -62 152.5t-168 58.5q-114 0 -179.5 -58 t-65.5 -161z" />
-<glyph unicode="'" horiz-adv-x="393" d="M133 1462h127l-33 -528h-61z" />
-<glyph unicode="(" horiz-adv-x="557" d="M82 561q0 265 77.5 496t223.5 405h113q-148 -182 -227 -412.5t-79 -486.5q0 -483 304 -887h-111q-147 170 -224 397t-77 488z" />
-<glyph unicode=")" horiz-adv-x="557" d="M61 1462h113q147 -175 224 -406.5t77 -494.5t-77.5 -490t-223.5 -395h-111q304 404 304 887q0 257 -79 487.5t-227 411.5z" />
-<glyph unicode="*" horiz-adv-x="1128" d="M104 1124l19 131l401 -104l-39 405h146l-37 -405l405 104l21 -131l-395 -39l247 -340l-124 -71l-191 379l-180 -379l-125 71l242 340z" />
-<glyph unicode="+" d="M111 682v82h432v434h82v-434h434v-82h-434v-432h-82v432h-432z" />
-<glyph unicode="," horiz-adv-x="440" d="M68 -264q77 275 110 502h117l12 -21q-75 -265 -174 -481h-65z" />
-<glyph unicode="-" horiz-adv-x="659" d="M92 512v82h475v-82h-475z" />
-<glyph unicode="." horiz-adv-x="487" d="M162 78q0 98 80 98q82 0 82 -98t-82 -98q-80 0 -80 98z" />
-<glyph unicode="/" horiz-adv-x="698" d="M25 0l544 1462h105l-545 -1462h-104z" />
-<glyph unicode="0" d="M115 735q0 382 115.5 566t351.5 184q231 0 352 -190.5t121 -559.5q0 -385 -117.5 -570t-355.5 -185q-229 0 -348 190.5t-119 564.5zM223 735q0 -340 89 -502.5t270 -162.5q189 0 275.5 168t86.5 497q0 324 -86.5 492t-275.5 168t-274 -168t-85 -492z" />
-<glyph unicode="1" d="M199 1165l397 297h86v-1462h-98v1065q0 145 12 301q-15 -15 -31 -29t-309 -243z" />
-<glyph unicode="2" d="M113 0v88l389 406q164 170 230 260t97 172t31 172q0 131 -86 213t-223 82q-183 0 -350 -133l-54 69q183 154 406 154q191 0 300.5 -102t109.5 -281q0 -145 -73.5 -280.5t-268.5 -334.5l-375 -385v-4h782v-96h-915z" />
-<glyph unicode="3" d="M94 63v99q84 -44 188.5 -69t196.5 -25q221 0 332 89.5t111 252.5q0 145 -113.5 223t-333.5 78h-158v96h160q182 0 288.5 86.5t106.5 234.5q0 122 -86.5 195.5t-226.5 73.5q-109 0 -199 -30.5t-202 -104.5l-49 67q85 71 205 112.5t243 41.5q202 0 312 -95.5t110 -269.5 q0 -136 -85.5 -229t-229.5 -119v-6q176 -22 268 -112t92 -242q0 -205 -139.5 -317.5t-401.5 -112.5q-223 0 -389 83z" />
-<glyph unicode="4" d="M43 373v67l725 1030h121v-1011h252v-86h-252v-373h-94v373h-752zM162 459h633v418q0 302 14 507h-8q-20 -37 -123 -188z" />
-<glyph unicode="5" d="M143 63v103q108 -55 192 -76.5t179 -21.5q192 0 308 101.5t116 274.5q0 163 -113 256t-307 93q-130 0 -272 -39l-60 39l58 669h704v-96h-610l-45 -516q156 29 244 29q234 0 368.5 -113t134.5 -311q0 -225 -140 -350t-386 -125q-109 0 -207 21.5t-164 61.5z" />
-<glyph unicode="6" d="M131 623q0 285 77.5 479.5t220 288.5t343.5 94q94 0 172 -23v-88q-73 27 -176 27q-247 0 -384.5 -178t-154.5 -518h13q76 98 174 148t207 50q205 0 320.5 -117t115.5 -323q0 -224 -121.5 -353.5t-327.5 -129.5q-222 0 -350.5 169.5t-128.5 473.5zM240 504 q0 -111 49.5 -213.5t134 -162.5t186.5 -60q164 0 255 103t91 294q0 168 -90 262t-245 94q-102 0 -189.5 -45t-139.5 -119.5t-52 -152.5z" />
-<glyph unicode="7" d="M109 1366v96h946v-73l-604 -1389h-117l602 1366h-827z" />
-<glyph unicode="8" d="M121 375q0 131 83 230t257 169q-161 76 -227 160.5t-66 202.5q0 105 53 184.5t148.5 122.5t212.5 43q186 0 299.5 -95t113.5 -257q0 -112 -70.5 -198t-228.5 -159q192 -79 270 -173t78 -228q0 -181 -126.5 -289t-339.5 -108q-221 0 -339 101t-118 294zM223 360 q0 -138 93.5 -214t261.5 -76q164 0 264 80.5t100 218.5q0 124 -78.5 201.5t-302.5 162.5q-184 -71 -261 -157t-77 -216zM268 1137q0 -70 31.5 -123.5t91 -97t199.5 -101.5q163 63 234 139t71 183q0 120 -84.5 190t-230.5 70q-141 0 -226.5 -69.5t-85.5 -190.5z" />
-<glyph unicode="9" d="M111 993q0 220 124.5 356t323.5 136q144 0 252 -75.5t166.5 -221.5t58.5 -346q0 -288 -75.5 -482t-220 -287t-349.5 -93q-104 0 -192 26v86q43 -14 103.5 -21.5t92.5 -7.5q247 0 387 178.5t156 520.5h-12q-73 -96 -174 -147.5t-211 -51.5q-203 0 -316.5 112t-113.5 318z M213 999q0 -174 87 -264t249 -90q101 0 188.5 45t139 119.5t51.5 151.5q0 117 -46.5 219t-130 159.5t-192.5 57.5q-158 0 -252 -106.5t-94 -291.5z" />
-<glyph unicode=":" horiz-adv-x="487" d="M162 78q0 98 80 98q82 0 82 -98t-82 -98q-80 0 -80 98zM162 971q0 98 80 98q82 0 82 -98q0 -53 -23.5 -76t-58.5 -23q-34 0 -57 23t-23 76z" />
-<glyph unicode=";" horiz-adv-x="487" d="M76 -264q29 97 62 245.5t48 256.5h117l12 -21q-75 -265 -174 -481h-65zM162 971q0 98 80 98q82 0 82 -98q0 -53 -23.5 -76t-58.5 -23q-34 0 -57 23t-23 76z" />
-<glyph unicode="&#x3c;" d="M111 682v61l948 474v-95l-823 -405l823 -355v-96z" />
-<glyph unicode="=" d="M111 477v82h948v-82h-948zM111 885v82h948v-82h-948z" />
-<glyph unicode="&#x3e;" d="M111 266v96l823 355l-823 405v95l948 -474v-61z" />
-<glyph unicode="?" horiz-adv-x="862" d="M57 1403q110 48 184.5 64t153.5 16q183 0 288 -98.5t105 -270.5q0 -68 -18 -119t-50.5 -94.5t-78.5 -84t-102 -87.5q-64 -54 -98.5 -98.5t-50 -93.5t-15.5 -146v-14h-82v37q0 123 37.5 201t138.5 167l91 79q72 61 103 121t31 138q0 127 -83.5 202t-219.5 75 q-79 0 -148 -17.5t-149 -56.5zM260 78q0 98 80 98q82 0 82 -98t-82 -98q-80 0 -80 98z" />
-<glyph unicode="@" horiz-adv-x="1815" d="M113 561q0 256 108.5 460.5t307 317.5t448.5 113q215 0 380.5 -89t255 -254.5t89.5 -383.5q0 -228 -90.5 -366t-245.5 -138q-89 0 -144.5 54t-64.5 147h-4q-43 -100 -124 -150.5t-189 -50.5q-148 0 -229 96.5t-81 270.5q0 202 120.5 330.5t314.5 128.5q138 0 286 -41 l-22 -464v-30q0 -104 35 -156.5t116 -52.5q103 0 168.5 116.5t65.5 303.5q0 194 -79 340t-225.5 224.5t-334.5 78.5q-230 0 -405.5 -99.5t-270 -281.5t-94.5 -418q0 -322 167 -497.5t474 -175.5q93 0 188.5 18t231.5 70v-99q-203 -80 -414 -80q-349 0 -544 200.5t-195 557.5 zM633 590q0 -143 55 -215t174 -72q255 0 273 346l16 291q-79 27 -193 27q-149 0 -237 -102.5t-88 -274.5z" />
-<glyph unicode="A" horiz-adv-x="1229" d="M0 0l588 1468h65l576 -1468h-115l-203 516h-594l-204 -516h-113zM354 608h523l-199 527q-25 62 -60 172q-27 -96 -59 -174z" />
-<glyph unicode="B" horiz-adv-x="1284" d="M207 0v1462h401q271 0 398 -92t127 -278q0 -127 -77.5 -211.5t-226.5 -108.5v-6q175 -26 257.5 -110.5t82.5 -235.5q0 -202 -134 -311t-380 -109h-448zM309 90h344q406 0 406 330q0 301 -428 301h-322v-631zM309 811h322q206 0 299.5 68.5t93.5 214.5t-105.5 212 t-314.5 66h-295v-561z" />
-<glyph unicode="C" horiz-adv-x="1272" d="M129 735q0 223 84.5 393t243 262.5t368.5 92.5q214 0 383 -80l-41 -92q-160 80 -336 80q-275 0 -433 -176t-158 -482q0 -313 149 -486t426 -173q184 0 338 47v-90q-145 -51 -362 -51q-308 0 -485 199t-177 556z" />
-<glyph unicode="D" horiz-adv-x="1446" d="M207 0v1462h395q350 0 532.5 -183t182.5 -534q0 -368 -193 -556.5t-567 -188.5h-350zM309 90h242q655 0 655 651q0 314 -159.5 472.5t-468.5 158.5h-269v-1282z" />
-<glyph unicode="E" horiz-adv-x="1130" d="M207 0v1462h799v-94h-697v-553h658v-94h-658v-627h697v-94h-799z" />
-<glyph unicode="F" horiz-adv-x="1028" d="M207 0v1462h801v-94h-699v-620h660v-95h-660v-653h-102z" />
-<glyph unicode="G" horiz-adv-x="1481" d="M129 729q0 223 91.5 395.5t262 266.5t391.5 94q239 0 429 -88l-41 -92q-190 88 -394 88q-289 0 -458.5 -178.5t-169.5 -481.5q0 -330 161 -496.5t473 -166.5q202 0 343 57v514h-435v96h539v-667q-212 -90 -477 -90q-346 0 -530.5 195.5t-184.5 553.5z" />
-<glyph unicode="H" horiz-adv-x="1473" d="M207 0v1462h102v-649h854v649h103v-1462h-103v719h-854v-719h-102z" />
-<glyph unicode="I" horiz-adv-x="516" d="M207 0v1462h102v-1462h-102z" />
-<glyph unicode="J" horiz-adv-x="506" d="M-184 -254q78 -20 149 -20q242 0 242 264v1472h102v-1462q0 -369 -342 -369q-92 0 -151 27v88z" />
-<glyph unicode="K" horiz-adv-x="1190" d="M207 0v1462h102v-760l162 162l573 598h130l-599 -618l615 -844h-125l-561 772l-195 -172v-600h-102z" />
-<glyph unicode="L" horiz-adv-x="1051" d="M207 0v1462h102v-1366h697v-96h-799z" />
-<glyph unicode="M" horiz-adv-x="1767" d="M207 0v1462h158l518 -1286h6l518 1286h154v-1462h-103v1108q0 116 12 240h-8l-547 -1348h-65l-545 1350h-8q8 -124 8 -254v-1096h-98z" />
-<glyph unicode="N" horiz-adv-x="1477" d="M207 0v1462h102l865 -1296h6q-9 180 -9 342v954h99v-1462h-103l-866 1298h-8q12 -232 12 -350v-948h-98z" />
-<glyph unicode="O" horiz-adv-x="1565" d="M129 735q0 349 175.5 549.5t479.5 200.5q306 0 479 -201.5t173 -550.5q0 -348 -174 -550.5t-480 -202.5q-305 0 -479 202.5t-174 552.5zM240 733q0 -314 140 -485.5t402 -171.5q264 0 403.5 170t139.5 487q0 316 -139.5 484.5t-401.5 168.5q-261 0 -402.5 -170 t-141.5 -483z" />
-<glyph unicode="P" horiz-adv-x="1198" d="M207 0v1462h358q522 0 522 -420q0 -212 -144 -325t-408 -113h-226v-604h-102zM309 692h201q247 0 357 81.5t110 264.5q0 169 -104 250.5t-322 81.5h-242v-678z" />
-<glyph unicode="Q" horiz-adv-x="1565" d="M129 735q0 349 175.5 549.5t479.5 200.5q306 0 479 -201.5t173 -550.5q0 -294 -126 -486.5t-349 -246.5l333 -348h-166l-282 330l-33 -2h-31q-305 0 -479 202.5t-174 552.5zM240 733q0 -314 140 -485.5t402 -171.5q264 0 403.5 170t139.5 487q0 316 -139.5 484.5 t-401.5 168.5q-261 0 -402.5 -170t-141.5 -483z" />
-<glyph unicode="R" horiz-adv-x="1217" d="M207 0v1462h348q272 0 402 -100.5t130 -302.5q0 -147 -77.5 -248t-235.5 -145l397 -666h-122l-377 637h-363v-637h-102zM309 725h279q185 0 287 82.5t102 243.5q0 167 -100 243t-326 76h-242v-645z" />
-<glyph unicode="S" horiz-adv-x="1116" d="M111 39v102q158 -67 403 -67q180 0 285.5 82.5t105.5 216.5q0 83 -35 137.5t-114 99.5t-232 97q-224 77 -309.5 166.5t-85.5 238.5q0 164 128.5 267.5t330.5 103.5q206 0 387 -78l-37 -88q-182 76 -348 76q-162 0 -258 -75t-96 -204q0 -81 29.5 -133t96.5 -93.5 t230 -99.5q171 -59 257 -114.5t125.5 -126t39.5 -170.5q0 -183 -134.5 -290t-357.5 -107q-268 0 -411 59z" />
-<glyph unicode="T" horiz-adv-x="1073" d="M10 1366v96h1053v-96h-475v-1366h-103v1366h-475z" />
-<glyph unicode="U" horiz-adv-x="1473" d="M190 520v942h103v-946q0 -211 117 -328.5t331 -117.5q209 0 324 115.5t115 320.5v956h102v-946q0 -252 -146 -394t-407 -142q-254 0 -396.5 142.5t-142.5 397.5z" />
-<glyph unicode="V" horiz-adv-x="1182" d="M0 1462h109l368 -995q84 -225 113 -338q20 75 79 233l402 1100h111l-547 -1462h-90z" />
-<glyph unicode="W" horiz-adv-x="1827" d="M51 1462h107l256 -942q15 -57 28 -105.5t23.5 -91t19 -82t15.5 -79.5q24 136 102 413l250 887h113l293 -1018q51 -176 73 -284q13 72 33.5 153t308.5 1149h103l-404 -1462h-84l-321 1128q-40 139 -60 228q-16 -87 -45.5 -200t-322.5 -1156h-86z" />
-<glyph unicode="X" horiz-adv-x="1102" d="M0 0l492 762l-447 700h115l395 -626l401 626h109l-453 -698l490 -764h-117l-432 682l-440 -682h-113z" />
-<glyph unicode="Y" horiz-adv-x="1081" d="M0 1462h117l426 -800l428 800h110l-487 -897v-565h-105v557z" />
-<glyph unicode="Z" horiz-adv-x="1180" d="M82 0v76l856 1290h-817v96h954v-76l-858 -1290h881v-96h-1016z" />
-<glyph unicode="[" horiz-adv-x="653" d="M174 -324v1786h428v-94h-330v-1597h330v-95h-428z" />
-<glyph unicode="\" horiz-adv-x="698" d="M25 1462h102l547 -1462h-103z" />
-<glyph unicode="]" horiz-adv-x="653" d="M51 -229h330v1597h-330v94h428v-1786h-428v95z" />
-<glyph unicode="^" d="M88 561l465 912h68l460 -912h-100l-395 791l-398 -791h-100z" />
-<glyph unicode="_" horiz-adv-x="842" d="M-4 -184h850v-82h-850v82z" />
-<glyph unicode="`" horiz-adv-x="1182" d="M393 1552v17h142q26 -48 98.5 -142t142.5 -170v-16h-69q-96 79 -188.5 171.5t-125.5 139.5z" />
-<glyph unicode="a" horiz-adv-x="1085" d="M98 289q0 159 132.5 247t383.5 93l207 6v72q0 155 -63 234t-203 79q-151 0 -313 -84l-37 86q179 84 354 84q179 0 267.5 -93t88.5 -290v-723h-73l-25 172h-8q-82 -105 -168.5 -148.5t-204.5 -43.5q-160 0 -249 82t-89 227zM203 285q0 -102 62.5 -158.5t176.5 -56.5 q174 0 274.5 99.5t100.5 276.5v107l-190 -8q-229 -11 -326.5 -71.5t-97.5 -188.5z" />
-<glyph unicode="b" horiz-adv-x="1219" d="M182 0v1556h99v-391q0 -88 -4 -162l-3 -85h7q62 98 149.5 144t210.5 46q228 0 343.5 -143.5t115.5 -419.5q0 -271 -121.5 -418t-341.5 -147q-116 0 -209 48t-147 136h-9l-28 -164h-62zM281 528q0 -246 86.5 -353t269.5 -107q178 0 268 124.5t90 354.5q0 471 -356 471 q-192 0 -275 -110t-83 -363v-17z" />
-<glyph unicode="c" horiz-adv-x="973" d="M119 537q0 270 137 420.5t375 150.5q141 0 270 -49l-27 -88q-141 47 -245 47q-200 0 -303 -123.5t-103 -355.5q0 -220 103 -344.5t288 -124.5q148 0 275 53v-92q-104 -51 -273 -51q-233 0 -365 147t-132 410z" />
-<glyph unicode="d" horiz-adv-x="1219" d="M119 528q0 282 118 431t343 149q118 0 204 -43t154 -147h6q-6 126 -6 247v391h98v-1556h-65l-25 166h-8q-124 -186 -356 -186q-225 0 -344 140t-119 408zM223 530q0 -462 359 -462q184 0 270 107t86 353v17q0 252 -84.5 362.5t-273.5 110.5q-178 0 -267.5 -125 t-89.5 -363z" />
-<glyph unicode="e" horiz-adv-x="1124" d="M119 535q0 260 128 416.5t345 156.5q192 0 303 -134t111 -364v-80h-783q2 -224 104.5 -342t293.5 -118q93 0 163.5 13t178.5 56v-90q-92 -40 -170 -54.5t-172 -14.5q-237 0 -369.5 146t-132.5 409zM229 618h672q0 189 -82 295.5t-227 106.5q-157 0 -252 -103.5 t-111 -298.5z" />
-<glyph unicode="f" horiz-adv-x="614" d="M29 1001v58l202 37v84q0 200 73.5 293.5t240.5 93.5q90 0 180 -27l-23 -86q-80 25 -159 25q-116 0 -164.5 -68.5t-48.5 -222.5v-101h256v-86h-256v-1001h-99v1001h-202z" />
-<glyph unicode="g" horiz-adv-x="1071" d="M45 -193q0 112 69.5 186t188.5 101q-49 21 -78.5 59.5t-29.5 88.5q0 109 139 192q-95 39 -148 122.5t-53 191.5q0 163 103.5 261.5t279.5 98.5q107 0 166 -21h348v-69l-225 -14q90 -112 90 -246q0 -157 -104.5 -254.5t-280.5 -97.5q-74 0 -104 6q-59 -31 -90 -73t-31 -89 q0 -52 39.5 -76t132.5 -24h190q177 0 271 -71.5t94 -211.5q0 -172 -139.5 -265.5t-397.5 -93.5q-205 0 -317.5 79t-112.5 220zM150 -184q0 -224 333 -224q428 0 428 273q0 98 -67 142t-217 44h-178q-299 0 -299 -235zM233 748q0 -126 76.5 -195.5t204.5 -69.5 q136 0 208.5 69t72.5 200q0 139 -74.5 208.5t-208.5 69.5q-130 0 -204.5 -74.5t-74.5 -207.5z" />
-<glyph unicode="h" horiz-adv-x="1208" d="M182 0v1556h99v-495l-5 -139h7q61 98 154 142t231 44q370 0 370 -397v-711h-98v705q0 164 -69 238.5t-214 74.5q-195 0 -285.5 -98.5t-90.5 -319.5v-600h-99z" />
-<glyph unicode="i" horiz-adv-x="463" d="M168 1389q0 96 63 96q31 0 48.5 -25t17.5 -71q0 -45 -17.5 -71t-48.5 -26q-63 0 -63 97zM182 0v1087h99v-1087h-99z" />
-<glyph unicode="j" horiz-adv-x="463" d="M-98 -381q69 -20 129 -20q151 0 151 176v1312h99v-1298q0 -135 -63.5 -208t-180.5 -73q-80 0 -135 25v86zM168 1389q0 96 63 96q31 0 48.5 -25t17.5 -71q0 -45 -17.5 -71t-48.5 -26q-63 0 -63 97z" />
-<glyph unicode="k" horiz-adv-x="991" d="M182 0v1556h99v-780l-7 -299h5l555 610h120l-428 -464l465 -623h-119l-413 549l-178 -162v-387h-99z" />
-<glyph unicode="l" horiz-adv-x="463" d="M182 0v1556h99v-1556h-99z" />
-<glyph unicode="m" horiz-adv-x="1808" d="M182 0v1087h82l21 -149h6q45 81 128 125.5t183 44.5q257 0 330 -193h4q53 93 142.5 143t203.5 50q178 0 267 -95t89 -302v-711h-98v713q0 159 -62 232t-190 73q-167 0 -247 -92t-80 -289v-637h-101v743q0 275 -252 275q-171 0 -249 -99.5t-78 -318.5v-600h-99z" />
-<glyph unicode="n" horiz-adv-x="1208" d="M182 0v1087h84l19 -149h6q106 170 377 170q370 0 370 -397v-711h-98v705q0 164 -69 238.5t-214 74.5q-195 0 -285.5 -98.5t-90.5 -319.5v-600h-99z" />
-<glyph unicode="o" horiz-adv-x="1200" d="M119 545q0 266 129 414.5t354 148.5q224 0 351.5 -150.5t127.5 -412.5q0 -266 -129 -415.5t-356 -149.5q-143 0 -252 69t-167 198t-58 298zM223 545q0 -224 98.5 -349.5t278.5 -125.5t278.5 125.5t98.5 349.5q0 225 -99.5 349t-279.5 124t-277.5 -123.5t-97.5 -349.5z " />
-<glyph unicode="p" horiz-adv-x="1219" d="M182 -492v1579h84l19 -155h6q112 176 358 176q220 0 335.5 -144.5t115.5 -420.5q0 -268 -121.5 -415.5t-331.5 -147.5q-251 0 -366 188h-7l3 -84q4 -74 4 -162v-414h-99zM281 541q0 -255 85.5 -364t278.5 -109q167 0 258.5 124t91.5 347q0 479 -346 479 q-190 0 -279 -104.5t-89 -340.5v-32z" />
-<glyph unicode="q" horiz-adv-x="1219" d="M119 532q0 275 118 425.5t338 150.5q236 0 353 -174h6l18 153h84v-1579h-98v414q0 122 6 248h-6q-118 -190 -369 -190q-214 0 -332 142t-118 410zM223 530q0 -229 89.5 -345.5t258.5 -116.5q198 0 282.5 109t84.5 366v12q0 245 -85 354t-271 109q-176 0 -267.5 -124 t-91.5 -364z" />
-<glyph unicode="r" horiz-adv-x="797" d="M182 0v1087h84l10 -196h7q67 120 143 168.5t184 48.5q69 0 148 -14l-19 -95q-68 17 -141 17q-139 0 -228 -118t-89 -298v-600h-99z" />
-<glyph unicode="s" horiz-adv-x="954" d="M84 47v107q164 -82 346 -82q161 0 244.5 53.5t83.5 142.5q0 82 -66.5 138t-218.5 110q-163 59 -229 101.5t-99.5 96t-33.5 130.5q0 122 102.5 193t286.5 71q176 0 334 -66l-37 -90q-160 66 -297 66q-133 0 -211 -44t-78 -122q0 -85 60.5 -136t236.5 -114 q147 -53 214 -95.5t100.5 -96.5t33.5 -127q0 -146 -111 -224.5t-315 -78.5q-218 0 -346 67z" />
-<glyph unicode="t" horiz-adv-x="686" d="M25 1001v58l161 45l50 246h51v-263h319v-86h-319v-688q0 -125 44 -185t138 -60t164 16v-80q-72 -24 -166 -24q-144 0 -212.5 77t-68.5 242v702h-161z" />
-<glyph unicode="u" horiz-adv-x="1208" d="M170 377v710h98v-704q0 -164 69 -238.5t214 -74.5q194 0 285.5 98t91.5 319v600h98v-1087h-84l-18 150h-6q-106 -170 -377 -170q-371 0 -371 397z" />
-<glyph unicode="v" horiz-adv-x="940" d="M0 1087h102l281 -739q56 -142 84 -248h6q41 136 84 250l281 737h102l-420 -1087h-100z" />
-<glyph unicode="w" horiz-adv-x="1481" d="M31 1087h106l174 -630q61 -234 80 -344h6q59 234 86 311l224 663h90l213 -661q72 -235 88 -311h6q8 65 80 348l166 624h100l-295 -1087h-104l-238 727q-23 74 -59 217h-6l-21 -74l-45 -145l-242 -725h-98z" />
-<glyph unicode="x" horiz-adv-x="1020" d="M55 0l394 559l-379 528h114l324 -458l321 458h109l-373 -528l400 -559h-115l-342 485l-344 -485h-109z" />
-<glyph unicode="y" horiz-adv-x="940" d="M0 1087h102l230 -610q105 -281 133 -379h6q42 129 137 385l230 604h102l-487 -1263q-59 -154 -99 -208t-93.5 -81t-129.5 -27q-57 0 -127 21v86q58 -16 125 -16q51 0 90 24t70.5 74.5t73 160t53.5 142.5z" />
-<glyph unicode="z" horiz-adv-x="944" d="M82 0v63l645 936h-598v88h727v-63l-649 -936h651v-88h-776z" />
-<glyph unicode="{" horiz-adv-x="723" d="M61 528v80q122 2 176 51t54 148v350q0 299 360 305v-90q-138 -5 -200 -58t-62 -157v-305q0 -130 -44 -194t-142 -85v-8q97 -20 141.5 -83.5t44.5 -186.5v-322q0 -102 59.5 -152.5t202.5 -53.5v-91q-195 0 -277.5 75t-82.5 231v337q0 205 -230 209z" />
-<glyph unicode="|" horiz-adv-x="1108" d="M508 -506v2067h92v-2067h-92z" />
-<glyph unicode="}" horiz-adv-x="723" d="M72 -233q141 2 201.5 52.5t60.5 153.5v322q0 123 44.5 186.5t141.5 83.5v8q-97 20 -141.5 84t-44.5 195v305q0 103 -61.5 156.5t-200.5 58.5v90q174 0 267 -77.5t93 -227.5v-350q0 -100 54.5 -148.5t175.5 -50.5v-80q-230 -4 -230 -209v-337q0 -155 -82.5 -230.5 t-277.5 -75.5v91z" />
-<glyph unicode="~" d="M111 625v94q108 110 233 110q61 0 115 -13.5t155 -57.5q126 -58 220 -58q56 0 109.5 30.5t115.5 94.5v-96q-48 -49 -104.5 -81t-129.5 -32q-116 0 -270 72q-124 57 -221 57q-49 0 -108 -30.5t-115 -89.5z" />
-<glyph unicode="&#xa1;" horiz-adv-x="492" d="M166 1010q0 98 80 98q82 0 82 -98q0 -53 -23.5 -76t-58.5 -23q-34 0 -57 23t-23 76zM186 -375l29 1086h61l29 -1086h-119z" />
-<glyph unicode="&#xa2;" d="M211 745q0 232 102.5 381.5t288.5 182.5v174h82v-166h14q131 0 275 -55l-31 -84q-134 51 -237 51q-187 0 -288.5 -122.5t-101.5 -358.5q0 -225 100.5 -349.5t280.5 -124.5q131 0 267 58v-92q-110 -56 -267 -56h-12v-204h-82v210q-186 30 -288.5 175t-102.5 380z" />
-<glyph unicode="&#xa3;" d="M78 0v84q110 21 171.5 110t61.5 224v258h-211v82h211v297q0 204 98 315t281 111q175 0 330 -68l-35 -86q-157 66 -295 66q-141 0 -209.5 -81t-68.5 -253v-301h411v-82h-411v-256q0 -116 -35 -196t-113 -128h809v-96h-995z" />
-<glyph unicode="&#xa4;" d="M127 326l139 141q-90 106 -90 256q0 147 90 258l-139 141l59 60l138 -142q103 93 260 93q155 0 260 -93l137 142l59 -60l-139 -141q90 -111 90 -258q0 -151 -90 -256l139 -141l-59 -60l-137 142q-110 -93 -260 -93q-153 0 -260 93l-138 -142zM260 723q0 -136 94.5 -232 t229.5 -96q134 0 228.5 95.5t94.5 232.5q0 136 -95 233t-228 97q-134 0 -229 -97t-95 -233z" />
-<glyph unicode="&#xa5;" d="M43 1462h117l426 -796l428 796h110l-432 -788h283v-82h-338v-205h338v-82h-338v-305h-105v305h-337v82h337v205h-337v82h278z" />
-<glyph unicode="&#xa6;" horiz-adv-x="1108" d="M508 258h92v-764h-92v764zM508 797v764h92v-764h-92z" />
-<glyph unicode="&#xa7;" horiz-adv-x="1057" d="M129 63v95q182 -78 332 -78q162 0 247 49.5t85 140.5q0 55 -25 87.5t-88.5 65.5t-190.5 79q-200 73 -272 141.5t-72 169.5q0 83 50.5 152.5t138.5 107.5q-86 47 -125 102t-39 136q0 117 101.5 183.5t275.5 66.5q175 0 336 -64l-35 -80q-91 34 -158.5 47t-144.5 13 q-134 0 -205.5 -44.5t-71.5 -119.5q0 -54 25.5 -88.5t85.5 -65.5t188 -74q192 -64 264 -132.5t72 -170.5q0 -173 -186 -274q86 -42 129 -96t43 -136q0 -135 -113 -207.5t-311 -72.5q-92 0 -171 15t-165 52zM246 825q0 -65 31.5 -104t105.5 -75t250 -99q82 41 126 98t44 121 q0 62 -32 102t-108.5 77t-236.5 87q-81 -23 -130.5 -79t-49.5 -128z" />
-<glyph unicode="&#xa8;" horiz-adv-x="1182" d="M336 1389q0 46 15.5 66t47.5 20q64 0 64 -86t-64 -86q-63 0 -63 86zM717 1389q0 46 15.5 66t47.5 20q64 0 64 -86t-64 -86q-63 0 -63 86z" />
-<glyph unicode="&#xa9;" horiz-adv-x="1704" d="M100 731q0 200 100 375t275 276t377 101q200 0 375 -100t276 -275t101 -377q0 -197 -97 -370t-272 -277t-383 -104q-207 0 -382 103.5t-272.5 276.5t-97.5 371zM193 731q0 -178 88.5 -329.5t240.5 -240.5t330 -89t329.5 88.5t240.5 240.5t89 330q0 174 -85.5 325 t-239 243t-334.5 92q-176 0 -328.5 -88.5t-241.5 -242.5t-89 -329zM489 725q0 208 111 332.5t297 124.5q119 0 227 -52l-37 -83q-98 45 -190 45q-142 0 -222.5 -94.5t-80.5 -264.5q0 -186 74.5 -275t220.5 -89q84 0 198 43v-88q-102 -45 -208 -45q-187 0 -288.5 115 t-101.5 331z" />
-<glyph unicode="&#xaa;" horiz-adv-x="686" d="M78 989q0 100 80 151.5t241 59.5l95 4v43q0 77 -38 114.5t-106 37.5q-87 0 -196 -49l-33 73q117 56 231 56q228 0 228 -215v-451h-68l-25 72q-84 -84 -202 -84q-95 0 -151 49t-56 139zM168 993q0 -54 35 -85t96 -31q90 0 142.5 50t52.5 142v64l-88 -5q-116 -6 -177 -36.5 t-61 -98.5z" />
-<glyph unicode="&#xab;" horiz-adv-x="885" d="M82 516v27l309 393l62 -43l-254 -363l254 -362l-62 -43zM442 516v27l310 393l61 -43l-254 -363l254 -362l-61 -43z" />
-<glyph unicode="&#xac;" d="M111 682v82h927v-494h-82v412h-845z" />
-<glyph unicode="&#xad;" horiz-adv-x="659" d="M92 512v82h475v-82h-475z" />
-<glyph unicode="&#xae;" horiz-adv-x="1704" d="M100 731q0 200 100 375t275 276t377 101q200 0 375 -100t276 -275t101 -377q0 -197 -97 -370t-272 -277t-383 -104q-207 0 -382 103.5t-272.5 276.5t-97.5 371zM193 731q0 -178 88.5 -329.5t240.5 -240.5t330 -89t329.5 88.5t240.5 240.5t89 330q0 174 -85.5 325 t-239 243t-334.5 92q-176 0 -328.5 -88.5t-241.5 -242.5t-89 -329zM608 291v880h211q143 0 222 -62t79 -191q0 -79 -38.5 -139.5t-110.5 -94.5l237 -393h-121l-210 360h-168v-360h-101zM709 731h112q91 0 143 46.5t52 135.5q0 172 -197 172h-110v-354z" />
-<glyph unicode="&#xaf;" horiz-adv-x="1024" d="M-6 1556v82h1036v-82h-1036z" />
-<glyph unicode="&#xb0;" horiz-adv-x="877" d="M139 1184q0 132 86.5 215.5t212.5 83.5t212.5 -83.5t86.5 -215.5t-86.5 -215.5t-212.5 -83.5q-130 0 -214.5 83t-84.5 216zM229 1184q0 -91 61 -154t148 -63q86 0 147.5 62t61.5 155q0 92 -60 154.5t-149 62.5q-90 0 -149.5 -64t-59.5 -153z" />
-<glyph unicode="&#xb1;" d="M111 1v82h948v-82h-948zM111 682v82h432v434h82v-434h434v-82h-434v-432h-82v432h-432z" />
-<glyph unicode="&#xb2;" horiz-adv-x="688" d="M53 586v78l242 237q125 121 172 193t47 149q0 71 -46.5 112.5t-123.5 41.5q-108 0 -217 -82l-49 65q119 103 270 103q124 0 194 -63.5t70 -174.5q0 -47 -13 -89t-40 -85.5t-68.5 -90t-308.5 -306.5h447v-88h-576z" />
-<glyph unicode="&#xb3;" horiz-adv-x="688" d="M41 629v88q136 -62 266 -62q115 0 174.5 49t59.5 136q0 83 -59.5 122t-178.5 39h-131v84h135q105 0 158 43.5t53 120.5q0 67 -47 107.5t-127 40.5q-128 0 -246 -78l-47 70q130 94 293 94q127 0 199.5 -60t72.5 -163q0 -78 -44 -131.5t-117 -75.5q186 -45 186 -211 q0 -130 -88.5 -201.5t-247.5 -71.5q-144 0 -264 60z" />
-<glyph unicode="&#xb4;" horiz-adv-x="1182" d="M393 1241v16q73 79 144.5 171.5t97.5 140.5h141v-17q-36 -52 -122.5 -138t-190.5 -173h-70z" />
-<glyph unicode="&#xb5;" horiz-adv-x="1221" d="M182 -492v1579h99v-704q0 -164 69 -238.5t213 -74.5q194 0 285.5 98t91.5 319v600h98v-1087h-84l-18 150h-6q-50 -77 -150 -123.5t-217 -46.5q-99 0 -167.5 27.5t-119.5 84.5q5 -92 5 -170v-414h-99z" />
-<glyph unicode="&#xb6;" horiz-adv-x="1341" d="M113 1042q0 260 109 387t341 127h543v-1816h-100v1722h-228v-1722h-100v819q-64 -18 -146 -18q-216 0 -317.5 125t-101.5 376z" />
-<glyph unicode="&#xb7;" horiz-adv-x="487" d="M162 721q0 98 80 98q82 0 82 -98t-82 -98q-80 0 -80 98z" />
-<glyph unicode="&#xb8;" horiz-adv-x="420" d="M43 -393q30 -10 92 -10q78 0 119 28t41 80q0 94 -193 121l93 174h96l-66 -117q168 -37 168 -174q0 -100 -67.5 -150.5t-188.5 -50.5q-68 0 -94 11v88z" />
-<glyph unicode="&#xb9;" horiz-adv-x="688" d="M76 1298l274 164h92v-876h-98v547q0 99 12 233q-26 -23 -233 -145z" />
-<glyph unicode="&#xba;" horiz-adv-x="739" d="M70 1141q0 162 78 250t223 88q142 0 220.5 -87t78.5 -251q0 -161 -80 -250.5t-223 -89.5t-220 86t-77 254zM160 1141q0 -264 209 -264t209 264q0 131 -50 194.5t-159 63.5t-159 -63.5t-50 -194.5z" />
-<glyph unicode="&#xbb;" horiz-adv-x="885" d="M72 168l254 362l-254 363l61 43l309 -391v-27l-309 -393zM432 168l254 362l-254 363l62 43l309 -391v-27l-309 -393z" />
-<glyph unicode="&#xbc;" horiz-adv-x="1516" d="M59 1298l274 164h92v-876h-98v547q0 99 12 233q-26 -23 -233 -145zM243 0l811 1462h94l-811 -1462h-94zM760 242v60l407 581h96v-563h129v-78h-129v-241h-90v241h-413zM864 320h309v221q0 132 8 232q-6 -12 -21.5 -35.5t-295.5 -417.5z" />
-<glyph unicode="&#xbd;" horiz-adv-x="1516" d="M11 1298l274 164h92v-876h-98v547q0 99 12 233q-26 -23 -233 -145zM168 0l811 1462h94l-811 -1462h-94zM827 1v78l242 237q125 121 172 193t47 149q0 71 -46.5 112.5t-123.5 41.5q-108 0 -217 -82l-49 65q119 103 270 103q124 0 194 -63.5t70 -174.5q0 -47 -13 -89 t-40 -85.5t-68.5 -90t-308.5 -306.5h447v-88h-576z" />
-<glyph unicode="&#xbe;" horiz-adv-x="1516" d="M41 629v88q136 -62 266 -62q115 0 174.5 49t59.5 136q0 83 -59.5 122t-178.5 39h-131v84h135q105 0 158 43.5t53 120.5q0 67 -47 107.5t-127 40.5q-128 0 -246 -78l-47 70q130 94 293 94q127 0 199.5 -60t72.5 -163q0 -78 -44 -131.5t-117 -75.5q186 -45 186 -211 q0 -130 -88.5 -201.5t-247.5 -71.5q-144 0 -264 60zM395 0l811 1462h94l-811 -1462h-94zM863 242v60l407 581h96v-563h129v-78h-129v-241h-90v241h-413zM967 320h309v221q0 132 8 232q-6 -12 -21.5 -35.5t-295.5 -417.5z" />
-<glyph unicode="&#xbf;" horiz-adv-x="862" d="M74 -27q0 70 20 124t58.5 102t171.5 159q64 53 98.5 98.5t49.5 94t15 145.5v15h82v-37q0 -125 -39.5 -204.5t-136.5 -164.5l-90 -79q-73 -61 -104 -120.5t-31 -138.5q0 -124 82 -200t221 -76q125 0 233 46l64 27l37 -79q-111 -48 -185.5 -64t-152.5 -16q-184 0 -288.5 99 t-104.5 269zM440 1010q0 98 80 98q82 0 82 -98q0 -53 -23.5 -76t-58.5 -23q-34 0 -57 23t-23 76z" />
-<glyph unicode="&#xc0;" horiz-adv-x="1229" d="M0 0l588 1468h65l576 -1468h-115l-203 516h-594l-204 -516h-113zM337 1890v17h142q26 -48 98.5 -142t142.5 -170v-16h-69q-96 79 -188.5 171.5t-125.5 139.5zM354 608h523l-199 527q-25 62 -60 172q-27 -96 -59 -174z" />
-<glyph unicode="&#xc1;" horiz-adv-x="1229" d="M0 0l588 1468h65l576 -1468h-115l-203 516h-594l-204 -516h-113zM354 608h523l-199 527q-25 62 -60 172q-27 -96 -59 -174zM504 1579v16q73 79 144.5 171.5t97.5 140.5h141v-17q-36 -52 -122.5 -138t-190.5 -173h-70z" />
-<glyph unicode="&#xc2;" horiz-adv-x="1229" d="M0 0l588 1468h65l576 -1468h-115l-203 516h-594l-204 -516h-113zM328 1579v16q62 67 131.5 156t110.5 156h98q68 -120 242 -312v-16h-70q-122 101 -221 207q-108 -114 -221 -207h-70zM354 608h523l-199 527q-25 62 -60 172q-27 -96 -59 -174z" />
-<glyph unicode="&#xc3;" horiz-adv-x="1229" d="M0 0l588 1468h65l576 -1468h-115l-203 516h-594l-204 -516h-113zM287 1581q10 111 63 174.5t137 63.5q48 0 88 -25t82 -59q34 -28 66 -50t61 -22q46 0 77 36.5t48 119.5h76q-16 -116 -69 -177t-132 -61q-36 0 -75 18.5t-101 71.5q-32 26 -62.5 46t-62.5 20 q-45 0 -75 -34.5t-48 -121.5h-73zM354 608h523l-199 527q-25 62 -60 172q-27 -96 -59 -174z" />
-<glyph unicode="&#xc4;" horiz-adv-x="1229" d="M0 0l588 1468h65l576 -1468h-115l-203 516h-594l-204 -516h-113zM354 608h523l-199 527q-25 62 -60 172q-27 -96 -59 -174zM367 1727q0 46 15.5 66t47.5 20q64 0 64 -86t-64 -86q-63 0 -63 86zM748 1727q0 46 15.5 66t47.5 20q64 0 64 -86t-64 -86q-63 0 -63 86z" />
-<glyph unicode="&#xc5;" horiz-adv-x="1229" d="M0 0l588 1468h65l576 -1468h-115l-203 516h-594l-204 -516h-113zM354 608h523l-199 527q-25 62 -60 172q-27 -96 -59 -174zM402 1610q0 94 60 152.5t157 58.5t157 -59t60 -152q0 -97 -60 -155t-157 -58t-157 58t-60 155zM482 1610q0 -66 37.5 -103.5t99.5 -37.5 t99.5 37.5t37.5 103.5q0 64 -39 101.5t-98 37.5q-62 0 -99.5 -38t-37.5 -101z" />
-<glyph unicode="&#xc6;" horiz-adv-x="1653" d="M-2 0l653 1462h877v-94h-615v-553h576v-94h-576v-627h615v-94h-717v516h-475l-227 -516h-111zM377 608h434v760h-100z" />
-<glyph unicode="&#xc7;" horiz-adv-x="1272" d="M129 735q0 223 84.5 393t243 262.5t368.5 92.5q214 0 383 -80l-41 -92q-160 80 -336 80q-275 0 -433 -176t-158 -482q0 -313 149 -486t426 -173q184 0 338 47v-90q-145 -51 -362 -51q-308 0 -485 199t-177 556zM561 -393q30 -10 92 -10q78 0 119 28t41 80q0 94 -193 121 l93 174h96l-66 -117q168 -37 168 -174q0 -100 -67.5 -150.5t-188.5 -50.5q-68 0 -94 11v88z" />
-<glyph unicode="&#xc8;" horiz-adv-x="1130" d="M207 0v1462h799v-94h-697v-553h658v-94h-658v-627h697v-94h-799zM314 1890v17h142q26 -48 98.5 -142t142.5 -170v-16h-69q-96 79 -188.5 171.5t-125.5 139.5z" />
-<glyph unicode="&#xc9;" horiz-adv-x="1130" d="M207 0v1462h799v-94h-697v-553h658v-94h-658v-627h697v-94h-799zM463 1579v16q73 79 144.5 171.5t97.5 140.5h141v-17q-36 -52 -122.5 -138t-190.5 -173h-70z" />
-<glyph unicode="&#xca;" horiz-adv-x="1130" d="M207 0v1462h799v-94h-697v-553h658v-94h-658v-627h697v-94h-799zM315 1579v16q62 67 131.5 156t110.5 156h98q68 -120 242 -312v-16h-70q-122 101 -221 207q-108 -114 -221 -207h-70z" />
-<glyph unicode="&#xcb;" horiz-adv-x="1130" d="M207 0v1462h799v-94h-697v-553h658v-94h-658v-627h697v-94h-799zM354 1727q0 46 15.5 66t47.5 20q64 0 64 -86t-64 -86q-63 0 -63 86zM735 1727q0 46 15.5 66t47.5 20q64 0 64 -86t-64 -86q-63 0 -63 86z" />
-<glyph unicode="&#xcc;" horiz-adv-x="516" d="M-63 1890v17h142q26 -48 98.5 -142t142.5 -170v-16h-69q-96 79 -188.5 171.5t-125.5 139.5zM207 0v1462h102v-1462h-102z" />
-<glyph unicode="&#xcd;" horiz-adv-x="516" d="M191 1579v16q73 79 144.5 171.5t97.5 140.5h141v-17q-36 -52 -122.5 -138t-190.5 -173h-70zM207 0v1462h102v-1462h-102z" />
-<glyph unicode="&#xce;" horiz-adv-x="516" d="M-32 1579v16q62 67 131.5 156t110.5 156h98q68 -120 242 -312v-16h-70q-122 101 -221 207q-108 -114 -221 -207h-70zM207 0v1462h102v-1462h-102z" />
-<glyph unicode="&#xcf;" horiz-adv-x="516" d="M5 1727q0 46 15.5 66t47.5 20q64 0 64 -86t-64 -86q-63 0 -63 86zM207 0v1462h102v-1462h-102zM386 1727q0 46 15.5 66t47.5 20q64 0 64 -86t-64 -86q-63 0 -63 86z" />
-<glyph unicode="&#xd0;" horiz-adv-x="1466" d="M47 678v94h160v690h395q350 0 532.5 -183t182.5 -534q0 -368 -193 -556.5t-567 -188.5h-350v678h-160zM309 90h242q655 0 655 651q0 314 -159.5 472.5t-468.5 158.5h-269v-600h406v-94h-406v-588z" />
-<glyph unicode="&#xd1;" horiz-adv-x="1477" d="M207 0v1462h102l865 -1296h6q-9 180 -9 342v954h99v-1462h-103l-866 1298h-8q12 -232 12 -350v-948h-98zM400 1581q10 111 63 174.5t137 63.5q48 0 88 -25t82 -59q34 -28 66 -50t61 -22q46 0 77 36.5t48 119.5h76q-16 -116 -69 -177t-132 -61q-36 0 -75 18.5t-101 71.5 q-32 26 -62.5 46t-62.5 20q-45 0 -75 -34.5t-48 -121.5h-73z" />
-<glyph unicode="&#xd2;" horiz-adv-x="1565" d="M129 735q0 349 175.5 549.5t479.5 200.5q306 0 479 -201.5t173 -550.5q0 -348 -174 -550.5t-480 -202.5q-305 0 -479 202.5t-174 552.5zM240 733q0 -314 140 -485.5t402 -171.5q264 0 403.5 170t139.5 487q0 316 -139.5 484.5t-401.5 168.5q-261 0 -402.5 -170 t-141.5 -483zM502 1890v17h142q26 -48 98.5 -142t142.5 -170v-16h-69q-96 79 -188.5 171.5t-125.5 139.5z" />
-<glyph unicode="&#xd3;" horiz-adv-x="1565" d="M129 735q0 349 175.5 549.5t479.5 200.5q306 0 479 -201.5t173 -550.5q0 -348 -174 -550.5t-480 -202.5q-305 0 -479 202.5t-174 552.5zM240 733q0 -314 140 -485.5t402 -171.5q264 0 403.5 170t139.5 487q0 316 -139.5 484.5t-401.5 168.5q-261 0 -402.5 -170 t-141.5 -483zM686 1579v16q73 79 144.5 171.5t97.5 140.5h141v-17q-36 -52 -122.5 -138t-190.5 -173h-70z" />
-<glyph unicode="&#xd4;" horiz-adv-x="1565" d="M129 735q0 349 175.5 549.5t479.5 200.5q306 0 479 -201.5t173 -550.5q0 -348 -174 -550.5t-480 -202.5q-305 0 -479 202.5t-174 552.5zM240 733q0 -314 140 -485.5t402 -171.5q264 0 403.5 170t139.5 487q0 316 -139.5 484.5t-401.5 168.5q-261 0 -402.5 -170 t-141.5 -483zM492 1579v16q62 67 131.5 156t110.5 156h98q68 -120 242 -312v-16h-70q-122 101 -221 207q-108 -114 -221 -207h-70z" />
-<glyph unicode="&#xd5;" horiz-adv-x="1565" d="M129 735q0 349 175.5 549.5t479.5 200.5q306 0 479 -201.5t173 -550.5q0 -348 -174 -550.5t-480 -202.5q-305 0 -479 202.5t-174 552.5zM240 733q0 -314 140 -485.5t402 -171.5q264 0 403.5 170t139.5 487q0 316 -139.5 484.5t-401.5 168.5q-261 0 -402.5 -170 t-141.5 -483zM443 1581q10 111 63 174.5t137 63.5q48 0 88 -25t82 -59q34 -28 66 -50t61 -22q46 0 77 36.5t48 119.5h76q-16 -116 -69 -177t-132 -61q-36 0 -75 18.5t-101 71.5q-32 26 -62.5 46t-62.5 20q-45 0 -75 -34.5t-48 -121.5h-73z" />
-<glyph unicode="&#xd6;" horiz-adv-x="1565" d="M129 735q0 349 175.5 549.5t479.5 200.5q306 0 479 -201.5t173 -550.5q0 -348 -174 -550.5t-480 -202.5q-305 0 -479 202.5t-174 552.5zM240 733q0 -314 140 -485.5t402 -171.5q264 0 403.5 170t139.5 487q0 316 -139.5 484.5t-401.5 168.5q-261 0 -402.5 -170 t-141.5 -483zM529 1727q0 46 15.5 66t47.5 20q64 0 64 -86t-64 -86q-63 0 -63 86zM910 1727q0 46 15.5 66t47.5 20q64 0 64 -86t-64 -86q-63 0 -63 86z" />
-<glyph unicode="&#xd7;" d="M119 1130l57 58l408 -408l409 408l58 -58l-408 -407l406 -408l-58 -57l-407 408l-406 -408l-57 57l405 408z" />
-<glyph unicode="&#xd8;" horiz-adv-x="1565" d="M129 735q0 349 175.5 549.5t479.5 200.5q232 0 392 -121l108 152l72 -60l-111 -153q191 -207 191 -570q0 -348 -174 -550.5t-480 -202.5q-236 0 -395 120l-86 -120l-74 59l90 127q-188 200 -188 569zM240 733q0 -312 139 -483l739 1034q-133 102 -334 102 q-261 0 -402.5 -170t-141.5 -483zM444 182q133 -106 338 -106q264 0 403.5 170t139.5 487q0 315 -139 486z" />
-<glyph unicode="&#xd9;" horiz-adv-x="1473" d="M190 520v942h103v-946q0 -211 117 -328.5t331 -117.5q209 0 324 115.5t115 320.5v956h102v-946q0 -252 -146 -394t-407 -142q-254 0 -396.5 142.5t-142.5 397.5zM450 1890v17h142q26 -48 98.5 -142t142.5 -170v-16h-69q-96 79 -188.5 171.5t-125.5 139.5z" />
-<glyph unicode="&#xda;" horiz-adv-x="1473" d="M190 520v942h103v-946q0 -211 117 -328.5t331 -117.5q209 0 324 115.5t115 320.5v956h102v-946q0 -252 -146 -394t-407 -142q-254 0 -396.5 142.5t-142.5 397.5zM633 1579v16q73 79 144.5 171.5t97.5 140.5h141v-17q-36 -52 -122.5 -138t-190.5 -173h-70z" />
-<glyph unicode="&#xdb;" horiz-adv-x="1473" d="M190 520v942h103v-946q0 -211 117 -328.5t331 -117.5q209 0 324 115.5t115 320.5v956h102v-946q0 -252 -146 -394t-407 -142q-254 0 -396.5 142.5t-142.5 397.5zM444 1579v16q62 67 131.5 156t110.5 156h98q68 -120 242 -312v-16h-70q-122 101 -221 207 q-108 -114 -221 -207h-70z" />
-<glyph unicode="&#xdc;" horiz-adv-x="1473" d="M190 520v942h103v-946q0 -211 117 -328.5t331 -117.5q209 0 324 115.5t115 320.5v956h102v-946q0 -252 -146 -394t-407 -142q-254 0 -396.5 142.5t-142.5 397.5zM481 1727q0 46 15.5 66t47.5 20q64 0 64 -86t-64 -86q-63 0 -63 86zM862 1727q0 46 15.5 66t47.5 20 q64 0 64 -86t-64 -86q-63 0 -63 86z" />
-<glyph unicode="&#xdd;" horiz-adv-x="1081" d="M0 1462h117l426 -800l428 800h110l-487 -897v-565h-105v557zM434 1579v16q73 79 144.5 171.5t97.5 140.5h141v-17q-36 -52 -122.5 -138t-190.5 -173h-70z" />
-<glyph unicode="&#xde;" horiz-adv-x="1198" d="M207 0v1462h102v-264h256q522 0 522 -420q0 -212 -144 -325t-408 -113h-226v-340h-102zM309 428h201q247 0 357 81.5t110 264.5q0 169 -104 250.5t-322 81.5h-242v-678z" />
-<glyph unicode="&#xdf;" horiz-adv-x="1194" d="M182 0v1206q0 173 103.5 267t292.5 94q188 0 285.5 -72.5t97.5 -210.5q0 -139 -139 -250q-81 -64 -110.5 -100.5t-29.5 -75.5q0 -44 14.5 -68t51.5 -57t102 -78q106 -75 151.5 -124.5t68 -103t22.5 -120.5q0 -156 -88 -241.5t-246 -85.5q-95 0 -174.5 18.5t-126.5 48.5 v107q65 -38 148.5 -62t152.5 -24q114 0 174.5 54.5t60.5 160.5q0 83 -39 144t-149 136q-127 87 -175 147t-48 146q0 60 32.5 110t106.5 108q74 57 106.5 105.5t32.5 106.5q0 93 -70 143t-202 50q-145 0 -226 -69t-81 -196v-1214h-99z" />
-<glyph unicode="&#xe0;" horiz-adv-x="1085" d="M98 289q0 159 132.5 247t383.5 93l207 6v72q0 155 -63 234t-203 79q-151 0 -313 -84l-37 86q179 84 354 84q179 0 267.5 -93t88.5 -290v-723h-73l-25 172h-8q-82 -105 -168.5 -148.5t-204.5 -43.5q-160 0 -249 82t-89 227zM203 285q0 -102 62.5 -158.5t176.5 -56.5 q174 0 274.5 99.5t100.5 276.5v107l-190 -8q-229 -11 -326.5 -71.5t-97.5 -188.5zM255 1552v17h142q26 -48 98.5 -142t142.5 -170v-16h-69q-96 79 -188.5 171.5t-125.5 139.5z" />
-<glyph unicode="&#xe1;" horiz-adv-x="1085" d="M98 289q0 159 132.5 247t383.5 93l207 6v72q0 155 -63 234t-203 79q-151 0 -313 -84l-37 86q179 84 354 84q179 0 267.5 -93t88.5 -290v-723h-73l-25 172h-8q-82 -105 -168.5 -148.5t-204.5 -43.5q-160 0 -249 82t-89 227zM203 285q0 -102 62.5 -158.5t176.5 -56.5 q174 0 274.5 99.5t100.5 276.5v107l-190 -8q-229 -11 -326.5 -71.5t-97.5 -188.5zM422 1241v16q73 79 144.5 171.5t97.5 140.5h141v-17q-36 -52 -122.5 -138t-190.5 -173h-70z" />
-<glyph unicode="&#xe2;" horiz-adv-x="1085" d="M98 289q0 159 132.5 247t383.5 93l207 6v72q0 155 -63 234t-203 79q-151 0 -313 -84l-37 86q179 84 354 84q179 0 267.5 -93t88.5 -290v-723h-73l-25 172h-8q-82 -105 -168.5 -148.5t-204.5 -43.5q-160 0 -249 82t-89 227zM203 285q0 -102 62.5 -158.5t176.5 -56.5 q174 0 274.5 99.5t100.5 276.5v107l-190 -8q-229 -11 -326.5 -71.5t-97.5 -188.5zM251 1241v16q62 67 131.5 156t110.5 156h98q68 -120 242 -312v-16h-70q-122 101 -221 207q-108 -114 -221 -207h-70z" />
-<glyph unicode="&#xe3;" horiz-adv-x="1085" d="M98 289q0 159 132.5 247t383.5 93l207 6v72q0 155 -63 234t-203 79q-151 0 -313 -84l-37 86q179 84 354 84q179 0 267.5 -93t88.5 -290v-723h-73l-25 172h-8q-82 -105 -168.5 -148.5t-204.5 -43.5q-160 0 -249 82t-89 227zM200 1243q10 111 63 174.5t137 63.5 q48 0 88 -25t82 -59q34 -28 66 -50t61 -22q46 0 77 36.5t48 119.5h76q-16 -116 -69 -177t-132 -61q-36 0 -75 18.5t-101 71.5q-32 26 -62.5 46t-62.5 20q-45 0 -75 -34.5t-48 -121.5h-73zM203 285q0 -102 62.5 -158.5t176.5 -56.5q174 0 274.5 99.5t100.5 276.5v107l-190 -8 q-229 -11 -326.5 -71.5t-97.5 -188.5z" />
-<glyph unicode="&#xe4;" horiz-adv-x="1085" d="M98 289q0 159 132.5 247t383.5 93l207 6v72q0 155 -63 234t-203 79q-151 0 -313 -84l-37 86q179 84 354 84q179 0 267.5 -93t88.5 -290v-723h-73l-25 172h-8q-82 -105 -168.5 -148.5t-204.5 -43.5q-160 0 -249 82t-89 227zM203 285q0 -102 62.5 -158.5t176.5 -56.5 q174 0 274.5 99.5t100.5 276.5v107l-190 -8q-229 -11 -326.5 -71.5t-97.5 -188.5zM282 1389q0 46 15.5 66t47.5 20q64 0 64 -86t-64 -86q-63 0 -63 86zM663 1389q0 46 15.5 66t47.5 20q64 0 64 -86t-64 -86q-63 0 -63 86z" />
-<glyph unicode="&#xe5;" horiz-adv-x="1085" d="M98 289q0 159 132.5 247t383.5 93l207 6v72q0 155 -63 234t-203 79q-151 0 -313 -84l-37 86q179 84 354 84q179 0 267.5 -93t88.5 -290v-723h-73l-25 172h-8q-82 -105 -168.5 -148.5t-204.5 -43.5q-160 0 -249 82t-89 227zM203 285q0 -102 62.5 -158.5t176.5 -56.5 q174 0 274.5 99.5t100.5 276.5v107l-190 -8q-229 -11 -326.5 -71.5t-97.5 -188.5zM325 1456q0 94 60 152.5t157 58.5t157 -59t60 -152q0 -97 -60 -155t-157 -58t-157 58t-60 155zM405 1456q0 -66 37.5 -103.5t99.5 -37.5t99.5 37.5t37.5 103.5q0 64 -39 101.5t-98 37.5 q-62 0 -99.5 -38t-37.5 -101z" />
-<glyph unicode="&#xe6;" horiz-adv-x="1731" d="M98 289q0 154 125 243t377 97l201 6v72q0 155 -61.5 234t-198.5 79q-148 0 -305 -84l-37 86q173 84 346 84q261 0 325 -211q111 213 347 213q184 0 289.5 -134.5t105.5 -363.5v-80h-715q0 -460 348 -460q85 0 150 12t174 57v-90q-92 -41 -165 -55t-161 -14 q-295 0 -397 256q-68 -133 -168 -194.5t-252 -61.5q-156 0 -242 82.5t-86 226.5zM203 285q0 -102 61 -158.5t170 -56.5q169 0 266 99.5t97 276.5v107l-187 -8q-219 -11 -313 -71.5t-94 -188.5zM903 618h604q0 188 -77.5 295t-212.5 107q-284 0 -314 -402z" />
-<glyph unicode="&#xe7;" horiz-adv-x="973" d="M119 537q0 270 137 420.5t375 150.5q141 0 270 -49l-27 -88q-141 47 -245 47q-200 0 -303 -123.5t-103 -355.5q0 -220 103 -344.5t288 -124.5q148 0 275 53v-92q-104 -51 -273 -51q-233 0 -365 147t-132 410zM373 -393q30 -10 92 -10q78 0 119 28t41 80q0 94 -193 121 l93 174h96l-66 -117q168 -37 168 -174q0 -100 -67.5 -150.5t-188.5 -50.5q-68 0 -94 11v88z" />
-<glyph unicode="&#xe8;" horiz-adv-x="1124" d="M119 535q0 260 128 416.5t345 156.5q192 0 303 -134t111 -364v-80h-783q2 -224 104.5 -342t293.5 -118q93 0 163.5 13t178.5 56v-90q-92 -40 -170 -54.5t-172 -14.5q-237 0 -369.5 146t-132.5 409zM229 618h672q0 189 -82 295.5t-227 106.5q-157 0 -252 -103.5 t-111 -298.5zM302 1552v17h142q26 -48 98.5 -142t142.5 -170v-16h-69q-96 79 -188.5 171.5t-125.5 139.5z" />
-<glyph unicode="&#xe9;" horiz-adv-x="1124" d="M119 535q0 260 128 416.5t345 156.5q192 0 303 -134t111 -364v-80h-783q2 -224 104.5 -342t293.5 -118q93 0 163.5 13t178.5 56v-90q-92 -40 -170 -54.5t-172 -14.5q-237 0 -369.5 146t-132.5 409zM229 618h672q0 189 -82 295.5t-227 106.5q-157 0 -252 -103.5 t-111 -298.5zM452 1241v16q73 79 144.5 171.5t97.5 140.5h141v-17q-36 -52 -122.5 -138t-190.5 -173h-70z" />
-<glyph unicode="&#xea;" horiz-adv-x="1124" d="M119 535q0 260 128 416.5t345 156.5q192 0 303 -134t111 -364v-80h-783q2 -224 104.5 -342t293.5 -118q93 0 163.5 13t178.5 56v-90q-92 -40 -170 -54.5t-172 -14.5q-237 0 -369.5 146t-132.5 409zM229 618h672q0 189 -82 295.5t-227 106.5q-157 0 -252 -103.5 t-111 -298.5zM290 1241v16q62 67 131.5 156t110.5 156h98q68 -120 242 -312v-16h-70q-122 101 -221 207q-108 -114 -221 -207h-70z" />
-<glyph unicode="&#xeb;" horiz-adv-x="1124" d="M119 535q0 260 128 416.5t345 156.5q192 0 303 -134t111 -364v-80h-783q2 -224 104.5 -342t293.5 -118q93 0 163.5 13t178.5 56v-90q-92 -40 -170 -54.5t-172 -14.5q-237 0 -369.5 146t-132.5 409zM229 618h672q0 189 -82 295.5t-227 106.5q-157 0 -252 -103.5 t-111 -298.5zM331 1389q0 46 15.5 66t47.5 20q64 0 64 -86t-64 -86q-63 0 -63 86zM712 1389q0 46 15.5 66t47.5 20q64 0 64 -86t-64 -86q-63 0 -63 86z" />
-<glyph unicode="&#xec;" horiz-adv-x="463" d="M-34 1552v17h142q26 -48 98.5 -142t142.5 -170v-16h-69q-96 79 -188.5 171.5t-125.5 139.5zM182 0v1087h99v-1087h-99z" />
-<glyph unicode="&#xed;" horiz-adv-x="463" d="M107 1241v16q73 79 144.5 171.5t97.5 140.5h141v-17q-36 -52 -122.5 -138t-190.5 -173h-70zM182 0v1087h99v-1087h-99z" />
-<glyph unicode="&#xee;" horiz-adv-x="463" d="M-58 1241v16q62 67 131.5 156t110.5 156h98q68 -120 242 -312v-16h-70q-122 101 -221 207q-108 -114 -221 -207h-70zM182 0v1087h99v-1087h-99z" />
-<glyph unicode="&#xef;" horiz-adv-x="463" d="M-21 1389q0 46 15.5 66t47.5 20q64 0 64 -86t-64 -86q-63 0 -63 86zM182 0v1087h99v-1087h-99zM360 1389q0 46 15.5 66t47.5 20q64 0 64 -86t-64 -86q-63 0 -63 86z" />
-<glyph unicode="&#xf0;" horiz-adv-x="1174" d="M117 471q0 228 126.5 357.5t342.5 129.5q108 0 187.5 -33t148.5 -96l4 2q-64 270 -269 459l-270 -157l-49 77l244 146q-86 62 -199 119l45 81q147 -69 248 -145l225 137l49 -84l-202 -121q154 -151 230.5 -353t76.5 -431q0 -276 -124 -427.5t-349 -151.5 q-214 0 -339.5 130t-125.5 361zM221 463q0 -186 94.5 -289.5t268.5 -103.5q179 0 272.5 123t93.5 364q0 146 -97 228.5t-267 82.5q-185 0 -275 -100.5t-90 -304.5z" />
-<glyph unicode="&#xf1;" horiz-adv-x="1208" d="M182 0v1087h84l19 -149h6q106 170 377 170q370 0 370 -397v-711h-98v705q0 164 -69 238.5t-214 74.5q-195 0 -285.5 -98.5t-90.5 -319.5v-600h-99zM282 1243q10 111 63 174.5t137 63.5q48 0 88 -25t82 -59q34 -28 66 -50t61 -22q46 0 77 36.5t48 119.5h76 q-16 -116 -69 -177t-132 -61q-36 0 -75 18.5t-101 71.5q-32 26 -62.5 46t-62.5 20q-45 0 -75 -34.5t-48 -121.5h-73z" />
-<glyph unicode="&#xf2;" horiz-adv-x="1200" d="M119 545q0 266 129 414.5t354 148.5q224 0 351.5 -150.5t127.5 -412.5q0 -266 -129 -415.5t-356 -149.5q-143 0 -252 69t-167 198t-58 298zM223 545q0 -224 98.5 -349.5t278.5 -125.5t278.5 125.5t98.5 349.5q0 225 -99.5 349t-279.5 124t-277.5 -123.5t-97.5 -349.5z M335 1552v17h142q26 -48 98.5 -142t142.5 -170v-16h-69q-96 79 -188.5 171.5t-125.5 139.5z" />
-<glyph unicode="&#xf3;" horiz-adv-x="1200" d="M119 545q0 266 129 414.5t354 148.5q224 0 351.5 -150.5t127.5 -412.5q0 -266 -129 -415.5t-356 -149.5q-143 0 -252 69t-167 198t-58 298zM223 545q0 -224 98.5 -349.5t278.5 -125.5t278.5 125.5t98.5 349.5q0 225 -99.5 349t-279.5 124t-277.5 -123.5t-97.5 -349.5z M499 1241v16q73 79 144.5 171.5t97.5 140.5h141v-17q-36 -52 -122.5 -138t-190.5 -173h-70z" />
-<glyph unicode="&#xf4;" horiz-adv-x="1200" d="M119 545q0 266 129 414.5t354 148.5q224 0 351.5 -150.5t127.5 -412.5q0 -266 -129 -415.5t-356 -149.5q-143 0 -252 69t-167 198t-58 298zM223 545q0 -224 98.5 -349.5t278.5 -125.5t278.5 125.5t98.5 349.5q0 225 -99.5 349t-279.5 124t-277.5 -123.5t-97.5 -349.5z M309 1241v16q62 67 131.5 156t110.5 156h98q68 -120 242 -312v-16h-70q-122 101 -221 207q-108 -114 -221 -207h-70z" />
-<glyph unicode="&#xf5;" horiz-adv-x="1200" d="M119 545q0 266 129 414.5t354 148.5q224 0 351.5 -150.5t127.5 -412.5q0 -266 -129 -415.5t-356 -149.5q-143 0 -252 69t-167 198t-58 298zM223 545q0 -224 98.5 -349.5t278.5 -125.5t278.5 125.5t98.5 349.5q0 225 -99.5 349t-279.5 124t-277.5 -123.5t-97.5 -349.5z M264 1243q10 111 63 174.5t137 63.5q48 0 88 -25t82 -59q34 -28 66 -50t61 -22q46 0 77 36.5t48 119.5h76q-16 -116 -69 -177t-132 -61q-36 0 -75 18.5t-101 71.5q-32 26 -62.5 46t-62.5 20q-45 0 -75 -34.5t-48 -121.5h-73z" />
-<glyph unicode="&#xf6;" horiz-adv-x="1200" d="M119 545q0 266 129 414.5t354 148.5q224 0 351.5 -150.5t127.5 -412.5q0 -266 -129 -415.5t-356 -149.5q-143 0 -252 69t-167 198t-58 298zM223 545q0 -224 98.5 -349.5t278.5 -125.5t278.5 125.5t98.5 349.5q0 225 -99.5 349t-279.5 124t-277.5 -123.5t-97.5 -349.5z M346 1389q0 46 15.5 66t47.5 20q64 0 64 -86t-64 -86q-63 0 -63 86zM727 1389q0 46 15.5 66t47.5 20q64 0 64 -86t-64 -86q-63 0 -63 86z" />
-<glyph unicode="&#xf7;" d="M111 682v82h948v-82h-948zM504 371q0 98 80 98q82 0 82 -98q0 -53 -23.5 -76t-58.5 -23q-34 0 -57 23t-23 76zM504 1075q0 99 80 99q82 0 82 -99q0 -52 -23.5 -75t-58.5 -23q-34 0 -57 23t-23 75z" />
-<glyph unicode="&#xf8;" horiz-adv-x="1200" d="M119 545q0 266 129 414.5t354 148.5q179 0 301 -104l96 124l74 -55l-104 -137q112 -147 112 -391q0 -266 -129 -415.5t-356 -149.5q-173 0 -291 98l-86 -113l-72 58l93 120q-121 153 -121 402zM223 545q0 -200 78 -322l543 705q-98 90 -246 90q-180 0 -277.5 -123.5 t-97.5 -349.5zM362 152q94 -82 238 -82q180 0 278.5 125.5t98.5 349.5q0 190 -72 309z" />
-<glyph unicode="&#xf9;" horiz-adv-x="1208" d="M170 377v710h98v-704q0 -164 69 -238.5t214 -74.5q194 0 285.5 98t91.5 319v600h98v-1087h-84l-18 150h-6q-106 -170 -377 -170q-371 0 -371 397zM304 1552v17h142q26 -48 98.5 -142t142.5 -170v-16h-69q-96 79 -188.5 171.5t-125.5 139.5z" />
-<glyph unicode="&#xfa;" horiz-adv-x="1208" d="M170 377v710h98v-704q0 -164 69 -238.5t214 -74.5q194 0 285.5 98t91.5 319v600h98v-1087h-84l-18 150h-6q-106 -170 -377 -170q-371 0 -371 397zM495 1241v16q73 79 144.5 171.5t97.5 140.5h141v-17q-36 -52 -122.5 -138t-190.5 -173h-70z" />
-<glyph unicode="&#xfb;" horiz-adv-x="1208" d="M170 377v710h98v-704q0 -164 69 -238.5t214 -74.5q194 0 285.5 98t91.5 319v600h98v-1087h-84l-18 150h-6q-106 -170 -377 -170q-371 0 -371 397zM313 1241v16q62 67 131.5 156t110.5 156h98q68 -120 242 -312v-16h-70q-122 101 -221 207q-108 -114 -221 -207h-70z" />
-<glyph unicode="&#xfc;" horiz-adv-x="1208" d="M170 377v710h98v-704q0 -164 69 -238.5t214 -74.5q194 0 285.5 98t91.5 319v600h98v-1087h-84l-18 150h-6q-106 -170 -377 -170q-371 0 -371 397zM350 1389q0 46 15.5 66t47.5 20q64 0 64 -86t-64 -86q-63 0 -63 86zM731 1389q0 46 15.5 66t47.5 20q64 0 64 -86t-64 -86 q-63 0 -63 86z" />
-<glyph unicode="&#xfd;" horiz-adv-x="940" d="M0 1087h102l230 -610q105 -281 133 -379h6q42 129 137 385l230 604h102l-487 -1263q-59 -154 -99 -208t-93.5 -81t-129.5 -27q-57 0 -127 21v86q58 -16 125 -16q51 0 90 24t70.5 74.5t73 160t53.5 142.5zM361 1241v16q73 79 144.5 171.5t97.5 140.5h141v-17 q-36 -52 -122.5 -138t-190.5 -173h-70z" />
-<glyph unicode="&#xfe;" horiz-adv-x="1219" d="M182 -492v2048h99v-391l-7 -247h7q114 190 368 190q220 0 335.5 -144.5t115.5 -420.5q0 -268 -121.5 -415.5t-331.5 -147.5q-251 0 -366 188h-7l3 -84q4 -74 4 -162v-414h-99zM281 541q0 -255 85.5 -364t278.5 -109q167 0 258.5 124t91.5 347q0 479 -348 479 q-193 0 -279.5 -105t-86.5 -354v-18z" />
-<glyph unicode="&#xff;" horiz-adv-x="940" d="M0 1087h102l230 -610q105 -281 133 -379h6q42 129 137 385l230 604h102l-487 -1263q-59 -154 -99 -208t-93.5 -81t-129.5 -27q-57 0 -127 21v86q58 -16 125 -16q51 0 90 24t70.5 74.5t73 160t53.5 142.5zM214 1389q0 46 15.5 66t47.5 20q64 0 64 -86t-64 -86 q-63 0 -63 86zM595 1389q0 46 15.5 66t47.5 20q64 0 64 -86t-64 -86q-63 0 -63 86z" />
-<glyph unicode="&#x152;" horiz-adv-x="1839" d="M129 735q0 347 174.5 545.5t480.5 198.5q78 0 183 -17h747v-94h-655v-553h616v-94h-616v-627h655v-94h-756q-76 -16 -176 -16q-305 0 -479 200t-174 551zM240 733q0 -315 140.5 -484t401.5 -169q109 0 174 18v1266q-62 16 -172 16q-262 0 -403 -167.5t-141 -479.5z" />
-<glyph unicode="&#x153;" horiz-adv-x="1942" d="M119 545q0 266 129 414.5t354 148.5q151 0 251 -70t157 -209q110 279 399 279q192 0 303 -134t111 -364v-80h-762q2 -230 100.5 -345t276.5 -115q93 0 163.5 13t178.5 56v-90q-92 -40 -170 -54.5t-172 -14.5q-156 0 -266.5 67.5t-165.5 198.5q-59 -128 -158 -197 t-252 -69q-143 0 -252 69t-167 198t-58 298zM223 545q0 -224 98.5 -349.5t278.5 -125.5q174 0 265 122.5t91 352.5q0 224 -93 348.5t-265 124.5q-180 0 -277.5 -123.5t-97.5 -349.5zM1065 618h653q0 189 -82 295.5t-227 106.5q-155 0 -242 -104t-102 -298z" />
-<glyph unicode="&#x178;" horiz-adv-x="1081" d="M0 1462h117l426 -800l428 800h110l-487 -897v-565h-105v557zM288 1727q0 46 15.5 66t47.5 20q64 0 64 -86t-64 -86q-63 0 -63 86zM669 1727q0 46 15.5 66t47.5 20q64 0 64 -86t-64 -86q-63 0 -63 86z" />
-<glyph unicode="&#x2c6;" horiz-adv-x="1182" d="M299 1241v16q62 67 131.5 156t110.5 156h98q68 -120 242 -312v-16h-70q-122 101 -221 207q-108 -114 -221 -207h-70z" />
-<glyph unicode="&#x2dc;" horiz-adv-x="1182" d="M283 1243q10 111 63 174.5t137 63.5q48 0 88 -25t82 -59q34 -28 66 -50t61 -22q46 0 77 36.5t48 119.5h76q-16 -116 -69 -177t-132 -61q-36 0 -75 18.5t-101 71.5q-32 26 -62.5 46t-62.5 20q-45 0 -75 -34.5t-48 -121.5h-73z" />
-<glyph unicode="&#x2000;" horiz-adv-x="953" />
-<glyph unicode="&#x2001;" horiz-adv-x="1907" />
-<glyph unicode="&#x2002;" horiz-adv-x="953" />
-<glyph unicode="&#x2003;" horiz-adv-x="1907" />
-<glyph unicode="&#x2004;" horiz-adv-x="635" />
-<glyph unicode="&#x2005;" horiz-adv-x="476" />
-<glyph unicode="&#x2006;" horiz-adv-x="317" />
-<glyph unicode="&#x2007;" horiz-adv-x="317" />
-<glyph unicode="&#x2008;" horiz-adv-x="238" />
-<glyph unicode="&#x2009;" horiz-adv-x="381" />
-<glyph unicode="&#x200a;" horiz-adv-x="105" />
-<glyph unicode="&#x2010;" horiz-adv-x="659" d="M92 512v82h475v-82h-475z" />
-<glyph unicode="&#x2011;" horiz-adv-x="659" d="M92 512v82h475v-82h-475z" />
-<glyph unicode="&#x2012;" horiz-adv-x="659" d="M92 512v82h475v-82h-475z" />
-<glyph unicode="&#x2013;" horiz-adv-x="1024" d="M82 512v82h860v-82h-860z" />
-<glyph unicode="&#x2014;" horiz-adv-x="2048" d="M82 512v82h1884v-82h-1884z" />
-<glyph unicode="&#x2018;" horiz-adv-x="297" d="M29 981q32 112 81.5 251t92.5 230h65q-30 -101 -64.5 -257t-45.5 -244h-117z" />
-<glyph unicode="&#x2019;" horiz-adv-x="297" d="M29 961q29 96 61 241.5t49 259.5h117l12 -20q-75 -265 -174 -481h-65z" />
-<glyph unicode="&#x201a;" horiz-adv-x="451" d="M68 -263q29 96 61 241.5t49 259.5h117l12 -20q-75 -265 -174 -481h-65z" />
-<glyph unicode="&#x201c;" horiz-adv-x="614" d="M29 981q32 112 81.5 251t92.5 230h65q-30 -101 -64.5 -257t-45.5 -244h-117zM346 981q34 120 83 255t91 226h66q-30 -98 -63 -248.5t-48 -252.5h-117z" />
-<glyph unicode="&#x201d;" horiz-adv-x="614" d="M29 961q29 96 61 241.5t49 259.5h117l12 -20q-75 -265 -174 -481h-65zM346 961q30 98 63 248.5t48 252.5h116l13 -20q-36 -128 -85 -261t-89 -220h-66z" />
-<glyph unicode="&#x201e;" horiz-adv-x="768" d="M68 -263q29 96 61 241.5t49 259.5h117l12 -20q-75 -265 -174 -481h-65zM385 -263q30 98 63 248.5t48 252.5h116l13 -20q-36 -128 -85 -261t-89 -220h-66z" />
-<glyph unicode="&#x2022;" horiz-adv-x="770" d="M231 748q0 89 40.5 134.5t113.5 45.5t113.5 -47t40.5 -133q0 -85 -41 -133t-113 -48t-113 47t-41 134z" />
-<glyph unicode="&#x2026;" horiz-adv-x="1466" d="M162 78q0 98 80 98q82 0 82 -98t-82 -98q-80 0 -80 98zM651 78q0 98 80 98q82 0 82 -98t-82 -98q-80 0 -80 98zM1141 78q0 98 80 98q82 0 82 -98t-82 -98q-80 0 -80 98z" />
-<glyph unicode="&#x202f;" horiz-adv-x="381" />
-<glyph unicode="&#x2039;" horiz-adv-x="524" d="M82 516v27l309 393l62 -43l-254 -363l254 -362l-62 -43z" />
-<glyph unicode="&#x203a;" horiz-adv-x="524" d="M72 168l254 362l-254 363l61 43l309 -391v-27l-309 -393z" />
-<glyph unicode="&#x205f;" horiz-adv-x="476" />
-<glyph unicode="&#x20ac;" d="M74 528v82h172q-4 38 -4 113l4 102h-172v82h184q39 272 183 425t362 153q88 0 161 -17t148 -57l-39 -86q-132 72 -270 72q-174 0 -288 -125.5t-155 -364.5h502v-82h-510l-4 -104v-24q0 -65 4 -87h449v-82h-443q30 -217 147.5 -338.5t301.5 -121.5q148 0 287 65v-94 q-81 -34 -150.5 -46.5t-140.5 -12.5q-228 0 -367.5 140t-181.5 408h-180z" />
-<glyph unicode="&#x2122;" horiz-adv-x="1485" d="M10 1384v78h522v-78h-219v-643h-86v643h-217zM608 741v721h125l221 -606l224 606h125v-721h-86v398l4 207h-7l-227 -605h-74l-221 609h-6l4 -201v-408h-82z" />
-<glyph unicode="&#x25fc;" horiz-adv-x="1085" d="M0 0v1085h1085v-1085h-1085z" />
-<hkern u1="&#x22;" u2="&#x178;" k="-20" />
-<hkern u1="&#x22;" u2="&#x153;" k="123" />
-<hkern u1="&#x22;" u2="&#xfc;" k="61" />
-<hkern u1="&#x22;" u2="&#xfb;" k="61" />
-<hkern u1="&#x22;" u2="&#xfa;" k="61" />
-<hkern u1="&#x22;" u2="&#xf9;" k="61" />
-<hkern u1="&#x22;" u2="&#xf8;" k="123" />
-<hkern u1="&#x22;" u2="&#xf6;" k="123" />
-<hkern u1="&#x22;" u2="&#xf5;" k="123" />
-<hkern u1="&#x22;" u2="&#xf4;" k="123" />
-<hkern u1="&#x22;" u2="&#xf3;" k="123" />
-<hkern u1="&#x22;" u2="&#xf2;" k="123" />
-<hkern u1="&#x22;" u2="&#xeb;" k="123" />
-<hkern u1="&#x22;" u2="&#xea;" k="123" />
-<hkern u1="&#x22;" u2="&#xe9;" k="123" />
-<hkern u1="&#x22;" u2="&#xe8;" k="123" />
-<hkern u1="&#x22;" u2="&#xe7;" k="123" />
-<hkern u1="&#x22;" u2="&#xe6;" k="82" />
-<hkern u1="&#x22;" u2="&#xe5;" k="82" />
-<hkern u1="&#x22;" u2="&#xe4;" k="82" />
-<hkern u1="&#x22;" u2="&#xe3;" k="82" />
-<hkern u1="&#x22;" u2="&#xe2;" k="82" />
-<hkern u1="&#x22;" u2="&#xe1;" k="82" />
-<hkern u1="&#x22;" u2="&#xe0;" k="123" />
-<hkern u1="&#x22;" u2="&#xdd;" k="-20" />
-<hkern u1="&#x22;" u2="&#xc5;" k="143" />
-<hkern u1="&#x22;" u2="&#xc4;" k="143" />
-<hkern u1="&#x22;" u2="&#xc3;" k="143" />
-<hkern u1="&#x22;" u2="&#xc2;" k="143" />
-<hkern u1="&#x22;" u2="&#xc1;" k="143" />
-<hkern u1="&#x22;" u2="&#xc0;" k="143" />
-<hkern u1="&#x22;" u2="u" k="61" />
-<hkern u1="&#x22;" u2="s" k="61" />
-<hkern u1="&#x22;" u2="r" k="61" />
-<hkern u1="&#x22;" u2="q" k="123" />
-<hkern u1="&#x22;" u2="p" k="61" />
-<hkern u1="&#x22;" u2="o" k="123" />
-<hkern u1="&#x22;" u2="n" k="61" />
-<hkern u1="&#x22;" u2="m" k="61" />
-<hkern u1="&#x22;" u2="g" k="61" />
-<hkern u1="&#x22;" u2="e" k="123" />
-<hkern u1="&#x22;" u2="d" k="123" />
-<hkern u1="&#x22;" u2="c" k="123" />
-<hkern u1="&#x22;" u2="a" k="82" />
-<hkern u1="&#x22;" u2="Y" k="-20" />
-<hkern u1="&#x22;" u2="W" k="-41" />
-<hkern u1="&#x22;" u2="V" k="-41" />
-<hkern u1="&#x22;" u2="T" k="-41" />
-<hkern u1="&#x22;" u2="A" k="143" />
-<hkern u1="&#x27;" u2="&#x178;" k="-20" />
-<hkern u1="&#x27;" u2="&#x153;" k="123" />
-<hkern u1="&#x27;" u2="&#xfc;" k="61" />
-<hkern u1="&#x27;" u2="&#xfb;" k="61" />
-<hkern u1="&#x27;" u2="&#xfa;" k="61" />
-<hkern u1="&#x27;" u2="&#xf9;" k="61" />
-<hkern u1="&#x27;" u2="&#xf8;" k="123" />
-<hkern u1="&#x27;" u2="&#xf6;" k="123" />
-<hkern u1="&#x27;" u2="&#xf5;" k="123" />
-<hkern u1="&#x27;" u2="&#xf4;" k="123" />
-<hkern u1="&#x27;" u2="&#xf3;" k="123" />
-<hkern u1="&#x27;" u2="&#xf2;" k="123" />
-<hkern u1="&#x27;" u2="&#xeb;" k="123" />
-<hkern u1="&#x27;" u2="&#xea;" k="123" />
-<hkern u1="&#x27;" u2="&#xe9;" k="123" />
-<hkern u1="&#x27;" u2="&#xe8;" k="123" />
-<hkern u1="&#x27;" u2="&#xe7;" k="123" />
-<hkern u1="&#x27;" u2="&#xe6;" k="82" />
-<hkern u1="&#x27;" u2="&#xe5;" k="82" />
-<hkern u1="&#x27;" u2="&#xe4;" k="82" />
-<hkern u1="&#x27;" u2="&#xe3;" k="82" />
-<hkern u1="&#x27;" u2="&#xe2;" k="82" />
-<hkern u1="&#x27;" u2="&#xe1;" k="82" />
-<hkern u1="&#x27;" u2="&#xe0;" k="123" />
-<hkern u1="&#x27;" u2="&#xdd;" k="-20" />
-<hkern u1="&#x27;" u2="&#xc5;" k="143" />
-<hkern u1="&#x27;" u2="&#xc4;" k="143" />
-<hkern u1="&#x27;" u2="&#xc3;" k="143" />
-<hkern u1="&#x27;" u2="&#xc2;" k="143" />
-<hkern u1="&#x27;" u2="&#xc1;" k="143" />
-<hkern u1="&#x27;" u2="&#xc0;" k="143" />
-<hkern u1="&#x27;" u2="u" k="61" />
-<hkern u1="&#x27;" u2="s" k="61" />
-<hkern u1="&#x27;" u2="r" k="61" />
-<hkern u1="&#x27;" u2="q" k="123" />
-<hkern u1="&#x27;" u2="p" k="61" />
-<hkern u1="&#x27;" u2="o" k="123" />
-<hkern u1="&#x27;" u2="n" k="61" />
-<hkern u1="&#x27;" u2="m" k="61" />
-<hkern u1="&#x27;" u2="g" k="61" />
-<hkern u1="&#x27;" u2="e" k="123" />
-<hkern u1="&#x27;" u2="d" k="123" />
-<hkern u1="&#x27;" u2="c" k="123" />
-<hkern u1="&#x27;" u2="a" k="82" />
-<hkern u1="&#x27;" u2="Y" k="-20" />
-<hkern u1="&#x27;" u2="W" k="-41" />
-<hkern u1="&#x27;" u2="V" k="-41" />
-<hkern u1="&#x27;" u2="T" k="-41" />
-<hkern u1="&#x27;" u2="A" k="143" />
-<hkern u1="&#x28;" u2="J" k="-184" />
-<hkern u1="&#x2c;" u2="&#x178;" k="123" />
-<hkern u1="&#x2c;" u2="&#x152;" k="102" />
-<hkern u1="&#x2c;" u2="&#xdd;" k="123" />
-<hkern u1="&#x2c;" u2="&#xdc;" k="41" />
-<hkern u1="&#x2c;" u2="&#xdb;" k="41" />
-<hkern u1="&#x2c;" u2="&#xda;" k="41" />
-<hkern u1="&#x2c;" u2="&#xd9;" k="41" />
-<hkern u1="&#x2c;" u2="&#xd8;" k="102" />
-<hkern u1="&#x2c;" u2="&#xd6;" k="102" />
-<hkern u1="&#x2c;" u2="&#xd5;" k="102" />
-<hkern u1="&#x2c;" u2="&#xd4;" k="102" />
-<hkern u1="&#x2c;" u2="&#xd3;" k="102" />
-<hkern u1="&#x2c;" u2="&#xd2;" k="102" />
-<hkern u1="&#x2c;" u2="&#xc7;" k="102" />
-<hkern u1="&#x2c;" u2="Y" k="123" />
-<hkern u1="&#x2c;" u2="W" k="123" />
-<hkern u1="&#x2c;" u2="V" k="123" />
-<hkern u1="&#x2c;" u2="U" k="41" />
-<hkern u1="&#x2c;" u2="T" k="143" />
-<hkern u1="&#x2c;" u2="Q" k="102" />
-<hkern u1="&#x2c;" u2="O" k="102" />
-<hkern u1="&#x2c;" u2="G" k="102" />
-<hkern u1="&#x2c;" u2="C" k="102" />
-<hkern u1="&#x2d;" u2="T" k="82" />
-<hkern u1="&#x2e;" u2="&#x178;" k="123" />
-<hkern u1="&#x2e;" u2="&#x152;" k="102" />
-<hkern u1="&#x2e;" u2="&#xdd;" k="123" />
-<hkern u1="&#x2e;" u2="&#xdc;" k="41" />
-<hkern u1="&#x2e;" u2="&#xdb;" k="41" />
-<hkern u1="&#x2e;" u2="&#xda;" k="41" />
-<hkern u1="&#x2e;" u2="&#xd9;" k="41" />
-<hkern u1="&#x2e;" u2="&#xd8;" k="102" />
-<hkern u1="&#x2e;" u2="&#xd6;" k="102" />
-<hkern u1="&#x2e;" u2="&#xd5;" k="102" />
-<hkern u1="&#x2e;" u2="&#xd4;" k="102" />
-<hkern u1="&#x2e;" u2="&#xd3;" k="102" />
-<hkern u1="&#x2e;" u2="&#xd2;" k="102" />
-<hkern u1="&#x2e;" u2="&#xc7;" k="102" />
-<hkern u1="&#x2e;" u2="Y" k="123" />
-<hkern u1="&#x2e;" u2="W" k="123" />
-<hkern u1="&#x2e;" u2="V" k="123" />
-<hkern u1="&#x2e;" u2="U" k="41" />
-<hkern u1="&#x2e;" u2="T" k="143" />
-<hkern u1="&#x2e;" u2="Q" k="102" />
-<hkern u1="&#x2e;" u2="O" k="102" />
-<hkern u1="&#x2e;" u2="G" k="102" />
-<hkern u1="&#x2e;" u2="C" k="102" />
-<hkern u1="A" u2="&#x201d;" k="143" />
-<hkern u1="A" u2="&#x2019;" k="143" />
-<hkern u1="A" u2="&#x178;" k="123" />
-<hkern u1="A" u2="&#x152;" k="41" />
-<hkern u1="A" u2="&#xdd;" k="123" />
-<hkern u1="A" u2="&#xd8;" k="41" />
-<hkern u1="A" u2="&#xd6;" k="41" />
-<hkern u1="A" u2="&#xd5;" k="41" />
-<hkern u1="A" u2="&#xd4;" k="41" />
-<hkern u1="A" u2="&#xd3;" k="41" />
-<hkern u1="A" u2="&#xd2;" k="41" />
-<hkern u1="A" u2="&#xc7;" k="41" />
-<hkern u1="A" u2="Y" k="123" />
-<hkern u1="A" u2="W" k="82" />
-<hkern u1="A" u2="V" k="82" />
-<hkern u1="A" u2="T" k="143" />
-<hkern u1="A" u2="Q" k="41" />
-<hkern u1="A" u2="O" k="41" />
-<hkern u1="A" u2="J" k="-266" />
-<hkern u1="A" u2="G" k="41" />
-<hkern u1="A" u2="C" k="41" />
-<hkern u1="A" u2="&#x27;" k="143" />
-<hkern u1="A" u2="&#x22;" k="143" />
-<hkern u1="B" u2="&#x201e;" k="82" />
-<hkern u1="B" u2="&#x201a;" k="82" />
-<hkern u1="B" u2="&#x178;" k="20" />
-<hkern u1="B" u2="&#xdd;" k="20" />
-<hkern u1="B" u2="&#xc5;" k="41" />
-<hkern u1="B" u2="&#xc4;" k="41" />
-<hkern u1="B" u2="&#xc3;" k="41" />
-<hkern u1="B" u2="&#xc2;" k="41" />
-<hkern u1="B" u2="&#xc1;" k="41" />
-<hkern u1="B" u2="&#xc0;" k="41" />
-<hkern u1="B" u2="Z" k="20" />
-<hkern u1="B" u2="Y" k="20" />
-<hkern u1="B" u2="X" k="41" />
-<hkern u1="B" u2="W" k="20" />
-<hkern u1="B" u2="V" k="20" />
-<hkern u1="B" u2="T" k="61" />
-<hkern u1="B" u2="A" k="41" />
-<hkern u1="B" u2="&#x2e;" k="82" />
-<hkern u1="B" u2="&#x2c;" k="82" />
-<hkern u1="C" u2="&#x152;" k="41" />
-<hkern u1="C" u2="&#xd8;" k="41" />
-<hkern u1="C" u2="&#xd6;" k="41" />
-<hkern u1="C" u2="&#xd5;" k="41" />
-<hkern u1="C" u2="&#xd4;" k="41" />
-<hkern u1="C" u2="&#xd3;" k="41" />
-<hkern u1="C" u2="&#xd2;" k="41" />
-<hkern u1="C" u2="&#xc7;" k="41" />
-<hkern u1="C" u2="Q" k="41" />
-<hkern u1="C" u2="O" k="41" />
-<hkern u1="C" u2="G" k="41" />
-<hkern u1="C" u2="C" k="41" />
-<hkern u1="D" u2="&#x201e;" k="82" />
-<hkern u1="D" u2="&#x201a;" k="82" />
-<hkern u1="D" u2="&#x178;" k="20" />
-<hkern u1="D" u2="&#xdd;" k="20" />
-<hkern u1="D" u2="&#xc5;" k="41" />
-<hkern u1="D" u2="&#xc4;" k="41" />
-<hkern u1="D" u2="&#xc3;" k="41" />
-<hkern u1="D" u2="&#xc2;" k="41" />
-<hkern u1="D" u2="&#xc1;" k="41" />
-<hkern u1="D" u2="&#xc0;" k="41" />
-<hkern u1="D" u2="Z" k="20" />
-<hkern u1="D" u2="Y" k="20" />
-<hkern u1="D" u2="X" k="41" />
-<hkern u1="D" u2="W" k="20" />
-<hkern u1="D" u2="V" k="20" />
-<hkern u1="D" u2="T" k="61" />
-<hkern u1="D" u2="A" k="41" />
-<hkern u1="D" u2="&#x2e;" k="82" />
-<hkern u1="D" u2="&#x2c;" k="82" />
-<hkern u1="E" u2="J" k="-123" />
-<hkern u1="F" u2="&#x201e;" k="123" />
-<hkern u1="F" u2="&#x201a;" k="123" />
-<hkern u1="F" u2="&#xc5;" k="41" />
-<hkern u1="F" u2="&#xc4;" k="41" />
-<hkern u1="F" u2="&#xc3;" k="41" />
-<hkern u1="F" u2="&#xc2;" k="41" />
-<hkern u1="F" u2="&#xc1;" k="41" />
-<hkern u1="F" u2="&#xc0;" k="41" />
-<hkern u1="F" u2="A" k="41" />
-<hkern u1="F" u2="&#x3f;" k="-41" />
-<hkern u1="F" u2="&#x2e;" k="123" />
-<hkern u1="F" u2="&#x2c;" k="123" />
-<hkern u1="K" u2="&#x152;" k="41" />
-<hkern u1="K" u2="&#xd8;" k="41" />
-<hkern u1="K" u2="&#xd6;" k="41" />
-<hkern u1="K" u2="&#xd5;" k="41" />
-<hkern u1="K" u2="&#xd4;" k="41" />
-<hkern u1="K" u2="&#xd3;" k="41" />
-<hkern u1="K" u2="&#xd2;" k="41" />
-<hkern u1="K" u2="&#xc7;" k="41" />
-<hkern u1="K" u2="Q" k="41" />
-<hkern u1="K" u2="O" k="41" />
-<hkern u1="K" u2="G" k="41" />
-<hkern u1="K" u2="C" k="41" />
-<hkern u1="L" u2="&#x201d;" k="164" />
-<hkern u1="L" u2="&#x2019;" k="164" />
-<hkern u1="L" u2="&#x178;" k="61" />
-<hkern u1="L" u2="&#x152;" k="41" />
-<hkern u1="L" u2="&#xdd;" k="61" />
-<hkern u1="L" u2="&#xdc;" k="20" />
-<hkern u1="L" u2="&#xdb;" k="20" />
-<hkern u1="L" u2="&#xda;" k="20" />
-<hkern u1="L" u2="&#xd9;" k="20" />
-<hkern u1="L" u2="&#xd8;" k="41" />
-<hkern u1="L" u2="&#xd6;" k="41" />
-<hkern u1="L" u2="&#xd5;" k="41" />
-<hkern u1="L" u2="&#xd4;" k="41" />
-<hkern u1="L" u2="&#xd3;" k="41" />
-<hkern u1="L" u2="&#xd2;" k="41" />
-<hkern u1="L" u2="&#xc7;" k="41" />
-<hkern u1="L" u2="Y" k="61" />
-<hkern u1="L" u2="W" k="41" />
-<hkern u1="L" u2="V" k="41" />
-<hkern u1="L" u2="U" k="20" />
-<hkern u1="L" u2="T" k="41" />
-<hkern u1="L" u2="Q" k="41" />
-<hkern u1="L" u2="O" k="41" />
-<hkern u1="L" u2="G" k="41" />
-<hkern u1="L" u2="C" k="41" />
-<hkern u1="L" u2="&#x27;" k="164" />
-<hkern u1="L" u2="&#x22;" k="164" />
-<hkern u1="O" u2="&#x201e;" k="82" />
-<hkern u1="O" u2="&#x201a;" k="82" />
-<hkern u1="O" u2="&#x178;" k="20" />
-<hkern u1="O" u2="&#xdd;" k="20" />
-<hkern u1="O" u2="&#xc5;" k="41" />
-<hkern u1="O" u2="&#xc4;" k="41" />
-<hkern u1="O" u2="&#xc3;" k="41" />
-<hkern u1="O" u2="&#xc2;" k="41" />
-<hkern u1="O" u2="&#xc1;" k="41" />
-<hkern u1="O" u2="&#xc0;" k="41" />
-<hkern u1="O" u2="Z" k="20" />
-<hkern u1="O" u2="Y" k="20" />
-<hkern u1="O" u2="X" k="41" />
-<hkern u1="O" u2="W" k="20" />
-<hkern u1="O" u2="V" k="20" />
-<hkern u1="O" u2="T" k="61" />
-<hkern u1="O" u2="A" k="41" />
-<hkern u1="O" u2="&#x2e;" k="82" />
-<hkern u1="O" u2="&#x2c;" k="82" />
-<hkern u1="P" u2="&#x201e;" k="266" />
-<hkern u1="P" u2="&#x201a;" k="266" />
-<hkern u1="P" u2="&#xc5;" k="102" />
-<hkern u1="P" u2="&#xc4;" k="102" />
-<hkern u1="P" u2="&#xc3;" k="102" />
-<hkern u1="P" u2="&#xc2;" k="102" />
-<hkern u1="P" u2="&#xc1;" k="102" />
-<hkern u1="P" u2="&#xc0;" k="102" />
-<hkern u1="P" u2="Z" k="20" />
-<hkern u1="P" u2="X" k="41" />
-<hkern u1="P" u2="A" k="102" />
-<hkern u1="P" u2="&#x2e;" k="266" />
-<hkern u1="P" u2="&#x2c;" k="266" />
-<hkern u1="Q" u2="&#x201e;" k="82" />
-<hkern u1="Q" u2="&#x201a;" k="82" />
-<hkern u1="Q" u2="&#x178;" k="20" />
-<hkern u1="Q" u2="&#xdd;" k="20" />
-<hkern u1="Q" u2="&#xc5;" k="41" />
-<hkern u1="Q" u2="&#xc4;" k="41" />
-<hkern u1="Q" u2="&#xc3;" k="41" />
-<hkern u1="Q" u2="&#xc2;" k="41" />
-<hkern u1="Q" u2="&#xc1;" k="41" />
-<hkern u1="Q" u2="&#xc0;" k="41" />
-<hkern u1="Q" u2="Z" k="20" />
-<hkern u1="Q" u2="Y" k="20" />
-<hkern u1="Q" u2="X" k="41" />
-<hkern u1="Q" u2="W" k="20" />
-<hkern u1="Q" u2="V" k="20" />
-<hkern u1="Q" u2="T" k="61" />
-<hkern u1="Q" u2="A" k="41" />
-<hkern u1="Q" u2="&#x2e;" k="82" />
-<hkern u1="Q" u2="&#x2c;" k="82" />
-<hkern u1="T" u2="&#x201e;" k="123" />
-<hkern u1="T" u2="&#x201a;" k="123" />
-<hkern u1="T" u2="&#x2014;" k="82" />
-<hkern u1="T" u2="&#x2013;" k="82" />
-<hkern u1="T" u2="&#x153;" k="143" />
-<hkern u1="T" u2="&#x152;" k="41" />
-<hkern u1="T" u2="&#xfd;" k="41" />
-<hkern u1="T" u2="&#xfc;" k="102" />
-<hkern u1="T" u2="&#xfb;" k="102" />
-<hkern u1="T" u2="&#xfa;" k="102" />
-<hkern u1="T" u2="&#xf9;" k="102" />
-<hkern u1="T" u2="&#xf8;" k="143" />
-<hkern u1="T" u2="&#xf6;" k="143" />
-<hkern u1="T" u2="&#xf5;" k="143" />
-<hkern u1="T" u2="&#xf4;" k="143" />
-<hkern u1="T" u2="&#xf3;" k="143" />
-<hkern u1="T" u2="&#xf2;" k="143" />
-<hkern u1="T" u2="&#xeb;" k="143" />
-<hkern u1="T" u2="&#xea;" k="143" />
-<hkern u1="T" u2="&#xe9;" k="143" />
-<hkern u1="T" u2="&#xe8;" k="143" />
-<hkern u1="T" u2="&#xe7;" k="143" />
-<hkern u1="T" u2="&#xe6;" k="164" />
-<hkern u1="T" u2="&#xe5;" k="164" />
-<hkern u1="T" u2="&#xe4;" k="164" />
-<hkern u1="T" u2="&#xe3;" k="164" />
-<hkern u1="T" u2="&#xe2;" k="164" />
-<hkern u1="T" u2="&#xe1;" k="164" />
-<hkern u1="T" u2="&#xe0;" k="143" />
-<hkern u1="T" u2="&#xd8;" k="41" />
-<hkern u1="T" u2="&#xd6;" k="41" />
-<hkern u1="T" u2="&#xd5;" k="41" />
-<hkern u1="T" u2="&#xd4;" k="41" />
-<hkern u1="T" u2="&#xd3;" k="41" />
-<hkern u1="T" u2="&#xd2;" k="41" />
-<hkern u1="T" u2="&#xc7;" k="41" />
-<hkern u1="T" u2="&#xc5;" k="143" />
-<hkern u1="T" u2="&#xc4;" k="143" />
-<hkern u1="T" u2="&#xc3;" k="143" />
-<hkern u1="T" u2="&#xc2;" k="143" />
-<hkern u1="T" u2="&#xc1;" k="143" />
-<hkern u1="T" u2="&#xc0;" k="143" />
-<hkern u1="T" u2="z" k="82" />
-<hkern u1="T" u2="y" k="41" />
-<hkern u1="T" u2="x" k="41" />
-<hkern u1="T" u2="w" k="41" />
-<hkern u1="T" u2="v" k="41" />
-<hkern u1="T" u2="u" k="102" />
-<hkern u1="T" u2="s" k="123" />
-<hkern u1="T" u2="r" k="102" />
-<hkern u1="T" u2="q" k="143" />
-<hkern u1="T" u2="p" k="102" />
-<hkern u1="T" u2="o" k="143" />
-<hkern u1="T" u2="n" k="102" />
-<hkern u1="T" u2="m" k="102" />
-<hkern u1="T" u2="g" k="143" />
-<hkern u1="T" u2="e" k="143" />
-<hkern u1="T" u2="d" k="143" />
-<hkern u1="T" u2="c" k="143" />
-<hkern u1="T" u2="a" k="164" />
-<hkern u1="T" u2="T" k="-41" />
-<hkern u1="T" u2="Q" k="41" />
-<hkern u1="T" u2="O" k="41" />
-<hkern u1="T" u2="G" k="41" />
-<hkern u1="T" u2="C" k="41" />
-<hkern u1="T" u2="A" k="143" />
-<hkern u1="T" u2="&#x3f;" k="-41" />
-<hkern u1="T" u2="&#x2e;" k="123" />
-<hkern u1="T" u2="&#x2d;" k="82" />
-<hkern u1="T" u2="&#x2c;" k="123" />
-<hkern u1="U" u2="&#x201e;" k="41" />
-<hkern u1="U" u2="&#x201a;" k="41" />
-<hkern u1="U" u2="&#xc5;" k="20" />
-<hkern u1="U" u2="&#xc4;" k="20" />
-<hkern u1="U" u2="&#xc3;" k="20" />
-<hkern u1="U" u2="&#xc2;" k="20" />
-<hkern u1="U" u2="&#xc1;" k="20" />
-<hkern u1="U" u2="&#xc0;" k="20" />
-<hkern u1="U" u2="A" k="20" />
-<hkern u1="U" u2="&#x2e;" k="41" />
-<hkern u1="U" u2="&#x2c;" k="41" />
-<hkern u1="V" u2="&#x201e;" k="102" />
-<hkern u1="V" u2="&#x201a;" k="102" />
-<hkern u1="V" u2="&#x153;" k="41" />
-<hkern u1="V" u2="&#x152;" k="20" />
-<hkern u1="V" u2="&#xfc;" k="20" />
-<hkern u1="V" u2="&#xfb;" k="20" />
-<hkern u1="V" u2="&#xfa;" k="20" />
-<hkern u1="V" u2="&#xf9;" k="20" />
-<hkern u1="V" u2="&#xf8;" k="41" />
-<hkern u1="V" u2="&#xf6;" k="41" />
-<hkern u1="V" u2="&#xf5;" k="41" />
-<hkern u1="V" u2="&#xf4;" k="41" />
-<hkern u1="V" u2="&#xf3;" k="41" />
-<hkern u1="V" u2="&#xf2;" k="41" />
-<hkern u1="V" u2="&#xeb;" k="41" />
-<hkern u1="V" u2="&#xea;" k="41" />
-<hkern u1="V" u2="&#xe9;" k="41" />
-<hkern u1="V" u2="&#xe8;" k="41" />
-<hkern u1="V" u2="&#xe7;" k="41" />
-<hkern u1="V" u2="&#xe6;" k="41" />
-<hkern u1="V" u2="&#xe5;" k="41" />
-<hkern u1="V" u2="&#xe4;" k="41" />
-<hkern u1="V" u2="&#xe3;" k="41" />
-<hkern u1="V" u2="&#xe2;" k="41" />
-<hkern u1="V" u2="&#xe1;" k="41" />
-<hkern u1="V" u2="&#xe0;" k="41" />
-<hkern u1="V" u2="&#xd8;" k="20" />
-<hkern u1="V" u2="&#xd6;" k="20" />
-<hkern u1="V" u2="&#xd5;" k="20" />
-<hkern u1="V" u2="&#xd4;" k="20" />
-<hkern u1="V" u2="&#xd3;" k="20" />
-<hkern u1="V" u2="&#xd2;" k="20" />
-<hkern u1="V" u2="&#xc7;" k="20" />
-<hkern u1="V" u2="&#xc5;" k="82" />
-<hkern u1="V" u2="&#xc4;" k="82" />
-<hkern u1="V" u2="&#xc3;" k="82" />
-<hkern u1="V" u2="&#xc2;" k="82" />
-<hkern u1="V" u2="&#xc1;" k="82" />
-<hkern u1="V" u2="&#xc0;" k="82" />
-<hkern u1="V" u2="u" k="20" />
-<hkern u1="V" u2="s" k="20" />
-<hkern u1="V" u2="r" k="20" />
-<hkern u1="V" u2="q" k="41" />
-<hkern u1="V" u2="p" k="20" />
-<hkern u1="V" u2="o" k="41" />
-<hkern u1="V" u2="n" k="20" />
-<hkern u1="V" u2="m" k="20" />
-<hkern u1="V" u2="g" k="20" />
-<hkern u1="V" u2="e" k="41" />
-<hkern u1="V" u2="d" k="41" />
-<hkern u1="V" u2="c" k="41" />
-<hkern u1="V" u2="a" k="41" />
-<hkern u1="V" u2="Q" k="20" />
-<hkern u1="V" u2="O" k="20" />
-<hkern u1="V" u2="G" k="20" />
-<hkern u1="V" u2="C" k="20" />
-<hkern u1="V" u2="A" k="82" />
-<hkern u1="V" u2="&#x3f;" k="-41" />
-<hkern u1="V" u2="&#x2e;" k="102" />
-<hkern u1="V" u2="&#x2c;" k="102" />
-<hkern u1="W" u2="&#x201e;" k="102" />
-<hkern u1="W" u2="&#x201a;" k="102" />
-<hkern u1="W" u2="&#x153;" k="41" />
-<hkern u1="W" u2="&#x152;" k="20" />
-<hkern u1="W" u2="&#xfc;" k="20" />
-<hkern u1="W" u2="&#xfb;" k="20" />
-<hkern u1="W" u2="&#xfa;" k="20" />
-<hkern u1="W" u2="&#xf9;" k="20" />
-<hkern u1="W" u2="&#xf8;" k="41" />
-<hkern u1="W" u2="&#xf6;" k="41" />
-<hkern u1="W" u2="&#xf5;" k="41" />
-<hkern u1="W" u2="&#xf4;" k="41" />
-<hkern u1="W" u2="&#xf3;" k="41" />
-<hkern u1="W" u2="&#xf2;" k="41" />
-<hkern u1="W" u2="&#xeb;" k="41" />
-<hkern u1="W" u2="&#xea;" k="41" />
-<hkern u1="W" u2="&#xe9;" k="41" />
-<hkern u1="W" u2="&#xe8;" k="41" />
-<hkern u1="W" u2="&#xe7;" k="41" />
-<hkern u1="W" u2="&#xe6;" k="41" />
-<hkern u1="W" u2="&#xe5;" k="41" />
-<hkern u1="W" u2="&#xe4;" k="41" />
-<hkern u1="W" u2="&#xe3;" k="41" />
-<hkern u1="W" u2="&#xe2;" k="41" />
-<hkern u1="W" u2="&#xe1;" k="41" />
-<hkern u1="W" u2="&#xe0;" k="41" />
-<hkern u1="W" u2="&#xd8;" k="20" />
-<hkern u1="W" u2="&#xd6;" k="20" />
-<hkern u1="W" u2="&#xd5;" k="20" />
-<hkern u1="W" u2="&#xd4;" k="20" />
-<hkern u1="W" u2="&#xd3;" k="20" />
-<hkern u1="W" u2="&#xd2;" k="20" />
-<hkern u1="W" u2="&#xc7;" k="20" />
-<hkern u1="W" u2="&#xc5;" k="82" />
-<hkern u1="W" u2="&#xc4;" k="82" />
-<hkern u1="W" u2="&#xc3;" k="82" />
-<hkern u1="W" u2="&#xc2;" k="82" />
-<hkern u1="W" u2="&#xc1;" k="82" />
-<hkern u1="W" u2="&#xc0;" k="82" />
-<hkern u1="W" u2="u" k="20" />
-<hkern u1="W" u2="s" k="20" />
-<hkern u1="W" u2="r" k="20" />
-<hkern u1="W" u2="q" k="41" />
-<hkern u1="W" u2="p" k="20" />
-<hkern u1="W" u2="o" k="41" />
-<hkern u1="W" u2="n" k="20" />
-<hkern u1="W" u2="m" k="20" />
-<hkern u1="W" u2="g" k="20" />
-<hkern u1="W" u2="e" k="41" />
-<hkern u1="W" u2="d" k="41" />
-<hkern u1="W" u2="c" k="41" />
-<hkern u1="W" u2="a" k="41" />
-<hkern u1="W" u2="Q" k="20" />
-<hkern u1="W" u2="O" k="20" />
-<hkern u1="W" u2="G" k="20" />
-<hkern u1="W" u2="C" k="20" />
-<hkern u1="W" u2="A" k="82" />
-<hkern u1="W" u2="&#x3f;" k="-41" />
-<hkern u1="W" u2="&#x2e;" k="102" />
-<hkern u1="W" u2="&#x2c;" k="102" />
-<hkern u1="X" u2="&#x152;" k="41" />
-<hkern u1="X" u2="&#xd8;" k="41" />
-<hkern u1="X" u2="&#xd6;" k="41" />
-<hkern u1="X" u2="&#xd5;" k="41" />
-<hkern u1="X" u2="&#xd4;" k="41" />
-<hkern u1="X" u2="&#xd3;" k="41" />
-<hkern u1="X" u2="&#xd2;" k="41" />
-<hkern u1="X" u2="&#xc7;" k="41" />
-<hkern u1="X" u2="Q" k="41" />
-<hkern u1="X" u2="O" k="41" />
-<hkern u1="X" u2="G" k="41" />
-<hkern u1="X" u2="C" k="41" />
-<hkern u1="Y" u2="&#x201e;" k="123" />
-<hkern u1="Y" u2="&#x201a;" k="123" />
-<hkern u1="Y" u2="&#x153;" k="102" />
-<hkern u1="Y" u2="&#x152;" k="41" />
-<hkern u1="Y" u2="&#xfc;" k="61" />
-<hkern u1="Y" u2="&#xfb;" k="61" />
-<hkern u1="Y" u2="&#xfa;" k="61" />
-<hkern u1="Y" u2="&#xf9;" k="61" />
-<hkern u1="Y" u2="&#xf8;" k="102" />
-<hkern u1="Y" u2="&#xf6;" k="102" />
-<hkern u1="Y" u2="&#xf5;" k="102" />
-<hkern u1="Y" u2="&#xf4;" k="102" />
-<hkern u1="Y" u2="&#xf3;" k="102" />
-<hkern u1="Y" u2="&#xf2;" k="102" />
-<hkern u1="Y" u2="&#xeb;" k="102" />
-<hkern u1="Y" u2="&#xea;" k="102" />
-<hkern u1="Y" u2="&#xe9;" k="102" />
-<hkern u1="Y" u2="&#xe8;" k="102" />
-<hkern u1="Y" u2="&#xe7;" k="102" />
-<hkern u1="Y" u2="&#xe6;" k="102" />
-<hkern u1="Y" u2="&#xe5;" k="102" />
-<hkern u1="Y" u2="&#xe4;" k="102" />
-<hkern u1="Y" u2="&#xe3;" k="102" />
-<hkern u1="Y" u2="&#xe2;" k="102" />
-<hkern u1="Y" u2="&#xe1;" k="102" />
-<hkern u1="Y" u2="&#xe0;" k="102" />
-<hkern u1="Y" u2="&#xd8;" k="41" />
-<hkern u1="Y" u2="&#xd6;" k="41" />
-<hkern u1="Y" u2="&#xd5;" k="41" />
-<hkern u1="Y" u2="&#xd4;" k="41" />
-<hkern u1="Y" u2="&#xd3;" k="41" />
-<hkern u1="Y" u2="&#xd2;" k="41" />
-<hkern u1="Y" u2="&#xc7;" k="41" />
-<hkern u1="Y" u2="&#xc5;" k="123" />
-<hkern u1="Y" u2="&#xc4;" k="123" />
-<hkern u1="Y" u2="&#xc3;" k="123" />
-<hkern u1="Y" u2="&#xc2;" k="123" />
-<hkern u1="Y" u2="&#xc1;" k="123" />
-<hkern u1="Y" u2="&#xc0;" k="123" />
-<hkern u1="Y" u2="z" k="41" />
-<hkern u1="Y" u2="u" k="61" />
-<hkern u1="Y" u2="s" k="82" />
-<hkern u1="Y" u2="r" k="61" />
-<hkern u1="Y" u2="q" k="102" />
-<hkern u1="Y" u2="p" k="61" />
-<hkern u1="Y" u2="o" k="102" />
-<hkern u1="Y" u2="n" k="61" />
-<hkern u1="Y" u2="m" k="61" />
-<hkern u1="Y" u2="g" k="41" />
-<hkern u1="Y" u2="e" k="102" />
-<hkern u1="Y" u2="d" k="102" />
-<hkern u1="Y" u2="c" k="102" />
-<hkern u1="Y" u2="a" k="102" />
-<hkern u1="Y" u2="Q" k="41" />
-<hkern u1="Y" u2="O" k="41" />
-<hkern u1="Y" u2="G" k="41" />
-<hkern u1="Y" u2="C" k="41" />
-<hkern u1="Y" u2="A" k="123" />
-<hkern u1="Y" u2="&#x3f;" k="-41" />
-<hkern u1="Y" u2="&#x2e;" k="123" />
-<hkern u1="Y" u2="&#x2c;" k="123" />
-<hkern u1="Z" u2="&#x152;" k="20" />
-<hkern u1="Z" u2="&#xd8;" k="20" />
-<hkern u1="Z" u2="&#xd6;" k="20" />
-<hkern u1="Z" u2="&#xd5;" k="20" />
-<hkern u1="Z" u2="&#xd4;" k="20" />
-<hkern u1="Z" u2="&#xd3;" k="20" />
-<hkern u1="Z" u2="&#xd2;" k="20" />
-<hkern u1="Z" u2="&#xc7;" k="20" />
-<hkern u1="Z" u2="Q" k="20" />
-<hkern u1="Z" u2="O" k="20" />
-<hkern u1="Z" u2="G" k="20" />
-<hkern u1="Z" u2="C" k="20" />
-<hkern u1="[" u2="J" k="-184" />
-<hkern u1="a" u2="&#x201d;" k="20" />
-<hkern u1="a" u2="&#x2019;" k="20" />
-<hkern u1="a" u2="&#x27;" k="20" />
-<hkern u1="a" u2="&#x22;" k="20" />
-<hkern u1="b" u2="&#x201d;" k="20" />
-<hkern u1="b" u2="&#x2019;" k="20" />
-<hkern u1="b" u2="&#xfd;" k="41" />
-<hkern u1="b" u2="z" k="20" />
-<hkern u1="b" u2="y" k="41" />
-<hkern u1="b" u2="x" k="41" />
-<hkern u1="b" u2="w" k="41" />
-<hkern u1="b" u2="v" k="41" />
-<hkern u1="b" u2="&#x27;" k="20" />
-<hkern u1="b" u2="&#x22;" k="20" />
-<hkern u1="c" u2="&#x201d;" k="-41" />
-<hkern u1="c" u2="&#x2019;" k="-41" />
-<hkern u1="c" u2="&#x27;" k="-41" />
-<hkern u1="c" u2="&#x22;" k="-41" />
-<hkern u1="e" u2="&#x201d;" k="20" />
-<hkern u1="e" u2="&#x2019;" k="20" />
-<hkern u1="e" u2="&#xfd;" k="41" />
-<hkern u1="e" u2="z" k="20" />
-<hkern u1="e" u2="y" k="41" />
-<hkern u1="e" u2="x" k="41" />
-<hkern u1="e" u2="w" k="41" />
-<hkern u1="e" u2="v" k="41" />
-<hkern u1="e" u2="&#x27;" k="20" />
-<hkern u1="e" u2="&#x22;" k="20" />
-<hkern u1="f" u2="&#x201d;" k="-123" />
-<hkern u1="f" u2="&#x2019;" k="-123" />
-<hkern u1="f" u2="&#x27;" k="-123" />
-<hkern u1="f" u2="&#x22;" k="-123" />
-<hkern u1="h" u2="&#x201d;" k="20" />
-<hkern u1="h" u2="&#x2019;" k="20" />
-<hkern u1="h" u2="&#x27;" k="20" />
-<hkern u1="h" u2="&#x22;" k="20" />
-<hkern u1="k" u2="&#x153;" k="41" />
-<hkern u1="k" u2="&#xf8;" k="41" />
-<hkern u1="k" u2="&#xf6;" k="41" />
-<hkern u1="k" u2="&#xf5;" k="41" />
-<hkern u1="k" u2="&#xf4;" k="41" />
-<hkern u1="k" u2="&#xf3;" k="41" />
-<hkern u1="k" u2="&#xf2;" k="41" />
-<hkern u1="k" u2="&#xeb;" k="41" />
-<hkern u1="k" u2="&#xea;" k="41" />
-<hkern u1="k" u2="&#xe9;" k="41" />
-<hkern u1="k" u2="&#xe8;" k="41" />
-<hkern u1="k" u2="&#xe7;" k="41" />
-<hkern u1="k" u2="&#xe0;" k="41" />
-<hkern u1="k" u2="q" k="41" />
-<hkern u1="k" u2="o" k="41" />
-<hkern u1="k" u2="e" k="41" />
-<hkern u1="k" u2="d" k="41" />
-<hkern u1="k" u2="c" k="41" />
-<hkern u1="m" u2="&#x201d;" k="20" />
-<hkern u1="m" u2="&#x2019;" k="20" />
-<hkern u1="m" u2="&#x27;" k="20" />
-<hkern u1="m" u2="&#x22;" k="20" />
-<hkern u1="n" u2="&#x201d;" k="20" />
-<hkern u1="n" u2="&#x2019;" k="20" />
-<hkern u1="n" u2="&#x27;" k="20" />
-<hkern u1="n" u2="&#x22;" k="20" />
-<hkern u1="o" u2="&#x201d;" k="20" />
-<hkern u1="o" u2="&#x2019;" k="20" />
-<hkern u1="o" u2="&#xfd;" k="41" />
-<hkern u1="o" u2="z" k="20" />
-<hkern u1="o" u2="y" k="41" />
-<hkern u1="o" u2="x" k="41" />
-<hkern u1="o" u2="w" k="41" />
-<hkern u1="o" u2="v" k="41" />
-<hkern u1="o" u2="&#x27;" k="20" />
-<hkern u1="o" u2="&#x22;" k="20" />
-<hkern u1="p" u2="&#x201d;" k="20" />
-<hkern u1="p" u2="&#x2019;" k="20" />
-<hkern u1="p" u2="&#xfd;" k="41" />
-<hkern u1="p" u2="z" k="20" />
-<hkern u1="p" u2="y" k="41" />
-<hkern u1="p" u2="x" k="41" />
-<hkern u1="p" u2="w" k="41" />
-<hkern u1="p" u2="v" k="41" />
-<hkern u1="p" u2="&#x27;" k="20" />
-<hkern u1="p" u2="&#x22;" k="20" />
-<hkern u1="r" u2="&#x201d;" k="-82" />
-<hkern u1="r" u2="&#x2019;" k="-82" />
-<hkern u1="r" u2="&#x153;" k="41" />
-<hkern u1="r" u2="&#xf8;" k="41" />
-<hkern u1="r" u2="&#xf6;" k="41" />
-<hkern u1="r" u2="&#xf5;" k="41" />
-<hkern u1="r" u2="&#xf4;" k="41" />
-<hkern u1="r" u2="&#xf3;" k="41" />
-<hkern u1="r" u2="&#xf2;" k="41" />
-<hkern u1="r" u2="&#xeb;" k="41" />
-<hkern u1="r" u2="&#xea;" k="41" />
-<hkern u1="r" u2="&#xe9;" k="41" />
-<hkern u1="r" u2="&#xe8;" k="41" />
-<hkern u1="r" u2="&#xe7;" k="41" />
-<hkern u1="r" u2="&#xe6;" k="41" />
-<hkern u1="r" u2="&#xe5;" k="41" />
-<hkern u1="r" u2="&#xe4;" k="41" />
-<hkern u1="r" u2="&#xe3;" k="41" />
-<hkern u1="r" u2="&#xe2;" k="41" />
-<hkern u1="r" u2="&#xe1;" k="41" />
-<hkern u1="r" u2="&#xe0;" k="41" />
-<hkern u1="r" u2="q" k="41" />
-<hkern u1="r" u2="o" k="41" />
-<hkern u1="r" u2="g" k="20" />
-<hkern u1="r" u2="e" k="41" />
-<hkern u1="r" u2="d" k="41" />
-<hkern u1="r" u2="c" k="41" />
-<hkern u1="r" u2="a" k="41" />
-<hkern u1="r" u2="&#x27;" k="-82" />
-<hkern u1="r" u2="&#x22;" k="-82" />
-<hkern u1="t" u2="&#x201d;" k="-41" />
-<hkern u1="t" u2="&#x2019;" k="-41" />
-<hkern u1="t" u2="&#x27;" k="-41" />
-<hkern u1="t" u2="&#x22;" k="-41" />
-<hkern u1="v" u2="&#x201e;" k="82" />
-<hkern u1="v" u2="&#x201d;" k="-82" />
-<hkern u1="v" u2="&#x201a;" k="82" />
-<hkern u1="v" u2="&#x2019;" k="-82" />
-<hkern u1="v" u2="&#x3f;" k="-41" />
-<hkern u1="v" u2="&#x2e;" k="82" />
-<hkern u1="v" u2="&#x2c;" k="82" />
-<hkern u1="v" u2="&#x27;" k="-82" />
-<hkern u1="v" u2="&#x22;" k="-82" />
-<hkern u1="w" u2="&#x201e;" k="82" />
-<hkern u1="w" u2="&#x201d;" k="-82" />
-<hkern u1="w" u2="&#x201a;" k="82" />
-<hkern u1="w" u2="&#x2019;" k="-82" />
-<hkern u1="w" u2="&#x3f;" k="-41" />
-<hkern u1="w" u2="&#x2e;" k="82" />
-<hkern u1="w" u2="&#x2c;" k="82" />
-<hkern u1="w" u2="&#x27;" k="-82" />
-<hkern u1="w" u2="&#x22;" k="-82" />
-<hkern u1="x" u2="&#x153;" k="41" />
-<hkern u1="x" u2="&#xf8;" k="41" />
-<hkern u1="x" u2="&#xf6;" k="41" />
-<hkern u1="x" u2="&#xf5;" k="41" />
-<hkern u1="x" u2="&#xf4;" k="41" />
-<hkern u1="x" u2="&#xf3;" k="41" />
-<hkern u1="x" u2="&#xf2;" k="41" />
-<hkern u1="x" u2="&#xeb;" k="41" />
-<hkern u1="x" u2="&#xea;" k="41" />
-<hkern u1="x" u2="&#xe9;" k="41" />
-<hkern u1="x" u2="&#xe8;" k="41" />
-<hkern u1="x" u2="&#xe7;" k="41" />
-<hkern u1="x" u2="&#xe0;" k="41" />
-<hkern u1="x" u2="q" k="41" />
-<hkern u1="x" u2="o" k="41" />
-<hkern u1="x" u2="e" k="41" />
-<hkern u1="x" u2="d" k="41" />
-<hkern u1="x" u2="c" k="41" />
-<hkern u1="y" u2="&#x201e;" k="82" />
-<hkern u1="y" u2="&#x201d;" k="-82" />
-<hkern u1="y" u2="&#x201a;" k="82" />
-<hkern u1="y" u2="&#x2019;" k="-82" />
-<hkern u1="y" u2="&#x3f;" k="-41" />
-<hkern u1="y" u2="&#x2e;" k="82" />
-<hkern u1="y" u2="&#x2c;" k="82" />
-<hkern u1="y" u2="&#x27;" k="-82" />
-<hkern u1="y" u2="&#x22;" k="-82" />
-<hkern u1="&#x7b;" u2="J" k="-184" />
-<hkern u1="&#xc0;" u2="&#x201d;" k="143" />
-<hkern u1="&#xc0;" u2="&#x2019;" k="143" />
-<hkern u1="&#xc0;" u2="&#x178;" k="123" />
-<hkern u1="&#xc0;" u2="&#x152;" k="41" />
-<hkern u1="&#xc0;" u2="&#xdd;" k="123" />
-<hkern u1="&#xc0;" u2="&#xd8;" k="41" />
-<hkern u1="&#xc0;" u2="&#xd6;" k="41" />
-<hkern u1="&#xc0;" u2="&#xd5;" k="41" />
-<hkern u1="&#xc0;" u2="&#xd4;" k="41" />
-<hkern u1="&#xc0;" u2="&#xd3;" k="41" />
-<hkern u1="&#xc0;" u2="&#xd2;" k="41" />
-<hkern u1="&#xc0;" u2="&#xc7;" k="41" />
-<hkern u1="&#xc0;" u2="Y" k="123" />
-<hkern u1="&#xc0;" u2="W" k="82" />
-<hkern u1="&#xc0;" u2="V" k="82" />
-<hkern u1="&#xc0;" u2="T" k="143" />
-<hkern u1="&#xc0;" u2="Q" k="41" />
-<hkern u1="&#xc0;" u2="O" k="41" />
-<hkern u1="&#xc0;" u2="J" k="-266" />
-<hkern u1="&#xc0;" u2="G" k="41" />
-<hkern u1="&#xc0;" u2="C" k="41" />
-<hkern u1="&#xc0;" u2="&#x27;" k="143" />
-<hkern u1="&#xc0;" u2="&#x22;" k="143" />
-<hkern u1="&#xc1;" u2="&#x201d;" k="143" />
-<hkern u1="&#xc1;" u2="&#x2019;" k="143" />
-<hkern u1="&#xc1;" u2="&#x178;" k="123" />
-<hkern u1="&#xc1;" u2="&#x152;" k="41" />
-<hkern u1="&#xc1;" u2="&#xdd;" k="123" />
-<hkern u1="&#xc1;" u2="&#xd8;" k="41" />
-<hkern u1="&#xc1;" u2="&#xd6;" k="41" />
-<hkern u1="&#xc1;" u2="&#xd5;" k="41" />
-<hkern u1="&#xc1;" u2="&#xd4;" k="41" />
-<hkern u1="&#xc1;" u2="&#xd3;" k="41" />
-<hkern u1="&#xc1;" u2="&#xd2;" k="41" />
-<hkern u1="&#xc1;" u2="&#xc7;" k="41" />
-<hkern u1="&#xc1;" u2="Y" k="123" />
-<hkern u1="&#xc1;" u2="W" k="82" />
-<hkern u1="&#xc1;" u2="V" k="82" />
-<hkern u1="&#xc1;" u2="T" k="143" />
-<hkern u1="&#xc1;" u2="Q" k="41" />
-<hkern u1="&#xc1;" u2="O" k="41" />
-<hkern u1="&#xc1;" u2="J" k="-266" />
-<hkern u1="&#xc1;" u2="G" k="41" />
-<hkern u1="&#xc1;" u2="C" k="41" />
-<hkern u1="&#xc1;" u2="&#x27;" k="143" />
-<hkern u1="&#xc1;" u2="&#x22;" k="143" />
-<hkern u1="&#xc2;" u2="&#x201d;" k="143" />
-<hkern u1="&#xc2;" u2="&#x2019;" k="143" />
-<hkern u1="&#xc2;" u2="&#x178;" k="123" />
-<hkern u1="&#xc2;" u2="&#x152;" k="41" />
-<hkern u1="&#xc2;" u2="&#xdd;" k="123" />
-<hkern u1="&#xc2;" u2="&#xd8;" k="41" />
-<hkern u1="&#xc2;" u2="&#xd6;" k="41" />
-<hkern u1="&#xc2;" u2="&#xd5;" k="41" />
-<hkern u1="&#xc2;" u2="&#xd4;" k="41" />
-<hkern u1="&#xc2;" u2="&#xd3;" k="41" />
-<hkern u1="&#xc2;" u2="&#xd2;" k="41" />
-<hkern u1="&#xc2;" u2="&#xc7;" k="41" />
-<hkern u1="&#xc2;" u2="Y" k="123" />
-<hkern u1="&#xc2;" u2="W" k="82" />
-<hkern u1="&#xc2;" u2="V" k="82" />
-<hkern u1="&#xc2;" u2="T" k="143" />
-<hkern u1="&#xc2;" u2="Q" k="41" />
-<hkern u1="&#xc2;" u2="O" k="41" />
-<hkern u1="&#xc2;" u2="J" k="-266" />
-<hkern u1="&#xc2;" u2="G" k="41" />
-<hkern u1="&#xc2;" u2="C" k="41" />
-<hkern u1="&#xc2;" u2="&#x27;" k="143" />
-<hkern u1="&#xc2;" u2="&#x22;" k="143" />
-<hkern u1="&#xc3;" u2="&#x201d;" k="143" />
-<hkern u1="&#xc3;" u2="&#x2019;" k="143" />
-<hkern u1="&#xc3;" u2="&#x178;" k="123" />
-<hkern u1="&#xc3;" u2="&#x152;" k="41" />
-<hkern u1="&#xc3;" u2="&#xdd;" k="123" />
-<hkern u1="&#xc3;" u2="&#xd8;" k="41" />
-<hkern u1="&#xc3;" u2="&#xd6;" k="41" />
-<hkern u1="&#xc3;" u2="&#xd5;" k="41" />
-<hkern u1="&#xc3;" u2="&#xd4;" k="41" />
-<hkern u1="&#xc3;" u2="&#xd3;" k="41" />
-<hkern u1="&#xc3;" u2="&#xd2;" k="41" />
-<hkern u1="&#xc3;" u2="&#xc7;" k="41" />
-<hkern u1="&#xc3;" u2="Y" k="123" />
-<hkern u1="&#xc3;" u2="W" k="82" />
-<hkern u1="&#xc3;" u2="V" k="82" />
-<hkern u1="&#xc3;" u2="T" k="143" />
-<hkern u1="&#xc3;" u2="Q" k="41" />
-<hkern u1="&#xc3;" u2="O" k="41" />
-<hkern u1="&#xc3;" u2="J" k="-266" />
-<hkern u1="&#xc3;" u2="G" k="41" />
-<hkern u1="&#xc3;" u2="C" k="41" />
-<hkern u1="&#xc3;" u2="&#x27;" k="143" />
-<hkern u1="&#xc3;" u2="&#x22;" k="143" />
-<hkern u1="&#xc4;" u2="&#x201d;" k="143" />
-<hkern u1="&#xc4;" u2="&#x2019;" k="143" />
-<hkern u1="&#xc4;" u2="&#x178;" k="123" />
-<hkern u1="&#xc4;" u2="&#x152;" k="41" />
-<hkern u1="&#xc4;" u2="&#xdd;" k="123" />
-<hkern u1="&#xc4;" u2="&#xd8;" k="41" />
-<hkern u1="&#xc4;" u2="&#xd6;" k="41" />
-<hkern u1="&#xc4;" u2="&#xd5;" k="41" />
-<hkern u1="&#xc4;" u2="&#xd4;" k="41" />
-<hkern u1="&#xc4;" u2="&#xd3;" k="41" />
-<hkern u1="&#xc4;" u2="&#xd2;" k="41" />
-<hkern u1="&#xc4;" u2="&#xc7;" k="41" />
-<hkern u1="&#xc4;" u2="Y" k="123" />
-<hkern u1="&#xc4;" u2="W" k="82" />
-<hkern u1="&#xc4;" u2="V" k="82" />
-<hkern u1="&#xc4;" u2="T" k="143" />
-<hkern u1="&#xc4;" u2="Q" k="41" />
-<hkern u1="&#xc4;" u2="O" k="41" />
-<hkern u1="&#xc4;" u2="J" k="-266" />
-<hkern u1="&#xc4;" u2="G" k="41" />
-<hkern u1="&#xc4;" u2="C" k="41" />
-<hkern u1="&#xc4;" u2="&#x27;" k="143" />
-<hkern u1="&#xc4;" u2="&#x22;" k="143" />
-<hkern u1="&#xc5;" u2="&#x201d;" k="143" />
-<hkern u1="&#xc5;" u2="&#x2019;" k="143" />
-<hkern u1="&#xc5;" u2="&#x178;" k="123" />
-<hkern u1="&#xc5;" u2="&#x152;" k="41" />
-<hkern u1="&#xc5;" u2="&#xdd;" k="123" />
-<hkern u1="&#xc5;" u2="&#xd8;" k="41" /

<TRUNCATED>
http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/assets/fonts/opensans-light-webfont.ttf
----------------------------------------------------------------------
diff --git a/website/assets/fonts/opensans-light-webfont.ttf b/website/assets/fonts/opensans-light-webfont.ttf
deleted file mode 100644
index 85c3fc8..0000000
Binary files a/website/assets/fonts/opensans-light-webfont.ttf and /dev/null differ


[51/52] [abbrv] kylin git commit: KYLIN-920 & KYLIN-782 Upgrade to HBase 1.1 (with help from murkrishn )

Posted by li...@apache.org.
http://git-wip-us.apache.org/repos/asf/kylin/blob/48cec941/job/src/test/java/org/apache/kylin/job/ExportHBaseData.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/org/apache/kylin/job/ExportHBaseData.java b/job/src/test/java/org/apache/kylin/job/ExportHBaseData.java
index e784a41..95a483d 100644
--- a/job/src/test/java/org/apache/kylin/job/ExportHBaseData.java
+++ b/job/src/test/java/org/apache/kylin/job/ExportHBaseData.java
@@ -22,10 +22,11 @@ import java.io.File;
 import java.io.IOException;
 
 import org.apache.commons.io.FileUtils;
+import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
-import org.apache.hadoop.hbase.client.HConnection;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.persistence.HBaseConnection;
 import org.apache.kylin.common.util.AbstractKylinTestCase;
@@ -39,7 +40,7 @@ public class ExportHBaseData {
     KylinConfig kylinConfig;
     HTableDescriptor[] allTables;
     Configuration config;
-    HBaseAdmin hbase;
+    Admin admin;
     CliCommandExecutor cli;
     String exportHdfsFolder;
     String exportLocalFolderParent;
@@ -75,12 +76,11 @@ public class ExportHBaseData {
         int cut = metadataUrl.indexOf('@');
         tableNameBase = metadataUrl.substring(0, cut);
         String hbaseUrl = cut < 0 ? metadataUrl : metadataUrl.substring(cut + 1);
-
-        HConnection conn = HBaseConnection.get(hbaseUrl);
+        Connection conn = HBaseConnection.get(hbaseUrl);
         try {
-            hbase = new HBaseAdmin(conn);
-            config = hbase.getConfiguration();
-            allTables = hbase.listTables();
+            admin = conn.getAdmin();
+            config = admin.getConfiguration();
+            allTables = admin.listTables();
         } catch (IOException e) {
             e.printStackTrace();
             throw e;
@@ -89,6 +89,8 @@ public class ExportHBaseData {
 
     public void tearDown() {
 
+        // close hbase admin
+        IOUtils.closeQuietly(admin);
         // cleanup hdfs
         try {
             if (cli != null && exportHdfsFolder != null) {

http://git-wip-us.apache.org/repos/asf/kylin/blob/48cec941/job/src/test/java/org/apache/kylin/job/hadoop/hbase/TestHbaseClient.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/org/apache/kylin/job/hadoop/hbase/TestHbaseClient.java b/job/src/test/java/org/apache/kylin/job/hadoop/hbase/TestHbaseClient.java
index f2b9ed6..5a04d20 100644
--- a/job/src/test/java/org/apache/kylin/job/hadoop/hbase/TestHbaseClient.java
+++ b/job/src/test/java/org/apache/kylin/job/hadoop/hbase/TestHbaseClient.java
@@ -22,8 +22,11 @@ import java.io.IOException;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.kylin.common.util.Bytes;
 
 /**
@@ -90,13 +93,15 @@ public class TestHbaseClient {
         conf.set("hbase.zookeeper.quorum", "hbase_host");
         conf.set("zookeeper.znode.parent", "/hbase-unsecure");
 
-        HTable table = new HTable(conf, "test1");
+        Connection connection = ConnectionFactory.createConnection(conf);
+        Table table = connection.getTable(TableName.valueOf("test1"));
         Put put = new Put(Bytes.toBytes("row1"));
 
-        put.add(Bytes.toBytes("colfam1"), Bytes.toBytes("qual1"), Bytes.toBytes("val1"));
-        put.add(Bytes.toBytes("colfam1"), Bytes.toBytes("qual2"), Bytes.toBytes("val2"));
+        put.addColumn(Bytes.toBytes("colfam1"), Bytes.toBytes("qual1"), Bytes.toBytes("val1"));
+        put.addColumn(Bytes.toBytes("colfam1"), Bytes.toBytes("qual2"), Bytes.toBytes("val2"));
 
         table.put(put);
         table.close();
+        connection.close();
     }
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/48cec941/job/src/test/java/org/apache/kylin/job/tools/HBaseRowDigestTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/org/apache/kylin/job/tools/HBaseRowDigestTest.java b/job/src/test/java/org/apache/kylin/job/tools/HBaseRowDigestTest.java
index 9f9c23c..f5f94c8 100644
--- a/job/src/test/java/org/apache/kylin/job/tools/HBaseRowDigestTest.java
+++ b/job/src/test/java/org/apache/kylin/job/tools/HBaseRowDigestTest.java
@@ -23,10 +23,11 @@ import java.io.IOException;
 
 import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.client.HConnection;
-import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.kylin.common.persistence.HBaseConnection;
 import org.apache.kylin.common.util.BytesUtil;
@@ -60,11 +61,11 @@ public class HBaseRowDigestTest extends HBaseMetadataTestCase {
     @Test
     public static void test() throws IOException {
         String hbaseUrl = "hbase"; // use hbase-site.xml on classpath
-        HConnection conn = null;
-        HTableInterface table = null;
+        Connection conn = null;
+        Table table = null;
         try {
             conn = HBaseConnection.get(hbaseUrl);
-            table = conn.getTable("KYLIN_II_YTYWP3CQGJ");
+            table = conn.getTable(TableName.valueOf("KYLIN_II_YTYWP3CQGJ"));
             ResultScanner scanner = table.getScanner(CF, QN);
             StringBuffer sb = new StringBuffer();
             while (true) {

http://git-wip-us.apache.org/repos/asf/kylin/blob/48cec941/monitor/pom.xml
----------------------------------------------------------------------
diff --git a/monitor/pom.xml b/monitor/pom.xml
index 820934f..399535d 100644
--- a/monitor/pom.xml
+++ b/monitor/pom.xml
@@ -39,6 +39,12 @@
 
     <dependencies>
         <dependency>
+            <groupId>org.apache.kylin</groupId>
+            <artifactId>kylin-common</artifactId>
+            <version>${project.parent.version}</version>
+        </dependency>
+
+        <dependency>
             <groupId>junit</groupId>
             <artifactId>junit</artifactId>
             <scope>test</scope>

http://git-wip-us.apache.org/repos/asf/kylin/blob/48cec941/monitor/src/main/java/org/apache/kylin/monitor/MonitorMetaManager.java
----------------------------------------------------------------------
diff --git a/monitor/src/main/java/org/apache/kylin/monitor/MonitorMetaManager.java b/monitor/src/main/java/org/apache/kylin/monitor/MonitorMetaManager.java
index 97200fc..94b3937 100644
--- a/monitor/src/main/java/org/apache/kylin/monitor/MonitorMetaManager.java
+++ b/monitor/src/main/java/org/apache/kylin/monitor/MonitorMetaManager.java
@@ -20,18 +20,21 @@ package org.apache.kylin.monitor;
 
 import java.io.IOException;
 
+import org.apache.commons.io.IOUtils;
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
-import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.kylin.common.persistence.HBaseConnection;
 import org.apache.log4j.Logger;
 
 /**
@@ -122,11 +125,10 @@ public class MonitorMetaManager {
     public static String getListWithRowkey(String table, String rowkey) throws IOException {
         Result result = getResultByRowKey(table, rowkey);
         String fileList = null;
-        if (result.list() != null) {
-            for (KeyValue kv : result.list()) {
-                fileList = Bytes.toString(kv.getValue());
+        if (result.listCells() != null) {
+            for (Cell cell : result.listCells()) {
+                fileList = Bytes.toString(cell.getValueArray(), cell.getValueOffset(), cell.getValueOffset());
             }
-
         }
         fileList = fileList == null ? "" : fileList;
         return fileList;
@@ -164,16 +166,20 @@ public class MonitorMetaManager {
      * create table in hbase
      */
     public static void creatTable(String tableName, String[] family) throws Exception {
-        HBaseAdmin admin = new HBaseAdmin(conf);
-        HTableDescriptor desc = new HTableDescriptor(tableName);
-        for (int i = 0; i < family.length; i++) {
-            desc.addFamily(new HColumnDescriptor(family[i]));
-        }
-        if (admin.tableExists(tableName)) {
-            logger.info("table Exists!");
-        } else {
-            admin.createTable(desc);
-            logger.info("create table Success!");
+        Admin admin = HBaseConnection.get().getAdmin();
+        try {
+            HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName));
+            for (int i = 0; i < family.length; i++) {
+                desc.addFamily(new HColumnDescriptor(family[i]));
+            }
+            if (admin.tableExists(TableName.valueOf(tableName))) {
+                logger.info("table Exists!");
+            } else {
+                admin.createTable(desc);
+                logger.info("create table Success!");
+            }
+        } finally {
+            IOUtils.closeQuietly(admin);
         }
     }
 
@@ -181,13 +187,15 @@ public class MonitorMetaManager {
      * update cell in hbase
      */
     public static void updateData(String tableName, String rowKey, String family, String column, String value) throws IOException {
-        HTable table = new HTable(conf, Bytes.toBytes(tableName));
+        Table table = HBaseConnection.get().getTable(TableName.valueOf(tableName));
         Put put = new Put(rowKey.getBytes());
-        put.add(family.getBytes(), column.getBytes(), value.getBytes());
+        put.addColumn(family.getBytes(), column.getBytes(), value.getBytes());
         try {
             table.put(put);
         } catch (IOException e) {
             e.printStackTrace();
+        } finally {
+            IOUtils.closeQuietly(table);
         }
         logger.info("update table [" + tableName + "]");
         logger.info("rowKey [" + rowKey + "]");
@@ -200,9 +208,10 @@ public class MonitorMetaManager {
      * get result by rowkey
      */
     public static Result getResultByRowKey(String tableName, String rowKey) throws IOException {
-        HTable table = new HTable(conf, Bytes.toBytes(tableName));
+        Table table = HBaseConnection.get().getTable(TableName.valueOf(tableName));
         Get get = new Get(Bytes.toBytes(rowKey));
         Result result = table.get(get);
+        IOUtils.closeQuietly(table);
         return result;
     }
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/48cec941/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 793a5f2..6c70fa8 100644
--- a/pom.xml
+++ b/pom.xml
@@ -45,12 +45,13 @@
         <project.reporting.outputEncoding>UTF-8</project.reporting.outputEncoding>
 
         <!-- Hadoop versions -->
-        <hadoop2.version>2.6.0</hadoop2.version>
-        <yarn.version>2.6.0</yarn.version>
+        <hadoop2.version>2.7.1</hadoop2.version>
+        <yarn.version>2.7.1</yarn.version>
         <zookeeper.version>3.4.6</zookeeper.version>
-        <hive.version>0.14.0</hive.version>
-        <hive-hcatalog.version>0.14.0</hive-hcatalog.version>
-        <hbase-hadoop2.version>0.98.8-hadoop2</hbase-hadoop2.version>
+        <hive.version>1.2.1</hive.version>
+        <hive-hcatalog.version>1.2.1</hive-hcatalog.version>
+        <hbase-hadoop2.version>1.1.1</hbase-hadoop2.version>
+        <curator.version>2.7.1</curator.version>
 
         <!-- Dependency versions -->
         <antlr.version>3.4</antlr.version>
@@ -90,9 +91,6 @@
         <!-- Calcite Version -->
         <calcite.version>1.6.0</calcite.version>
 
-        <!-- Curator.version Version -->
-        <curator.version>2.6.0</curator.version>
-
         <!-- Sonar -->
         <sonar.java.coveragePlugin>jacoco</sonar.java.coveragePlugin>
         <sonar.dynamicAnalysis>reuseReports</sonar.dynamicAnalysis>

http://git-wip-us.apache.org/repos/asf/kylin/blob/48cec941/server/src/main/java/org/apache/kylin/rest/service/AclService.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/kylin/rest/service/AclService.java b/server/src/main/java/org/apache/kylin/rest/service/AclService.java
index ea2a48e..8a1cf6d 100644
--- a/server/src/main/java/org/apache/kylin/rest/service/AclService.java
+++ b/server/src/main/java/org/apache/kylin/rest/service/AclService.java
@@ -29,13 +29,14 @@ import java.util.Map;
 import java.util.NavigableMap;
 
 import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
 import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
 import org.apache.kylin.common.KylinConfig;
@@ -130,9 +131,9 @@ public class AclService implements MutableAclService {
     @Override
     public List<ObjectIdentity> findChildren(ObjectIdentity parentIdentity) {
         List<ObjectIdentity> oids = new ArrayList<ObjectIdentity>();
-        HTableInterface htable = null;
+        Table htable = null;
         try {
-            htable = HBaseConnection.get(hbaseUrl).getTable(aclTableName);
+            htable = HBaseConnection.get(hbaseUrl).getTable(TableName.valueOf(aclTableName));
 
             Scan scan = new Scan();
             SingleColumnValueFilter parentFilter = new SingleColumnValueFilter(Bytes.toBytes(ACL_INFO_FAMILY), Bytes.toBytes(ACL_INFO_FAMILY_PARENT_COLUMN), CompareOp.EQUAL, domainObjSerializer.serialize(new DomainObjectInfo(parentIdentity)));
@@ -179,10 +180,10 @@ public class AclService implements MutableAclService {
     @Override
     public Map<ObjectIdentity, Acl> readAclsById(List<ObjectIdentity> oids, List<Sid> sids) throws NotFoundException {
         Map<ObjectIdentity, Acl> aclMaps = new HashMap<ObjectIdentity, Acl>();
-        HTableInterface htable = null;
+        Table htable = null;
         Result result = null;
         try {
-            htable = HBaseConnection.get(hbaseUrl).getTable(aclTableName);
+            htable = HBaseConnection.get(hbaseUrl).getTable(TableName.valueOf(aclTableName));
 
             for (ObjectIdentity oid : oids) {
                 result = htable.get(new Get(Bytes.toBytes(String.valueOf(oid.getIdentifier()))));
@@ -231,16 +232,15 @@ public class AclService implements MutableAclService {
         Authentication auth = SecurityContextHolder.getContext().getAuthentication();
         PrincipalSid sid = new PrincipalSid(auth);
 
-        HTableInterface htable = null;
+        Table htable = null;
         try {
-            htable = HBaseConnection.get(hbaseUrl).getTable(aclTableName);
+            htable = HBaseConnection.get(hbaseUrl).getTable(TableName.valueOf(aclTableName));
             Put put = new Put(Bytes.toBytes(String.valueOf(objectIdentity.getIdentifier())));
-            put.add(Bytes.toBytes(ACL_INFO_FAMILY), Bytes.toBytes(ACL_INFO_FAMILY_TYPE_COLUMN), Bytes.toBytes(objectIdentity.getType()));
-            put.add(Bytes.toBytes(ACL_INFO_FAMILY), Bytes.toBytes(ACL_INFO_FAMILY_OWNER_COLUMN), sidSerializer.serialize(new SidInfo(sid)));
-            put.add(Bytes.toBytes(ACL_INFO_FAMILY), Bytes.toBytes(ACL_INFO_FAMILY_ENTRY_INHERIT_COLUMN), Bytes.toBytes(true));
+            put.addColumn(Bytes.toBytes(ACL_INFO_FAMILY), Bytes.toBytes(ACL_INFO_FAMILY_TYPE_COLUMN), Bytes.toBytes(objectIdentity.getType()));
+            put.addColumn(Bytes.toBytes(ACL_INFO_FAMILY), Bytes.toBytes(ACL_INFO_FAMILY_OWNER_COLUMN), sidSerializer.serialize(new SidInfo(sid)));
+            put.addColumn(Bytes.toBytes(ACL_INFO_FAMILY), Bytes.toBytes(ACL_INFO_FAMILY_ENTRY_INHERIT_COLUMN), Bytes.toBytes(true));
 
             htable.put(put);
-            htable.flushCommits();
 
             logger.debug("ACL of " + objectIdentity + " created successfully.");
         } catch (IOException e) {
@@ -254,9 +254,9 @@ public class AclService implements MutableAclService {
 
     @Override
     public void deleteAcl(ObjectIdentity objectIdentity, boolean deleteChildren) throws ChildrenExistException {
-        HTableInterface htable = null;
+        Table htable = null;
         try {
-            htable = HBaseConnection.get(hbaseUrl).getTable(aclTableName);
+            htable = HBaseConnection.get(hbaseUrl).getTable(TableName.valueOf(aclTableName));
             Delete delete = new Delete(Bytes.toBytes(String.valueOf(objectIdentity.getIdentifier())));
 
             List<ObjectIdentity> children = findChildren(objectIdentity);
@@ -269,7 +269,6 @@ public class AclService implements MutableAclService {
             }
 
             htable.delete(delete);
-            htable.flushCommits();
 
             logger.debug("ACL of " + objectIdentity + " deleted successfully.");
         } catch (IOException e) {
@@ -287,27 +286,26 @@ public class AclService implements MutableAclService {
             throw e;
         }
 
-        HTableInterface htable = null;
+        Table htable = null;
         try {
-            htable = HBaseConnection.get(hbaseUrl).getTable(aclTableName);
+            htable = HBaseConnection.get(hbaseUrl).getTable(TableName.valueOf(aclTableName));
             Delete delete = new Delete(Bytes.toBytes(String.valueOf(acl.getObjectIdentity().getIdentifier())));
-            delete.deleteFamily(Bytes.toBytes(ACL_ACES_FAMILY));
+            delete.addFamily(Bytes.toBytes(ACL_ACES_FAMILY));
             htable.delete(delete);
 
             Put put = new Put(Bytes.toBytes(String.valueOf(acl.getObjectIdentity().getIdentifier())));
 
             if (null != acl.getParentAcl()) {
-                put.add(Bytes.toBytes(ACL_INFO_FAMILY), Bytes.toBytes(ACL_INFO_FAMILY_PARENT_COLUMN), domainObjSerializer.serialize(new DomainObjectInfo(acl.getParentAcl().getObjectIdentity())));
+                put.addColumn(Bytes.toBytes(ACL_INFO_FAMILY), Bytes.toBytes(ACL_INFO_FAMILY_PARENT_COLUMN), domainObjSerializer.serialize(new DomainObjectInfo(acl.getParentAcl().getObjectIdentity())));
             }
 
             for (AccessControlEntry ace : acl.getEntries()) {
                 AceInfo aceInfo = new AceInfo(ace);
-                put.add(Bytes.toBytes(ACL_ACES_FAMILY), Bytes.toBytes(aceInfo.getSidInfo().getSid()), aceSerializer.serialize(aceInfo));
+                put.addColumn(Bytes.toBytes(ACL_ACES_FAMILY), Bytes.toBytes(aceInfo.getSidInfo().getSid()), aceSerializer.serialize(aceInfo));
             }
 
             if (!put.isEmpty()) {
                 htable.put(put);
-                htable.flushCommits();
 
                 logger.debug("ACL of " + acl.getObjectIdentity() + " updated successfully.");
             }

http://git-wip-us.apache.org/repos/asf/kylin/blob/48cec941/server/src/main/java/org/apache/kylin/rest/service/CubeService.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/kylin/rest/service/CubeService.java b/server/src/main/java/org/apache/kylin/rest/service/CubeService.java
index 601814c..a1acccd 100644
--- a/server/src/main/java/org/apache/kylin/rest/service/CubeService.java
+++ b/server/src/main/java/org/apache/kylin/rest/service/CubeService.java
@@ -28,9 +28,9 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.Connection;
 import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.persistence.HBaseConnection;
 import org.apache.kylin.common.util.HBaseRegionSizeCalculator;
 import org.apache.kylin.common.util.HadoopUtil;
 import org.apache.kylin.cube.CubeInstance;
@@ -397,33 +397,24 @@ public class CubeService extends BasicService {
      * @throws IOException Exception when HTable resource is not closed correctly.
      */
     public HBaseResponse getHTableInfo(String tableName) throws IOException {
-        Configuration hconf = HadoopUtil.getCurrentHBaseConfiguration();
-        HTable table = null;
+        Connection conn = HBaseConnection.get();
         HBaseResponse hr = null;
         long tableSize = 0;
         int regionCount = 0;
 
-        try {
-            table = new HTable(hconf, tableName);
-
-            HBaseRegionSizeCalculator cal = new HBaseRegionSizeCalculator(table);
-            Map<byte[], Long> sizeMap = cal.getRegionSizeMap();
+        HBaseRegionSizeCalculator cal = new HBaseRegionSizeCalculator(tableName, conn);
+        Map<byte[], Long> sizeMap = cal.getRegionSizeMap();
 
-            for (long s : sizeMap.values()) {
-                tableSize += s;
-            }
+        for (long s : sizeMap.values()) {
+            tableSize += s;
+        }
 
-            regionCount = sizeMap.size();
+        regionCount = sizeMap.size();
 
-            // Set response.
-            hr = new HBaseResponse();
-            hr.setTableSize(tableSize);
-            hr.setRegionCount(regionCount);
-        } finally {
-            if (null != table) {
-                table.close();
-            }
-        }
+        // Set response.
+        hr = new HBaseResponse();
+        hr.setTableSize(tableSize);
+        hr.setRegionCount(regionCount);
 
         return hr;
     }

http://git-wip-us.apache.org/repos/asf/kylin/blob/48cec941/server/src/main/java/org/apache/kylin/rest/service/QueryService.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/kylin/rest/service/QueryService.java b/server/src/main/java/org/apache/kylin/rest/service/QueryService.java
index 764df4b..7d14021 100644
--- a/server/src/main/java/org/apache/kylin/rest/service/QueryService.java
+++ b/server/src/main/java/org/apache/kylin/rest/service/QueryService.java
@@ -42,10 +42,11 @@ import javax.sql.DataSource;
 
 import org.apache.calcite.avatica.ColumnMetaData.Rep;
 import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.debug.BackdoorToggles;
 import org.apache.kylin.common.persistence.HBaseConnection;
@@ -124,14 +125,13 @@ public class QueryService extends BasicService {
         Query[] queryArray = new Query[queries.size()];
 
         byte[] bytes = querySerializer.serialize(queries.toArray(queryArray));
-        HTableInterface htable = null;
+        Table htable = null;
         try {
-            htable = HBaseConnection.get(hbaseUrl).getTable(userTableName);
+            htable = HBaseConnection.get(hbaseUrl).getTable(TableName.valueOf(userTableName));
             Put put = new Put(Bytes.toBytes(creator));
-            put.add(Bytes.toBytes(USER_QUERY_FAMILY), Bytes.toBytes(USER_QUERY_COLUMN), bytes);
+            put.addColumn(Bytes.toBytes(USER_QUERY_FAMILY), Bytes.toBytes(USER_QUERY_COLUMN), bytes);
 
             htable.put(put);
-            htable.flushCommits();
         } finally {
             IOUtils.closeQuietly(htable);
         }
@@ -157,14 +157,13 @@ public class QueryService extends BasicService {
 
         Query[] queryArray = new Query[queries.size()];
         byte[] bytes = querySerializer.serialize(queries.toArray(queryArray));
-        HTableInterface htable = null;
+        Table htable = null;
         try {
-            htable = HBaseConnection.get(hbaseUrl).getTable(userTableName);
+            htable = HBaseConnection.get(hbaseUrl).getTable(TableName.valueOf(userTableName));
             Put put = new Put(Bytes.toBytes(creator));
-            put.add(Bytes.toBytes(USER_QUERY_FAMILY), Bytes.toBytes(USER_QUERY_COLUMN), bytes);
+            put.addColumn(Bytes.toBytes(USER_QUERY_FAMILY), Bytes.toBytes(USER_QUERY_COLUMN), bytes);
 
             htable.put(put);
-            htable.flushCommits();
         } finally {
             IOUtils.closeQuietly(htable);
         }
@@ -176,9 +175,9 @@ public class QueryService extends BasicService {
         }
 
         List<Query> queries = new ArrayList<Query>();
-        HTableInterface htable = null;
+        Table htable = null;
         try {
-            htable = HBaseConnection.get(hbaseUrl).getTable(userTableName);
+            htable = HBaseConnection.get(hbaseUrl).getTable(TableName.valueOf(userTableName));
             Get get = new Get(Bytes.toBytes(creator));
             get.addFamily(Bytes.toBytes(USER_QUERY_FAMILY));
             Result result = htable.get(get);

http://git-wip-us.apache.org/repos/asf/kylin/blob/48cec941/server/src/main/java/org/apache/kylin/rest/service/UserService.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/kylin/rest/service/UserService.java b/server/src/main/java/org/apache/kylin/rest/service/UserService.java
index d665ab9..d03cd55 100644
--- a/server/src/main/java/org/apache/kylin/rest/service/UserService.java
+++ b/server/src/main/java/org/apache/kylin/rest/service/UserService.java
@@ -25,13 +25,14 @@ import java.util.Collection;
 import java.util.List;
 
 import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.persistence.HBaseConnection;
 import org.apache.kylin.common.util.Bytes;
@@ -75,9 +76,9 @@ public class UserService implements UserManager {
 
     @Override
     public UserDetails loadUserByUsername(String username) throws UsernameNotFoundException {
-        HTableInterface htable = null;
+        Table htable = null;
         try {
-            htable = HBaseConnection.get(hbaseUrl).getTable(userTableName);
+            htable = HBaseConnection.get(hbaseUrl).getTable(TableName.valueOf(userTableName));
 
             Get get = new Get(Bytes.toBytes(username));
             get.addFamily(Bytes.toBytes(USER_AUTHORITY_FAMILY));
@@ -106,15 +107,14 @@ public class UserService implements UserManager {
 
     @Override
     public void updateUser(UserDetails user) {
-        HTableInterface htable = null;
+        Table htable = null;
         try {
             byte[] userAuthorities = serialize(user.getAuthorities());
-            htable = HBaseConnection.get(hbaseUrl).getTable(userTableName);
+            htable = HBaseConnection.get(hbaseUrl).getTable(TableName.valueOf(userTableName));
             Put put = new Put(Bytes.toBytes(user.getUsername()));
-            put.add(Bytes.toBytes(USER_AUTHORITY_FAMILY), Bytes.toBytes(USER_AUTHORITY_COLUMN), userAuthorities);
+            put.addColumn(Bytes.toBytes(USER_AUTHORITY_FAMILY), Bytes.toBytes(USER_AUTHORITY_COLUMN), userAuthorities);
 
             htable.put(put);
-            htable.flushCommits();
         } catch (IOException e) {
             throw new RuntimeException(e.getMessage(), e);
         } finally {
@@ -124,13 +124,12 @@ public class UserService implements UserManager {
 
     @Override
     public void deleteUser(String username) {
-        HTableInterface htable = null;
+        Table htable = null;
         try {
-            htable = HBaseConnection.get(hbaseUrl).getTable(userTableName);
+            htable = HBaseConnection.get(hbaseUrl).getTable(TableName.valueOf(userTableName));
             Delete delete = new Delete(Bytes.toBytes(username));
 
             htable.delete(delete);
-            htable.flushCommits();
         } catch (IOException e) {
             throw new RuntimeException(e.getMessage(), e);
         } finally {
@@ -145,9 +144,9 @@ public class UserService implements UserManager {
 
     @Override
     public boolean userExists(String username) {
-        HTableInterface htable = null;
+        Table htable = null;
         try {
-            htable = HBaseConnection.get(hbaseUrl).getTable(userTableName);
+            htable = HBaseConnection.get(hbaseUrl).getTable(TableName.valueOf(userTableName));
             Result result = htable.get(new Get(Bytes.toBytes(username)));
 
             return null != result && !result.isEmpty();
@@ -164,10 +163,10 @@ public class UserService implements UserManager {
         s.addColumn(Bytes.toBytes(USER_AUTHORITY_FAMILY), Bytes.toBytes(USER_AUTHORITY_COLUMN));
 
         List<String> authorities = new ArrayList<String>();
-        HTableInterface htable = null;
+        Table htable = null;
         ResultScanner scanner = null;
         try {
-            htable = HBaseConnection.get(hbaseUrl).getTable(userTableName);
+            htable = HBaseConnection.get(hbaseUrl).getTable(TableName.valueOf(userTableName));
             scanner = htable.getScanner(s);
 
             for (Result result = scanner.next(); result != null; result = scanner.next()) {

http://git-wip-us.apache.org/repos/asf/kylin/blob/48cec941/storage/src/main/java/org/apache/kylin/storage/filter/BitMapFilterEvaluator.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/filter/BitMapFilterEvaluator.java b/storage/src/main/java/org/apache/kylin/storage/filter/BitMapFilterEvaluator.java
index f529145..4aeb676 100644
--- a/storage/src/main/java/org/apache/kylin/storage/filter/BitMapFilterEvaluator.java
+++ b/storage/src/main/java/org/apache/kylin/storage/filter/BitMapFilterEvaluator.java
@@ -25,7 +25,6 @@ import org.apache.kylin.metadata.filter.CompareTupleFilter;
 import org.apache.kylin.metadata.filter.LogicalTupleFilter;
 import org.apache.kylin.metadata.filter.TupleFilter;
 import org.apache.kylin.metadata.model.TblColRef;
-
 import org.roaringbitmap.RoaringBitmap;
 
 /**

http://git-wip-us.apache.org/repos/asf/kylin/blob/48cec941/storage/src/main/java/org/apache/kylin/storage/hbase/CubeSegmentTupleIterator.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/hbase/CubeSegmentTupleIterator.java b/storage/src/main/java/org/apache/kylin/storage/hbase/CubeSegmentTupleIterator.java
index 26c4f29..41526c2 100644
--- a/storage/src/main/java/org/apache/kylin/storage/hbase/CubeSegmentTupleIterator.java
+++ b/storage/src/main/java/org/apache/kylin/storage/hbase/CubeSegmentTupleIterator.java
@@ -28,16 +28,16 @@ import java.util.Map;
 import java.util.Map.Entry;
 import java.util.NoSuchElementException;
 
-import org.apache.hadoop.hbase.client.HConnection;
-import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
 import org.apache.hadoop.hbase.filter.Filter;
 import org.apache.hadoop.hbase.filter.FilterList;
 import org.apache.hadoop.hbase.filter.FuzzyRowFilter;
-import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.persistence.StorageException;
 import org.apache.kylin.common.util.Array;
@@ -84,7 +84,7 @@ public class CubeSegmentTupleIterator implements ITupleIterator {
     private final List<RowValueDecoder> rowValueDecoders;
     private final StorageContext context;
     private final String tableName;
-    private final HTableInterface table;
+    private final Table table;
     private final RowKeyDecoder rowKeyDecoder;
     private final Iterator<HBaseKeyRange> rangeIterator;
 
@@ -105,7 +105,7 @@ public class CubeSegmentTupleIterator implements ITupleIterator {
     private int advMeasureRowsRemaining;
     private int advMeasureRowIndex;
 
-    public CubeSegmentTupleIterator(CubeSegment cubeSeg, List<HBaseKeyRange> keyRanges, HConnection conn, Collection<TblColRef> dimensions, TupleFilter filter, Collection<TblColRef> groupBy, List<RowValueDecoder> rowValueDecoders, StorageContext context) {
+    public CubeSegmentTupleIterator(CubeSegment cubeSeg, List<HBaseKeyRange> keyRanges, Connection conn, Collection<TblColRef> dimensions, TupleFilter filter, Collection<TblColRef> groupBy, List<RowValueDecoder> rowValueDecoders, StorageContext context) {
         this.cube = cubeSeg.getCubeInstance();
         this.cubeSeg = cubeSeg;
         this.dimensions = dimensions;
@@ -127,7 +127,7 @@ public class CubeSegmentTupleIterator implements ITupleIterator {
 
 
         try {
-            this.table = conn.getTable(tableName);
+            this.table = conn.getTable(TableName.valueOf(tableName));
         } catch (Throwable t) {
             throw new StorageException("Error when open connection to table " + tableName, t);
         }
@@ -144,9 +144,8 @@ public class CubeSegmentTupleIterator implements ITupleIterator {
 
         if (logger.isDebugEnabled() && scan != null) {
             logger.debug("Scan " + scan.toString());
-            byte[] metricsBytes = scan.getAttribute(Scan.SCAN_ATTRIBUTES_METRICS_DATA);
-            if (metricsBytes != null) {
-                ScanMetrics scanMetrics = ProtobufUtil.toScanMetrics(metricsBytes);
+            ScanMetrics scanMetrics = scan.getScanMetrics();
+            if (scanMetrics != null) {
                 logger.debug("HBase Metrics: " + "count={}, ms={}, bytes={}, remote_bytes={}, regions={}, not_serving_region={}, rpc={}, rpc_retries={}, remote_rpc={}, remote_rpc_retries={}", new Object[] { scanCount, scanMetrics.sumOfMillisSecBetweenNexts, scanMetrics.countOfBytesInResults, scanMetrics.countOfBytesInRemoteResults, scanMetrics.countOfRegions, scanMetrics.countOfNSRE, scanMetrics.countOfRPCcalls, scanMetrics.countOfRPCRetries, scanMetrics.countOfRemoteRPCcalls, scanMetrics.countOfRemoteRPCRetries });
             }
         }
@@ -303,7 +302,7 @@ public class CubeSegmentTupleIterator implements ITupleIterator {
     private Scan buildScan(HBaseKeyRange keyRange) {
         Scan scan = new Scan();
         tuneScanParameters(scan);
-        scan.setAttribute(Scan.SCAN_ATTRIBUTES_METRICS_ENABLE, Bytes.toBytes(Boolean.TRUE));
+        scan.setScanMetricsEnabled(true);
         for (RowValueDecoder valueDecoder : this.rowValueDecoders) {
             HBaseColumnDesc hbaseColumn = valueDecoder.getHBaseColumn();
             byte[] byteFamily = Bytes.toBytes(hbaseColumn.getColumnFamilyName());

http://git-wip-us.apache.org/repos/asf/kylin/blob/48cec941/storage/src/main/java/org/apache/kylin/storage/hbase/CubeStorageEngine.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/hbase/CubeStorageEngine.java b/storage/src/main/java/org/apache/kylin/storage/hbase/CubeStorageEngine.java
index 626b784..7510dcd 100644
--- a/storage/src/main/java/org/apache/kylin/storage/hbase/CubeStorageEngine.java
+++ b/storage/src/main/java/org/apache/kylin/storage/hbase/CubeStorageEngine.java
@@ -32,7 +32,7 @@ import java.util.Map;
 import java.util.Set;
 import java.util.TreeSet;
 
-import org.apache.hadoop.hbase.client.HConnection;
+import org.apache.hadoop.hbase.client.Connection;
 import org.apache.kylin.common.persistence.HBaseConnection;
 import org.apache.kylin.common.util.Bytes;
 import org.apache.kylin.common.util.Dictionary;
@@ -142,7 +142,7 @@ public class CubeStorageEngine implements IStorageEngine {
         setCoprocessor(groupsCopD, valueDecoders, context); // enable coprocessor if beneficial
         setLimit(filter, context);
 
-        HConnection conn = HBaseConnection.get(context.getConnUrl());
+        Connection conn = HBaseConnection.get(context.getConnUrl());
         return new SerializedHBaseTupleIterator(conn, scans, cubeInstance, dimensionsD, filterD, groupsCopD, valueDecoders, context);
     }
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/48cec941/storage/src/main/java/org/apache/kylin/storage/hbase/HBaseClientKVIterator.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/hbase/HBaseClientKVIterator.java b/storage/src/main/java/org/apache/kylin/storage/hbase/HBaseClientKVIterator.java
index 918fd4b..6a76baa 100644
--- a/storage/src/main/java/org/apache/kylin/storage/hbase/HBaseClientKVIterator.java
+++ b/storage/src/main/java/org/apache/kylin/storage/hbase/HBaseClientKVIterator.java
@@ -1,93 +1,94 @@
-/*
- * 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.storage.hbase;
-
-import java.io.Closeable;
-import java.io.IOException;
-import java.util.Iterator;
-
-import org.apache.commons.io.IOUtils;
-import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.client.HConnection;
-import org.apache.hadoop.hbase.client.HTableInterface;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.ResultScanner;
-import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.kylin.common.util.Pair;
-import org.apache.kylin.invertedindex.model.IIDesc;
-
-/**
- * @author yangli9
- * 
- */
-public class HBaseClientKVIterator implements Iterable<Pair<ImmutableBytesWritable, ImmutableBytesWritable>>, Closeable {
-
-    byte[] family;
-    byte[] qualifier;
-
-    HTableInterface table;
-    ResultScanner scanner;
-    Iterator<Result> iterator;
-
-    public HBaseClientKVIterator(HConnection hconn, String tableName, byte[] family, byte[] qualifier) throws IOException {
-        this.family = family;
-        this.qualifier = qualifier;
-
-        this.table = hconn.getTable(tableName);
-        this.scanner = table.getScanner(family, qualifier);
-        this.iterator = scanner.iterator();
-    }
-
-    @Override
-    public void close() {
-        IOUtils.closeQuietly(scanner);
-        IOUtils.closeQuietly(table);
-    }
-
-    @Override
-    public Iterator<Pair<ImmutableBytesWritable, ImmutableBytesWritable>> iterator() {
-        return new MyIterator();
-    }
-
-    private class MyIterator implements Iterator<Pair<ImmutableBytesWritable, ImmutableBytesWritable>> {
-
-        ImmutableBytesWritable key = new ImmutableBytesWritable();
-        ImmutableBytesWritable value = new ImmutableBytesWritable();
-        Pair<ImmutableBytesWritable, ImmutableBytesWritable> pair = new Pair<ImmutableBytesWritable, ImmutableBytesWritable>(key, value);
-
-        @Override
-        public boolean hasNext() {
-            return iterator.hasNext();
-        }
-
-        @Override
-        public Pair<ImmutableBytesWritable, ImmutableBytesWritable> next() {
-            Result r = iterator.next();
-            Cell c = r.getColumnLatestCell(IIDesc.HBASE_FAMILY_BYTES, IIDesc.HBASE_QUALIFIER_BYTES);
-            key.set(c.getRowArray(), c.getRowOffset(), c.getRowLength());
-            value.set(c.getValueArray(), c.getValueOffset(), c.getValueLength());
-            return pair;
-        }
-
-        public void remove() {
-            throw new UnsupportedOperationException();
-        }
-
-    }
-}
+/*
+ * 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.storage.hbase;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Iterator;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.kylin.common.util.Pair;
+import org.apache.kylin.invertedindex.model.IIDesc;
+
+/**
+ * @author yangli9
+ * 
+ */
+public class HBaseClientKVIterator implements Iterable<Pair<ImmutableBytesWritable, ImmutableBytesWritable>>, Closeable {
+
+    byte[] family;
+    byte[] qualifier;
+
+    Table table;
+    ResultScanner scanner;
+    Iterator<Result> iterator;
+
+    public HBaseClientKVIterator(Connection hconn, String tableName, byte[] family, byte[] qualifier) throws IOException {
+        this.family = family;
+        this.qualifier = qualifier;
+
+        this.table = hconn.getTable(TableName.valueOf(tableName));
+        this.scanner = table.getScanner(family, qualifier);
+        this.iterator = scanner.iterator();
+    }
+
+    @Override
+    public void close() {
+        IOUtils.closeQuietly(scanner);
+        IOUtils.closeQuietly(table);
+    }
+
+    @Override
+    public Iterator<Pair<ImmutableBytesWritable, ImmutableBytesWritable>> iterator() {
+        return new MyIterator();
+    }
+
+    private class MyIterator implements Iterator<Pair<ImmutableBytesWritable, ImmutableBytesWritable>> {
+
+        ImmutableBytesWritable key = new ImmutableBytesWritable();
+        ImmutableBytesWritable value = new ImmutableBytesWritable();
+        Pair<ImmutableBytesWritable, ImmutableBytesWritable> pair = new Pair<ImmutableBytesWritable, ImmutableBytesWritable>(key, value);
+
+        @Override
+        public boolean hasNext() {
+            return iterator.hasNext();
+        }
+
+        @Override
+        public Pair<ImmutableBytesWritable, ImmutableBytesWritable> next() {
+            Result r = iterator.next();
+            Cell c = r.getColumnLatestCell(IIDesc.HBASE_FAMILY_BYTES, IIDesc.HBASE_QUALIFIER_BYTES);
+            key.set(c.getRowArray(), c.getRowOffset(), c.getRowLength());
+            value.set(c.getValueArray(), c.getValueOffset(), c.getValueLength());
+            return pair;
+        }
+
+        public void remove() {
+            throw new UnsupportedOperationException();
+        }
+
+    }
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/48cec941/storage/src/main/java/org/apache/kylin/storage/hbase/InvertedIndexStorageEngine.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/hbase/InvertedIndexStorageEngine.java b/storage/src/main/java/org/apache/kylin/storage/hbase/InvertedIndexStorageEngine.java
index afb49c0..e518a4c 100644
--- a/storage/src/main/java/org/apache/kylin/storage/hbase/InvertedIndexStorageEngine.java
+++ b/storage/src/main/java/org/apache/kylin/storage/hbase/InvertedIndexStorageEngine.java
@@ -1,57 +1,57 @@
-/*
- * 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.storage.hbase;
-
-import java.util.ArrayList;
-
-import org.apache.hadoop.hbase.client.HConnection;
-import org.apache.kylin.common.persistence.HBaseConnection;
-import org.apache.kylin.invertedindex.IIInstance;
-import org.apache.kylin.invertedindex.IISegment;
-import org.apache.kylin.metadata.realization.SQLDigest;
-import org.apache.kylin.metadata.tuple.ITupleIterator;
-import org.apache.kylin.storage.IStorageEngine;
-import org.apache.kylin.storage.StorageContext;
-import org.apache.kylin.storage.hbase.coprocessor.endpoint.EndpointTupleIterator;
-
-/**
- * @author yangli9
- */
-public class InvertedIndexStorageEngine implements IStorageEngine {
-
-    private IISegment seg;
-
-    public InvertedIndexStorageEngine(IIInstance ii) {
-        this.seg = ii.getFirstSegment();
-    }
-
-    @Override
-    public ITupleIterator search(StorageContext context, SQLDigest sqlDigest) {
-        String tableName = seg.getStorageLocationIdentifier();
-
-        //HConnection is cached, so need not be closed
-        HConnection conn = HBaseConnection.get(context.getConnUrl());
-        try {
-            return new EndpointTupleIterator(seg, sqlDigest.filter, sqlDigest.groupbyColumns, new ArrayList<>(sqlDigest.aggregations), context, conn);
-        } catch (Throwable e) {
-            e.printStackTrace();
-            throw new IllegalStateException("Error when connecting to II htable " + tableName, e);
-        }
-    }
-}
+/*
+ * 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.storage.hbase;
+
+import java.util.ArrayList;
+
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.kylin.common.persistence.HBaseConnection;
+import org.apache.kylin.invertedindex.IIInstance;
+import org.apache.kylin.invertedindex.IISegment;
+import org.apache.kylin.metadata.realization.SQLDigest;
+import org.apache.kylin.metadata.tuple.ITupleIterator;
+import org.apache.kylin.storage.IStorageEngine;
+import org.apache.kylin.storage.StorageContext;
+import org.apache.kylin.storage.hbase.coprocessor.endpoint.EndpointTupleIterator;
+
+/**
+ * @author yangli9
+ */
+public class InvertedIndexStorageEngine implements IStorageEngine {
+
+    private IISegment seg;
+
+    public InvertedIndexStorageEngine(IIInstance ii) {
+        this.seg = ii.getFirstSegment();
+    }
+
+    @Override
+    public ITupleIterator search(StorageContext context, SQLDigest sqlDigest) {
+        String tableName = seg.getStorageLocationIdentifier();
+
+        // Connection is cached, so need not be closed
+        Connection conn = HBaseConnection.get(context.getConnUrl());
+        try {
+            return new EndpointTupleIterator(seg, sqlDigest.filter, sqlDigest.groupbyColumns, new ArrayList<>(sqlDigest.aggregations), context, conn);
+        } catch (Throwable e) {
+            e.printStackTrace();
+            throw new IllegalStateException("Error when connecting to II htable " + tableName, e);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/48cec941/storage/src/main/java/org/apache/kylin/storage/hbase/PingHBaseCLI.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/hbase/PingHBaseCLI.java b/storage/src/main/java/org/apache/kylin/storage/hbase/PingHBaseCLI.java
index d4e8529..4a9c574 100644
--- a/storage/src/main/java/org/apache/kylin/storage/hbase/PingHBaseCLI.java
+++ b/storage/src/main/java/org/apache/kylin/storage/hbase/PingHBaseCLI.java
@@ -1,88 +1,91 @@
-/*
- * 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.storage.hbase;
-
-import java.io.IOException;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.client.HConnection;
-import org.apache.hadoop.hbase.client.HConnectionManager;
-import org.apache.hadoop.hbase.client.HTableInterface;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.ResultScanner;
-import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.security.User;
-import org.apache.hadoop.hbase.security.token.TokenUtil;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.kylin.common.util.Bytes;
-import org.apache.kylin.common.util.HadoopUtil;
-
-/**
- * @author yangli9
- * 
- */
-public class PingHBaseCLI {
-
-    public static void main(String[] args) throws IOException {
-        String hbaseTable = args[0];
-
-        System.out.println("Hello friend.");
-
-        Configuration hconf = HadoopUtil.getCurrentHBaseConfiguration();
-        if (User.isHBaseSecurityEnabled(hconf)) {
-            try {
-                System.out.println("--------------Getting kerberos credential for user " + UserGroupInformation.getCurrentUser().getUserName());
-                TokenUtil.obtainAndCacheToken(hconf, UserGroupInformation.getCurrentUser());
-            } catch (InterruptedException e) {
-                System.out.println("--------------Error while getting kerberos credential for user " + UserGroupInformation.getCurrentUser().getUserName());
-            }
-        }
-
-        Scan scan = new Scan();
-        int limit = 20;
-
-        HConnection conn = null;
-        HTableInterface table = null;
-        ResultScanner scanner = null;
-        try {
-            conn = HConnectionManager.createConnection(hconf);
-            table = conn.getTable(hbaseTable);
-            scanner = table.getScanner(scan);
-            int count = 0;
-            for (Result r : scanner) {
-                byte[] rowkey = r.getRow();
-                System.out.println(Bytes.toStringBinary(rowkey));
-                count++;
-                if (count == limit)
-                    break;
-            }
-        } finally {
-            if (scanner != null) {
-                scanner.close();
-            }
-            if (table != null) {
-                table.close();
-            }
-            if (conn != null) {
-                conn.close();
-            }
-        }
-
-    }
-}
+/*
+ * 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.storage.hbase;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.security.UserProvider;
+import org.apache.hadoop.hbase.security.token.TokenUtil;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.kylin.common.util.Bytes;
+import org.apache.kylin.common.util.HadoopUtil;
+
+/**
+ * @author yangli9
+ * 
+ */
+public class PingHBaseCLI {
+
+    public static void main(String[] args) throws IOException, InterruptedException {
+        String hbaseTable = args[0];
+
+        System.out.println("Hello friend.");
+
+        Configuration hconf = HadoopUtil.getCurrentHBaseConfiguration();
+        if (User.isHBaseSecurityEnabled(hconf)) {
+            Connection conn = ConnectionFactory.createConnection(hconf);
+            try {
+                UserProvider userProvider = UserProvider.instantiate(hconf);
+                TokenUtil.obtainAndCacheToken(conn, userProvider.create(UserGroupInformation.getCurrentUser()));
+            } finally {
+                conn.close();
+            }
+        }
+
+        Scan scan = new Scan();
+        int limit = 20;
+
+        Connection conn = null;
+        Table table = null;
+        ResultScanner scanner = null;
+        try {
+            conn = ConnectionFactory.createConnection(hconf);
+            table = conn.getTable(TableName.valueOf(hbaseTable));
+            scanner = table.getScanner(scan);
+            int count = 0;
+            for (Result r : scanner) {
+                byte[] rowkey = r.getRow();
+                System.out.println(Bytes.toStringBinary(rowkey));
+                count++;
+                if (count == limit)
+                    break;
+            }
+        } finally {
+            if (scanner != null) {
+                scanner.close();
+            }
+            if (table != null) {
+                table.close();
+            }
+            if (conn != null) {
+                conn.close();
+            }
+        }
+
+    }
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/48cec941/storage/src/main/java/org/apache/kylin/storage/hbase/RegionScannerAdapter.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/hbase/RegionScannerAdapter.java b/storage/src/main/java/org/apache/kylin/storage/hbase/RegionScannerAdapter.java
index e2eeed0..a07cbe4 100644
--- a/storage/src/main/java/org/apache/kylin/storage/hbase/RegionScannerAdapter.java
+++ b/storage/src/main/java/org/apache/kylin/storage/hbase/RegionScannerAdapter.java
@@ -26,6 +26,7 @@ import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
+import org.apache.hadoop.hbase.regionserver.ScannerContext;
 
 /**
  * @author yangli9
@@ -50,7 +51,7 @@ public class RegionScannerAdapter implements RegionScanner {
     }
 
     @Override
-    public boolean next(List<Cell> result, int limit) throws IOException {
+    public boolean next(List<Cell> result, ScannerContext scannerContext) throws IOException {
         return next(result);
     }
 
@@ -60,11 +61,16 @@ public class RegionScannerAdapter implements RegionScanner {
     }
 
     @Override
-    public boolean nextRaw(List<Cell> result, int limit) throws IOException {
+    public boolean nextRaw(List<Cell> result, ScannerContext scannerContext) throws IOException {
         return next(result);
     }
 
     @Override
+    public int getBatch() {
+        return -1;
+    }
+
+    @Override
     public void close() throws IOException {
         scanner.close();
     }

http://git-wip-us.apache.org/repos/asf/kylin/blob/48cec941/storage/src/main/java/org/apache/kylin/storage/hbase/SerializedHBaseTupleIterator.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/hbase/SerializedHBaseTupleIterator.java b/storage/src/main/java/org/apache/kylin/storage/hbase/SerializedHBaseTupleIterator.java
index d188a44..bbe3397 100644
--- a/storage/src/main/java/org/apache/kylin/storage/hbase/SerializedHBaseTupleIterator.java
+++ b/storage/src/main/java/org/apache/kylin/storage/hbase/SerializedHBaseTupleIterator.java
@@ -24,7 +24,7 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 
-import org.apache.hadoop.hbase.client.HConnection;
+import org.apache.hadoop.hbase.client.Connection;
 import org.apache.kylin.cube.CubeInstance;
 import org.apache.kylin.cube.CubeSegment;
 import org.apache.kylin.cube.kv.RowValueDecoder;
@@ -52,7 +52,7 @@ public class SerializedHBaseTupleIterator implements ITupleIterator {
     private ITupleIterator segmentIterator;
     private int scanCount;
 
-    public SerializedHBaseTupleIterator(HConnection conn, List<HBaseKeyRange> segmentKeyRanges, CubeInstance cube, Collection<TblColRef> dimensions, TupleFilter filter, Collection<TblColRef> groupBy, List<RowValueDecoder> rowValueDecoders, StorageContext context) {
+    public SerializedHBaseTupleIterator(Connection conn, List<HBaseKeyRange> segmentKeyRanges, CubeInstance cube, Collection<TblColRef> dimensions, TupleFilter filter, Collection<TblColRef> groupBy, List<RowValueDecoder> rowValueDecoders, StorageContext context) {
 
         this.context = context;
         int limit = context.getLimit();

http://git-wip-us.apache.org/repos/asf/kylin/blob/48cec941/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/endpoint/EndpointTupleIterator.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/endpoint/EndpointTupleIterator.java b/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/endpoint/EndpointTupleIterator.java
index 8587075..450b1ae 100644
--- a/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/endpoint/EndpointTupleIterator.java
+++ b/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/endpoint/EndpointTupleIterator.java
@@ -26,8 +26,9 @@ import java.util.List;
 import java.util.Map;
 
 import org.apache.commons.io.IOUtils;
-import org.apache.hadoop.hbase.client.HConnection;
-import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.client.coprocessor.Batch;
 import org.apache.hadoop.hbase.ipc.BlockingRpcCallback;
 import org.apache.hadoop.hbase.ipc.ServerRpcController;
@@ -80,14 +81,14 @@ public class EndpointTupleIterator implements ITupleIterator {
 
     Iterator<List<IIProtos.IIResponse.IIRow>> regionResponsesIterator = null;
     ITupleIterator tupleIterator = null;
-    HTableInterface table = null;
+    Table table = null;
 
     int rowsInAllMetric = 0;
 
-    public EndpointTupleIterator(IISegment segment, TupleFilter rootFilter, Collection<TblColRef> groupBy, List<FunctionDesc> measures, StorageContext context, HConnection conn) throws Throwable {
+    public EndpointTupleIterator(IISegment segment, TupleFilter rootFilter, Collection<TblColRef> groupBy, List<FunctionDesc> measures, StorageContext context, Connection conn) throws Throwable {
 
         String tableName = segment.getStorageLocationIdentifier();
-        table = conn.getTable(tableName);
+        table = conn.getTable(TableName.valueOf(tableName));
         factTableName = segment.getIIDesc().getFactTableName();
 
         if (rootFilter == null) {
@@ -213,7 +214,7 @@ public class EndpointTupleIterator implements ITupleIterator {
     }
 
     //TODO : async callback
-    private Iterator<List<IIProtos.IIResponse.IIRow>> getResults(final IIProtos.IIRequest request, HTableInterface table) throws Throwable {
+    private Iterator<List<IIProtos.IIResponse.IIRow>> getResults(final IIProtos.IIRequest request, Table table) throws Throwable {
         Map<byte[], List<IIProtos.IIResponse.IIRow>> results = table.coprocessorService(IIProtos.RowsService.class, null, null, new Batch.Call<IIProtos.RowsService, List<IIProtos.IIResponse.IIRow>>() {
             public List<IIProtos.IIResponse.IIRow> call(IIProtos.RowsService rowsService) throws IOException {
                 ServerRpcController controller = new ServerRpcController();
@@ -236,7 +237,7 @@ public class EndpointTupleIterator implements ITupleIterator {
         int index = 0;
 
         for (int i = 0; i < columns.size(); i++) {
-            TblColRef column = columns.get(i);
+            //            TblColRef column = columns.get(i);
             //            if (!dimensions.contains(column)) {
             //                continue;
             //            }

http://git-wip-us.apache.org/repos/asf/kylin/blob/48cec941/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/endpoint/IIEndpoint.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/endpoint/IIEndpoint.java b/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/endpoint/IIEndpoint.java
index a770f55..adf1bf1 100644
--- a/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/endpoint/IIEndpoint.java
+++ b/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/endpoint/IIEndpoint.java
@@ -90,7 +90,7 @@ public class IIEndpoint extends IIProtos.RowsService implements Coprocessor, Cop
         RegionScanner innerScanner = null;
         HRegion region = null;
         try {
-            region = env.getRegion();
+            region = (HRegion) env.getRegion();
             innerScanner = region.getScanner(buildScan());
             region.startRegionOperation();
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/48cec941/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/observer/AggregateRegionObserver.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/observer/AggregateRegionObserver.java b/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/observer/AggregateRegionObserver.java
index 2cecd5c..c21ee36 100644
--- a/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/observer/AggregateRegionObserver.java
+++ b/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/observer/AggregateRegionObserver.java
@@ -93,7 +93,7 @@ public class AggregateRegionObserver extends BaseRegionObserver {
         // start/end region operation & sync on scanner is suggested by the
         // javadoc of RegionScanner.nextRaw()
         // FIXME: will the lock still work when a iterator is returned? is it safe? Is readonly attribute helping here? by mhb
-        HRegion region = ctxt.getEnvironment().getRegion();
+        HRegion region = (HRegion) ctxt.getEnvironment().getRegion();
         region.startRegionOperation();
         try {
             synchronized (innerScanner) {

http://git-wip-us.apache.org/repos/asf/kylin/blob/48cec941/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/observer/AggregationScanner.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/observer/AggregationScanner.java b/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/observer/AggregationScanner.java
index 8075bc3..eaa7d20 100644
--- a/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/observer/AggregationScanner.java
+++ b/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/observer/AggregationScanner.java
@@ -25,6 +25,7 @@ import java.util.List;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
+import org.apache.hadoop.hbase.regionserver.ScannerContext;
 import org.apache.kylin.measure.MeasureAggregator;
 import org.apache.kylin.storage.hbase.coprocessor.AggrKey;
 import org.apache.kylin.storage.hbase.coprocessor.CoprocessorFilter;
@@ -93,13 +94,18 @@ public class AggregationScanner implements RegionScanner {
     }
 
     @Override
+    public int getBatch() {
+        return outerScanner.getBatch();
+    }
+
+    @Override
     public boolean next(List<Cell> results) throws IOException {
         return outerScanner.next(results);
     }
 
     @Override
-    public boolean next(List<Cell> result, int limit) throws IOException {
-        return outerScanner.next(result, limit);
+    public boolean next(List<Cell> result, ScannerContext scannerContext) throws IOException {
+        return outerScanner.next(result, scannerContext);
     }
 
     @Override
@@ -108,8 +114,8 @@ public class AggregationScanner implements RegionScanner {
     }
 
     @Override
-    public boolean nextRaw(List<Cell> result, int limit) throws IOException {
-        return outerScanner.nextRaw(result, limit);
+    public boolean nextRaw(List<Cell> result, ScannerContext scannerContext) throws IOException {
+        return outerScanner.nextRaw(result, scannerContext);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/kylin/blob/48cec941/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/observer/ObserverAggregationCache.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/observer/ObserverAggregationCache.java b/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/observer/ObserverAggregationCache.java
index f5fb497..b1f642f 100644
--- a/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/observer/ObserverAggregationCache.java
+++ b/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/observer/ObserverAggregationCache.java
@@ -30,6 +30,7 @@ import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValue.Type;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
+import org.apache.hadoop.hbase.regionserver.ScannerContext;
 import org.apache.kylin.measure.MeasureAggregator;
 import org.apache.kylin.storage.hbase.coprocessor.AggrKey;
 import org.apache.kylin.storage.hbase.coprocessor.AggregationCache;
@@ -107,7 +108,7 @@ public class ObserverAggregationCache extends AggregationCache {
         }
 
         @Override
-        public boolean next(List<Cell> result, int limit) throws IOException {
+        public boolean next(List<Cell> result, ScannerContext scannerContext) throws IOException {
             return next(result);
         }
 
@@ -117,11 +118,16 @@ public class ObserverAggregationCache extends AggregationCache {
         }
 
         @Override
-        public boolean nextRaw(List<Cell> result, int limit) throws IOException {
+        public boolean nextRaw(List<Cell> result, ScannerContext scannerContext) throws IOException {
             return next(result);
         }
 
         @Override
+        public int getBatch() {
+            return innerScanner.getBatch();
+        }
+
+        @Override
         public void close() throws IOException {
             // AggregateRegionObserver.LOG.info("Kylin Scanner close()");
             innerScanner.close();

http://git-wip-us.apache.org/repos/asf/kylin/blob/48cec941/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/observer/ObserverEnabler.java
----------------------------------------------------------------------
diff --git a/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/observer/ObserverEnabler.java b/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/observer/ObserverEnabler.java
index 5278326..b941a5e 100644
--- a/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/observer/ObserverEnabler.java
+++ b/storage/src/main/java/org/apache/kylin/storage/hbase/coprocessor/observer/ObserverEnabler.java
@@ -23,9 +23,9 @@ import java.util.Collection;
 import java.util.Map;
 import java.util.Set;
 
-import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.debug.BackdoorToggles;
@@ -58,7 +58,7 @@ public class ObserverEnabler {
     static final Map<String, Boolean> CUBE_OVERRIDES = Maps.newConcurrentMap();
 
     public static ResultScanner scanWithCoprocessorIfBeneficial(CubeSegment segment, Cuboid cuboid, TupleFilter tupleFiler, //
-            Collection<TblColRef> groupBy, Collection<RowValueDecoder> rowValueDecoders, StorageContext context, HTableInterface table, Scan scan) throws IOException {
+            Collection<TblColRef> groupBy, Collection<RowValueDecoder> rowValueDecoders, StorageContext context, Table table, Scan scan) throws IOException {
 
         if (context.isCoprocessorEnabled() == false) {
             return table.getScanner(scan);

http://git-wip-us.apache.org/repos/asf/kylin/blob/48cec941/storage/src/test/java/org/apache/kylin/storage/hbase/InvertedIndexHBaseTest.java
----------------------------------------------------------------------
diff --git a/storage/src/test/java/org/apache/kylin/storage/hbase/InvertedIndexHBaseTest.java b/storage/src/test/java/org/apache/kylin/storage/hbase/InvertedIndexHBaseTest.java
index f7fcef1..50069a1 100644
--- a/storage/src/test/java/org/apache/kylin/storage/hbase/InvertedIndexHBaseTest.java
+++ b/storage/src/test/java/org/apache/kylin/storage/hbase/InvertedIndexHBaseTest.java
@@ -1,115 +1,112 @@
-/*
- * 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.storage.hbase;
-
-import java.util.List;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.client.HConnection;
-import org.apache.hadoop.hbase.client.HConnectionManager;
-import org.apache.kylin.common.util.BytesUtil;
-import org.apache.kylin.common.util.HBaseMetadataTestCase;
-import org.apache.kylin.common.util.HadoopUtil;
-import org.apache.kylin.invertedindex.IIInstance;
-import org.apache.kylin.invertedindex.IIManager;
-import org.apache.kylin.invertedindex.IISegment;
-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.IIDesc;
-import org.apache.kylin.invertedindex.model.IIKeyValueCodec;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-import com.google.common.collect.Lists;
-
-/**
- * @author yangli9
- */
-public class InvertedIndexHBaseTest extends HBaseMetadataTestCase {
-
-    IIInstance ii;
-    IISegment seg;
-    HConnection hconn;
-
-    TableRecordInfo info;
-
-    @Before
-    public void setup() throws Exception {
-        this.createTestMetadata();
-
-        this.ii = IIManager.getInstance(getTestConfig()).getII("test_kylin_ii");
-        this.seg = ii.getFirstSegment();
-
-        Configuration hconf = HadoopUtil.getCurrentHBaseConfiguration();
-        hconn = HConnectionManager.createConnection(hconf);
-
-        this.info = new TableRecordInfo(seg);
-    }
-
-    @After
-    public void after() throws Exception {
-        this.cleanupTestMetadata();
-    }
-
-    @Test
-    public void testLoad() throws Exception {
-
-        String tableName = seg.getStorageLocationIdentifier();
-        IIKeyValueCodec codec = new IIKeyValueCodec(info.getDigest());
-
-        List<Slice> slices = Lists.newArrayList();
-        HBaseClientKVIterator kvIterator = new HBaseClientKVIterator(hconn, tableName, IIDesc.HBASE_FAMILY_BYTES, IIDesc.HBASE_QUALIFIER_BYTES);
-        try {
-            for (Slice slice : codec.decodeKeyValue(kvIterator)) {
-                slices.add(slice);
-            }
-        } finally {
-            kvIterator.close();
-        }
-
-        List<TableRecord> records = iterateRecords(slices);
-        dump(records);
-        System.out.println(records.size() + " records");
-    }
-
-    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());
-
-            byte[] x = rec.getBytes();
-            String y = BytesUtil.toReadableText(x);
-            System.out.println(y);
-            System.out.println();
-        }
-    }
-
-}
+/*
+ * 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.storage.hbase;
+
+import java.util.List;
+
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.kylin.common.persistence.HBaseConnection;
+import org.apache.kylin.common.util.BytesUtil;
+import org.apache.kylin.common.util.HBaseMetadataTestCase;
+import org.apache.kylin.invertedindex.IIInstance;
+import org.apache.kylin.invertedindex.IIManager;
+import org.apache.kylin.invertedindex.IISegment;
+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.IIDesc;
+import org.apache.kylin.invertedindex.model.IIKeyValueCodec;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import com.google.common.collect.Lists;
+
+/**
+ * @author yangli9
+ */
+public class InvertedIndexHBaseTest extends HBaseMetadataTestCase {
+
+    IIInstance ii;
+    IISegment seg;
+    Connection hconn;
+
+    TableRecordInfo info;
+
+    @Before
+    public void setup() throws Exception {
+        this.createTestMetadata();
+
+        this.ii = IIManager.getInstance(getTestConfig()).getII("test_kylin_ii");
+        this.seg = ii.getFirstSegment();
+
+        this.hconn = HBaseConnection.get();
+
+        this.info = new TableRecordInfo(seg);
+    }
+
+    @After
+    public void after() throws Exception {
+        this.cleanupTestMetadata();
+    }
+
+    @Test
+    public void testLoad() throws Exception {
+
+        String tableName = seg.getStorageLocationIdentifier();
+        IIKeyValueCodec codec = new IIKeyValueCodec(info.getDigest());
+
+        List<Slice> slices = Lists.newArrayList();
+        HBaseClientKVIterator kvIterator = new HBaseClientKVIterator(hconn, tableName, IIDesc.HBASE_FAMILY_BYTES, IIDesc.HBASE_QUALIFIER_BYTES);
+        try {
+            for (Slice slice : codec.decodeKeyValue(kvIterator)) {
+                slices.add(slice);
+            }
+        } finally {
+            kvIterator.close();
+        }
+
+        List<TableRecord> records = iterateRecords(slices);
+        dump(records);
+        System.out.println(records.size() + " records");
+    }
+
+    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());
+
+            byte[] x = rec.getBytes();
+            String y = BytesUtil.toReadableText(x);
+            System.out.println(y);
+            System.out.println();
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/48cec941/storage/src/test/java/org/apache/kylin/storage/hbase/coprocessor/observer/AggregateRegionObserverTest.java
----------------------------------------------------------------------
diff --git a/storage/src/test/java/org/apache/kylin/storage/hbase/coprocessor/observer/AggregateRegionObserverTest.java b/storage/src/test/java/org/apache/kylin/storage/hbase/coprocessor/observer/AggregateRegionObserverTest.java
index 0454b4c..3ace91e 100644
--- a/storage/src/test/java/org/apache/kylin/storage/hbase/coprocessor/observer/AggregateRegionObserverTest.java
+++ b/storage/src/test/java/org/apache/kylin/storage/hbase/coprocessor/observer/AggregateRegionObserverTest.java
@@ -35,6 +35,7 @@ import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValue.Type;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
+import org.apache.hadoop.hbase.regionserver.ScannerContext;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.kylin.common.util.Bytes;
 import org.apache.kylin.cube.kv.RowConstants;
@@ -223,102 +224,46 @@ public class AggregateRegionObserverTest {
             this.input = cellInputs;
         }
 
-        /*
-         * (non-Javadoc)
-         * 
-         * @see
-         * org.apache.hadoop.hbase.regionserver.InternalScanner#next(java.util
-         * .List)
-         */
         @Override
         public boolean next(List<Cell> results) throws IOException {
             return nextRaw(results);
         }
 
-        /*
-         * (non-Javadoc)
-         * 
-         * @see
-         * org.apache.hadoop.hbase.regionserver.InternalScanner#next(java.util
-         * .List, int)
-         */
         @Override
-        public boolean next(List<Cell> result, int limit) throws IOException {
+        public boolean next(List<Cell> result, ScannerContext scannerContext) throws IOException {
             return next(result);
         }
 
-        /*
-         * (non-Javadoc)
-         * 
-         * @see org.apache.hadoop.hbase.regionserver.InternalScanner#close()
-         */
         @Override
         public void close() throws IOException {
 
         }
 
-        /*
-         * (non-Javadoc)
-         * 
-         * @see
-         * org.apache.hadoop.hbase.regionserver.RegionScanner#getRegionInfo()
-         */
         @Override
         public HRegionInfo getRegionInfo() {
             return null;
         }
 
-        /*
-         * (non-Javadoc)
-         * 
-         * @see
-         * org.apache.hadoop.hbase.regionserver.RegionScanner#isFilterDone()
-         */
         @Override
         public boolean isFilterDone() throws IOException {
             return false;
         }
 
-        /*
-         * (non-Javadoc)
-         * 
-         * @see
-         * org.apache.hadoop.hbase.regionserver.RegionScanner#reseek(byte[])
-         */
         @Override
         public boolean reseek(byte[] row) throws IOException {
             return false;
         }
 
-        /*
-         * (non-Javadoc)
-         * 
-         * @see
-         * org.apache.hadoop.hbase.regionserver.RegionScanner#getMaxResultSize()
-         */
         @Override
         public long getMaxResultSize() {
             return 0;
         }
 
-        /*
-         * (non-Javadoc)
-         * 
-         * @see
-         * org.apache.hadoop.hbase.regionserver.RegionScanner#getMvccReadPoint()
-         */
         @Override
         public long getMvccReadPoint() {
             return 0;
         }
 
-        /*
-         * (non-Javadoc)
-         * 
-         * @see
-         * org.apache.hadoop.hbase.regionserver.RegionScanner#nextRaw(java.util
-         * .List)
-         */
         @Override
         public boolean nextRaw(List<Cell> result) throws IOException {
             if (i < input.size()) {
@@ -328,18 +273,15 @@ public class AggregateRegionObserverTest {
             return i < input.size();
         }
 
-        /*
-         * (non-Javadoc)
-         * 
-         * @see
-         * org.apache.hadoop.hbase.regionserver.RegionScanner#nextRaw(java.util
-         * .List, int)
-         */
         @Override
-        public boolean nextRaw(List<Cell> result, int limit) throws IOException {
+        public boolean nextRaw(List<Cell> result, ScannerContext scannerContext) throws IOException {
             return nextRaw(result);
         }
 
+        @Override
+        public int getBatch() {
+            return -1;
+        }
     }
 
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/48cec941/storage/src/test/java/org/apache/kylin/storage/minicluster/HiveMiniClusterTest.java
----------------------------------------------------------------------
diff --git a/storage/src/test/java/org/apache/kylin/storage/minicluster/HiveMiniClusterTest.java b/storage/src/test/java/org/apache/kylin/storage/minicluster/HiveMiniClusterTest.java
index d17cfa6..b1f6626 100644
--- a/storage/src/test/java/org/apache/kylin/storage/minicluster/HiveMiniClusterTest.java
+++ b/storage/src/test/java/org/apache/kylin/storage/minicluster/HiveMiniClusterTest.java
@@ -27,7 +27,6 @@ import java.sql.SQLException;
 import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.service.HiveInterface;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.MiniMRCluster;
 
@@ -47,7 +46,7 @@ public class HiveMiniClusterTest extends HiveJDBCClientTest {
     public static final File HIVE_WAREHOUSE_DIR = new File(HIVE_BASE_DIR + "/warehouse");
     public static final File HIVE_TESTDATA_DIR = new File(HIVE_BASE_DIR + "/testdata");
     public static final File HIVE_HADOOP_TMP_DIR = new File(HIVE_BASE_DIR + "/hadooptmp");
-    protected HiveInterface client;
+    //protected HiveInterface client;
 
     protected MiniDFSCluster miniDFS;
     protected MiniMRCluster miniMR;


[04/52] [abbrv] kylin git commit: KYLIN-1343 Query IT pass

Posted by li...@apache.org.
KYLIN-1343 Query IT pass


Project: http://git-wip-us.apache.org/repos/asf/kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/d0f610fc
Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/d0f610fc
Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/d0f610fc

Branch: refs/heads/1.x-HBase1.1.3
Commit: d0f610fc9ec9962fd206a9d47b6f712688d2db11
Parents: df7ae17
Author: Yang Li <li...@apache.org>
Authored: Sun Jan 24 08:20:42 2016 +0800
Committer: Yang Li <li...@apache.org>
Committed: Sun Jan 24 08:20:42 2016 +0800

----------------------------------------------------------------------
 .../apache/calcite/sql2rel/RelFieldTrimmer.java | 44 ++++++++++++++++++++
 .../calcite/sql2rel/SqlToRelConverter.java      | 16 +++----
 .../kylin/query/relnode/OLAPAggregateRel.java   |  6 ++-
 .../apache/kylin/query/relnode/OLAPJoinRel.java |  2 +-
 .../kylin/query/relnode/OLAPTableScan.java      |  4 ++
 .../apache/kylin/query/schema/OLAPTable.java    | 17 ++++----
 query/src/test/resources/logging.properties     |  2 +-
 query/src/test/resources/query/sql/query86.sql  | 30 +++++++++++++
 .../resources/query/sql/query86.sql.disabled    | 30 -------------
 query/src/test/resources/query/sql/query87.sql  | 30 +++++++++++++
 .../resources/query/sql/query87.sql.disabled    | 30 -------------
 query/src/test/resources/query/sql/query88.sql  | 30 +++++++++++++
 .../resources/query/sql/query88.sql.disabled    | 30 -------------
 query/src/test/resources/query/sql/query89.sql  | 30 +++++++++++++
 .../resources/query/sql/query89.sql.disabled    | 30 -------------
 15 files changed, 193 insertions(+), 138 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/d0f610fc/atopcalcite/src/main/java/org/apache/calcite/sql2rel/RelFieldTrimmer.java
----------------------------------------------------------------------
diff --git a/atopcalcite/src/main/java/org/apache/calcite/sql2rel/RelFieldTrimmer.java b/atopcalcite/src/main/java/org/apache/calcite/sql2rel/RelFieldTrimmer.java
new file mode 100644
index 0000000..f88157c
--- /dev/null
+++ b/atopcalcite/src/main/java/org/apache/calcite/sql2rel/RelFieldTrimmer.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.calcite.sql2rel;
+
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.RelFactories;
+import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.tools.RelBuilder;
+
+/*
+ * OVERRIDE POINT:
+ * - disable the whole RelFieldTrimmer
+ */
+
+public class RelFieldTrimmer {
+
+    public RelFieldTrimmer(SqlValidator validator, RelBuilder relBuilder) {
+    }
+
+    public RelFieldTrimmer(SqlValidator validator, RelOptCluster cluster, RelFactories.ProjectFactory projectFactory, RelFactories.FilterFactory filterFactory, RelFactories.JoinFactory joinFactory, RelFactories.SemiJoinFactory semiJoinFactory, RelFactories.SortFactory sortFactory, RelFactories.AggregateFactory aggregateFactory, RelFactories.SetOpFactory setOpFactory) {
+    }
+
+    public RelNode trim(RelNode rootRel) {
+        return rootRel;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/d0f610fc/atopcalcite/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
----------------------------------------------------------------------
diff --git a/atopcalcite/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java b/atopcalcite/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
index 76203e8..aed7c27 100644
--- a/atopcalcite/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
+++ b/atopcalcite/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
@@ -1,12 +1,13 @@
 /*
- * 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
+ * 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
+ *     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,
@@ -14,6 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.calcite.sql2rel;
 
 import org.apache.calcite.avatica.util.Spaces;

http://git-wip-us.apache.org/repos/asf/kylin/blob/d0f610fc/query/src/main/java/org/apache/kylin/query/relnode/OLAPAggregateRel.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/org/apache/kylin/query/relnode/OLAPAggregateRel.java b/query/src/main/java/org/apache/kylin/query/relnode/OLAPAggregateRel.java
index 3d1ddb3..26b987a 100644
--- a/query/src/main/java/org/apache/kylin/query/relnode/OLAPAggregateRel.java
+++ b/query/src/main/java/org/apache/kylin/query/relnode/OLAPAggregateRel.java
@@ -43,6 +43,7 @@ import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rel.type.RelDataTypeFactory.FieldInfoBuilder;
 import org.apache.calcite.schema.AggregateFunction;
 import org.apache.calcite.schema.FunctionParameter;
 import org.apache.calcite.schema.impl.AggregateFunctionImpl;
@@ -63,6 +64,7 @@ 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 org.apache.kylin.query.schema.OLAPTable;
 
 import com.google.common.base.Preconditions;
 
@@ -281,6 +283,7 @@ public class OLAPAggregateRel extends Aggregate implements OLAPRel {
         fillbackOptimizedColumn();
 
         ColumnRowType inputColumnRowType = ((OLAPRel) getInput()).getColumnRowType();
+        RelDataTypeFactory typeFactory = getCluster().getTypeFactory();
         for (int i = 0; i < this.aggregations.size(); i++) {
             FunctionDesc aggFunc = this.aggregations.get(i);
 
@@ -291,7 +294,8 @@ public class OLAPAggregateRel extends Aggregate implements OLAPRel {
 
             if (aggFunc.needRewrite()) {
                 String rewriteFieldName = aggFunc.getRewriteFieldName();
-                this.context.rewriteFields.put(rewriteFieldName, null);
+                RelDataType rewriteFieldType = OLAPTable.createSqlType(typeFactory, aggFunc.getRewriteFieldType(), true);
+                this.context.rewriteFields.put(rewriteFieldName, rewriteFieldType);
 
                 TblColRef column = buildRewriteColumn(aggFunc);
                 this.context.metricsColumns.add(column);

http://git-wip-us.apache.org/repos/asf/kylin/blob/d0f610fc/query/src/main/java/org/apache/kylin/query/relnode/OLAPJoinRel.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/org/apache/kylin/query/relnode/OLAPJoinRel.java b/query/src/main/java/org/apache/kylin/query/relnode/OLAPJoinRel.java
index c079357..623a43e 100644
--- a/query/src/main/java/org/apache/kylin/query/relnode/OLAPJoinRel.java
+++ b/query/src/main/java/org/apache/kylin/query/relnode/OLAPJoinRel.java
@@ -281,7 +281,7 @@ public class OLAPJoinRel extends EnumerableJoin implements OLAPRel {
         implementor.visitChild(this, this.right);
 
         this.rowType = this.deriveRowType();
-
+        
         if (this.isTopJoin && RewriteImplementor.needRewrite(this.context)) {
             // find missed rewrite fields
             int paramIndex = this.rowType.getFieldList().size();

http://git-wip-us.apache.org/repos/asf/kylin/blob/d0f610fc/query/src/main/java/org/apache/kylin/query/relnode/OLAPTableScan.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/org/apache/kylin/query/relnode/OLAPTableScan.java b/query/src/main/java/org/apache/kylin/query/relnode/OLAPTableScan.java
index 48cf662..45d00b3 100644
--- a/query/src/main/java/org/apache/kylin/query/relnode/OLAPTableScan.java
+++ b/query/src/main/java/org/apache/kylin/query/relnode/OLAPTableScan.java
@@ -47,8 +47,10 @@ import org.apache.calcite.rel.rules.AggregateProjectMergeRule;
 import org.apache.calcite.rel.rules.FilterJoinRule;
 import org.apache.calcite.rel.rules.FilterProjectTransposeRule;
 import org.apache.calcite.rel.rules.JoinCommuteRule;
+import org.apache.calcite.rel.rules.JoinPushExpressionsRule;
 import org.apache.calcite.rel.rules.JoinPushThroughJoinRule;
 import org.apache.calcite.rel.rules.ReduceExpressionsRule;
+import org.apache.calcite.rel.rules.SortJoinTransposeRule;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.rel.type.RelDataTypeField;
@@ -148,6 +150,8 @@ public class OLAPTableScan extends TableScan implements OLAPRel, EnumerableRel {
         planner.removeRule(AggregateJoinTransposeRule.INSTANCE);
         planner.removeRule(AggregateProjectMergeRule.INSTANCE);
         planner.removeRule(FilterProjectTransposeRule.INSTANCE);
+        planner.removeRule(SortJoinTransposeRule.INSTANCE);
+        planner.removeRule(JoinPushExpressionsRule.INSTANCE);
         // distinct count will be split into a separated query that is joined with the left query
         planner.removeRule(AggregateExpandDistinctAggregatesRule.INSTANCE);
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/d0f610fc/query/src/main/java/org/apache/kylin/query/schema/OLAPTable.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/org/apache/kylin/query/schema/OLAPTable.java b/query/src/main/java/org/apache/kylin/query/schema/OLAPTable.java
index 8b1ad29..fad33ed 100644
--- a/query/src/main/java/org/apache/kylin/query/schema/OLAPTable.java
+++ b/query/src/main/java/org/apache/kylin/query/schema/OLAPTable.java
@@ -43,6 +43,7 @@ import org.apache.calcite.schema.impl.AbstractTableQueryable;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.sql.type.SqlTypeUtil;
 import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.kylin.metadata.datatype.DataType;
 import org.apache.kylin.metadata.model.ColumnDesc;
 import org.apache.kylin.metadata.model.FunctionDesc;
 import org.apache.kylin.metadata.model.MeasureDesc;
@@ -128,20 +129,20 @@ public class OLAPTable extends AbstractQueryableTable implements TranslatableTab
     private RelDataType deriveRowType(RelDataTypeFactory typeFactory) {
         RelDataTypeFactory.FieldInfoBuilder fieldInfo = typeFactory.builder();
         for (ColumnDesc column : exposedColumns) {
-            RelDataType sqlType = createSqlType(typeFactory, column);
+            RelDataType sqlType = createSqlType(typeFactory, column.getType(), column.isNullable());
             sqlType = SqlTypeUtil.addCharsetAndCollation(sqlType, typeFactory);
             fieldInfo.add(column.getName(), sqlType);
         }
         return typeFactory.createStructType(fieldInfo);
     }
 
-    private RelDataType createSqlType(RelDataTypeFactory typeFactory, ColumnDesc column) {
-        SqlTypeName sqlTypeName = SQLTYPE_MAPPING.get(column.getTypeName());
+    public static RelDataType createSqlType(RelDataTypeFactory typeFactory, DataType dataType, boolean isNullable) {
+        SqlTypeName sqlTypeName = SQLTYPE_MAPPING.get(dataType.getName());
         if (sqlTypeName == null)
-            throw new IllegalArgumentException("Unrecognized column type " + column.getTypeName() + " from " + column);
+            throw new IllegalArgumentException("Unrecognized data type " + dataType);
 
-        int precision = column.getTypePrecision();
-        int scale = column.getTypeScale();
+        int precision = dataType.getPrecision();
+        int scale = dataType.getScale();
 
         RelDataType result;
         if (precision >= 0 && scale >= 0)
@@ -152,7 +153,7 @@ public class OLAPTable extends AbstractQueryableTable implements TranslatableTab
             result = typeFactory.createSqlType(sqlTypeName);
 
         // due to left join and uncertain data quality, dimension value can be null
-        if (column.isNullable()) {
+        if (isNullable) {
             result = typeFactory.createTypeWithNullability(result, true);
         } else {
             result = typeFactory.createTypeWithNullability(result, false);
@@ -160,7 +161,7 @@ public class OLAPTable extends AbstractQueryableTable implements TranslatableTab
 
         return result;
     }
-
+    
     private List<ColumnDesc> listSourceColumns() {
         ProjectManager mgr = ProjectManager.getInstance(olapSchema.getConfig());
         List<ColumnDesc> tableColumns = Lists.newArrayList(mgr.listExposedColumns(olapSchema.getProjectName(), sourceTable.getIdentity()));

http://git-wip-us.apache.org/repos/asf/kylin/blob/d0f610fc/query/src/test/resources/logging.properties
----------------------------------------------------------------------
diff --git a/query/src/test/resources/logging.properties b/query/src/test/resources/logging.properties
index 5767d05..4edb856 100644
--- a/query/src/test/resources/logging.properties
+++ b/query/src/test/resources/logging.properties
@@ -17,6 +17,6 @@
 
 handlers=java.util.logging.ConsoleHandler
 .level=INFO
-#org.eigenbase.relopt.RelOptPlanner.level=FINEST
+#org.apache.calcite.plan.RelOptPlanner.level=FINEST
 java.util.logging.ConsoleHandler.level=ALL
 java.util.logging.ConsoleHandler.formatter=org.apache.kylin.common.util.MyLogFormatter
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kylin/blob/d0f610fc/query/src/test/resources/query/sql/query86.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query86.sql b/query/src/test/resources/query/sql/query86.sql
new file mode 100644
index 0000000..e551a45
--- /dev/null
+++ b/query/src/test/resources/query/sql/query86.sql
@@ -0,0 +1,30 @@
+--
+-- 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.
+--
+
+select meta_categ_name, count(1) as cnt, sum(price) as GMV 
+
+ from test_kylin_fact 
+ left JOIN edw.test_cal_dt as test_cal_dt
+ ON test_kylin_fact.cal_dt = test_cal_dt.cal_dt
+ left JOIN test_category_groupings
+ ON test_kylin_fact.leaf_categ_id = test_category_groupings.leaf_categ_id AND test_kylin_fact.lstg_site_id = test_category_groupings.site_id
+ left JOIN edw.test_sites as test_sites
+ ON test_kylin_fact.lstg_site_id = test_sites.site_id
+
+ where meta_categ_name not in ('', 'a')
+ group by meta_categ_name 

http://git-wip-us.apache.org/repos/asf/kylin/blob/d0f610fc/query/src/test/resources/query/sql/query86.sql.disabled
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query86.sql.disabled b/query/src/test/resources/query/sql/query86.sql.disabled
deleted file mode 100644
index e551a45..0000000
--- a/query/src/test/resources/query/sql/query86.sql.disabled
+++ /dev/null
@@ -1,30 +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.
---
-
-select meta_categ_name, count(1) as cnt, sum(price) as GMV 
-
- from test_kylin_fact 
- left JOIN edw.test_cal_dt as test_cal_dt
- ON test_kylin_fact.cal_dt = test_cal_dt.cal_dt
- left JOIN test_category_groupings
- ON test_kylin_fact.leaf_categ_id = test_category_groupings.leaf_categ_id AND test_kylin_fact.lstg_site_id = test_category_groupings.site_id
- left JOIN edw.test_sites as test_sites
- ON test_kylin_fact.lstg_site_id = test_sites.site_id
-
- where meta_categ_name not in ('', 'a')
- group by meta_categ_name 

http://git-wip-us.apache.org/repos/asf/kylin/blob/d0f610fc/query/src/test/resources/query/sql/query87.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query87.sql b/query/src/test/resources/query/sql/query87.sql
new file mode 100644
index 0000000..cc6dca5
--- /dev/null
+++ b/query/src/test/resources/query/sql/query87.sql
@@ -0,0 +1,30 @@
+--
+-- 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.
+--
+
+select meta_categ_name, count(1) as cnt, sum(price) as GMV 
+
+ from test_kylin_fact 
+ left JOIN edw.test_cal_dt as test_cal_dt
+ ON test_kylin_fact.cal_dt = test_cal_dt.cal_dt
+ left JOIN test_category_groupings
+ ON test_kylin_fact.leaf_categ_id = test_category_groupings.leaf_categ_id AND test_kylin_fact.lstg_site_id = test_category_groupings.site_id
+ left JOIN edw.test_sites as test_sites
+ ON test_kylin_fact.lstg_site_id = test_sites.site_id
+
+ where meta_categ_name is not null
+ group by meta_categ_name 

http://git-wip-us.apache.org/repos/asf/kylin/blob/d0f610fc/query/src/test/resources/query/sql/query87.sql.disabled
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query87.sql.disabled b/query/src/test/resources/query/sql/query87.sql.disabled
deleted file mode 100644
index cc6dca5..0000000
--- a/query/src/test/resources/query/sql/query87.sql.disabled
+++ /dev/null
@@ -1,30 +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.
---
-
-select meta_categ_name, count(1) as cnt, sum(price) as GMV 
-
- from test_kylin_fact 
- left JOIN edw.test_cal_dt as test_cal_dt
- ON test_kylin_fact.cal_dt = test_cal_dt.cal_dt
- left JOIN test_category_groupings
- ON test_kylin_fact.leaf_categ_id = test_category_groupings.leaf_categ_id AND test_kylin_fact.lstg_site_id = test_category_groupings.site_id
- left JOIN edw.test_sites as test_sites
- ON test_kylin_fact.lstg_site_id = test_sites.site_id
-
- where meta_categ_name is not null
- group by meta_categ_name 

http://git-wip-us.apache.org/repos/asf/kylin/blob/d0f610fc/query/src/test/resources/query/sql/query88.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query88.sql b/query/src/test/resources/query/sql/query88.sql
new file mode 100644
index 0000000..c7899fd
--- /dev/null
+++ b/query/src/test/resources/query/sql/query88.sql
@@ -0,0 +1,30 @@
+--
+-- 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.
+--
+
+select meta_categ_name, count(1) as cnt, sum(price) as GMV 
+
+ from test_kylin_fact 
+ left JOIN edw.test_cal_dt as test_cal_dt
+ ON test_kylin_fact.cal_dt = test_cal_dt.cal_dt
+ left JOIN test_category_groupings
+ ON test_kylin_fact.leaf_categ_id = test_category_groupings.leaf_categ_id AND test_kylin_fact.lstg_site_id = test_category_groupings.site_id
+ left JOIN edw.test_sites as test_sites
+ ON test_kylin_fact.lstg_site_id = test_sites.site_id
+
+ where meta_categ_name not in ('Unknown')
+ group by meta_categ_name 

http://git-wip-us.apache.org/repos/asf/kylin/blob/d0f610fc/query/src/test/resources/query/sql/query88.sql.disabled
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query88.sql.disabled b/query/src/test/resources/query/sql/query88.sql.disabled
deleted file mode 100644
index c7899fd..0000000
--- a/query/src/test/resources/query/sql/query88.sql.disabled
+++ /dev/null
@@ -1,30 +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.
---
-
-select meta_categ_name, count(1) as cnt, sum(price) as GMV 
-
- from test_kylin_fact 
- left JOIN edw.test_cal_dt as test_cal_dt
- ON test_kylin_fact.cal_dt = test_cal_dt.cal_dt
- left JOIN test_category_groupings
- ON test_kylin_fact.leaf_categ_id = test_category_groupings.leaf_categ_id AND test_kylin_fact.lstg_site_id = test_category_groupings.site_id
- left JOIN edw.test_sites as test_sites
- ON test_kylin_fact.lstg_site_id = test_sites.site_id
-
- where meta_categ_name not in ('Unknown')
- group by meta_categ_name 

http://git-wip-us.apache.org/repos/asf/kylin/blob/d0f610fc/query/src/test/resources/query/sql/query89.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query89.sql b/query/src/test/resources/query/sql/query89.sql
new file mode 100644
index 0000000..578b93f
--- /dev/null
+++ b/query/src/test/resources/query/sql/query89.sql
@@ -0,0 +1,30 @@
+--
+-- 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.
+--
+
+select meta_categ_name, count(1) as cnt, sum(price) as GMV 
+
+ from test_kylin_fact 
+ left JOIN edw.test_cal_dt as test_cal_dt
+ ON test_kylin_fact.cal_dt = test_cal_dt.cal_dt
+ left JOIN test_category_groupings
+ ON test_kylin_fact.leaf_categ_id = test_category_groupings.leaf_categ_id AND test_kylin_fact.lstg_site_id = test_category_groupings.site_id
+ left JOIN edw.test_sites as test_sites
+ ON test_kylin_fact.lstg_site_id = test_sites.site_id
+
+ where meta_categ_name not in ('Unknown', 'ToyHobbies', '', 'a', 'BookMagazines')
+ group by meta_categ_name 

http://git-wip-us.apache.org/repos/asf/kylin/blob/d0f610fc/query/src/test/resources/query/sql/query89.sql.disabled
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql/query89.sql.disabled b/query/src/test/resources/query/sql/query89.sql.disabled
deleted file mode 100644
index 578b93f..0000000
--- a/query/src/test/resources/query/sql/query89.sql.disabled
+++ /dev/null
@@ -1,30 +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.
---
-
-select meta_categ_name, count(1) as cnt, sum(price) as GMV 
-
- from test_kylin_fact 
- left JOIN edw.test_cal_dt as test_cal_dt
- ON test_kylin_fact.cal_dt = test_cal_dt.cal_dt
- left JOIN test_category_groupings
- ON test_kylin_fact.leaf_categ_id = test_category_groupings.leaf_categ_id AND test_kylin_fact.lstg_site_id = test_category_groupings.site_id
- left JOIN edw.test_sites as test_sites
- ON test_kylin_fact.lstg_site_id = test_sites.site_id
-
- where meta_categ_name not in ('Unknown', 'ToyHobbies', '', 'a', 'BookMagazines')
- group by meta_categ_name 


[33/52] [abbrv] kylin git commit: KYLIN-1416 remove website from coding branch

Posted by li...@apache.org.
http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/assets/fonts/helveticaneue_lt.svg
----------------------------------------------------------------------
diff --git a/website/assets/fonts/helveticaneue_lt.svg b/website/assets/fonts/helveticaneue_lt.svg
deleted file mode 100644
index 112404c..0000000
--- a/website/assets/fonts/helveticaneue_lt.svg
+++ /dev/null
@@ -1,573 +0,0 @@
-<?xml version="1.0" standalone="no"?>
-<!DOCTYPE svg PUBLIC "-//W3C//DTD SVG 1.1//EN" "http://www.w3.org/Graphics/SVG/1.1/DTD/svg11.dtd" >
-<svg>
-<metadata>
-Created by FontForge 20110222 at Tue Mar 25 18:21:03 2014
- By Orthosie Webhosting
-Copyright (c) 1988, 1990 Adobe Systems Incorporated.  All Rights Reserved.Helvetica is a registered trademark of Linotype AG and/or its subsidiaries.
-</metadata>
-<defs>
-<font id="HelveticaNeueLight" horiz-adv-x="556" >
-  <font-face 
-    font-family="HelveticaNeueLight"
-    font-weight="400"
-    font-stretch="normal"
-    units-per-em="1000"
-    panose-1="0 0 4 0 0 0 0 0 0 0"
-    ascent="800"
-    descent="-200"
-    x-height="516"
-    cap-height="714"
-    bbox="-166 -211 1050 967"
-    underline-thickness="50"
-    underline-position="-75"
-    unicode-range="U+0020-F002"
-  />
-<missing-glyph horiz-adv-x="500" 
-d="M63 0v797h375v-797h-375zM125 63h250v672h-250v-672z" />
-    <glyph glyph-name=".notdef" horiz-adv-x="500" 
-d="M63 0v797h375v-797h-375zM125 63h250v672h-250v-672z" />
-    <glyph glyph-name=".null" horiz-adv-x="0" 
- />
-    <glyph glyph-name="nonmarkingreturn" horiz-adv-x="278" 
- />
-    <glyph glyph-name="space" unicode=" " horiz-adv-x="278" 
- />
-    <glyph glyph-name="exclam" unicode="!" horiz-adv-x="241" 
-d="M102 178l-15 323v213h68v-213l-15 -323h-38zM163 0h-84v106h84v-106z" />
-    <glyph glyph-name="quotedbl" unicode="&#x22;" horiz-adv-x="370" 
-d="M87 471v243h63v-243h-63zM220 471v243h63v-243h-63z" />
-    <glyph glyph-name="numbersign" unicode="#" 
-d="M497 268v-45h-111l-31 -223h-50l31 223h-141l-31 -223h-50l31 223h-110v45h116l22 156h-110v45h116l31 222h50l-31 -222h141l31 222h50l-31 -222h105v-45h-111l-22 -156h105zM364 424h-141l-22 -156h141z" />
-    <glyph glyph-name="dollar" unicode="$" 
-d="M296 326v-288q70 0 116 34q52 39 52 112q0 63 -55 100q-36 23 -113 42zM257 397v279q-65 0 -103 -35q-44 -39 -44 -114q0 -62 45 -93q29 -19 102 -37zM29 228h63q1 -171 165 -190v296q-103 23 -147 57q-63 48 -63 143q0 88 62 143q59 52 148 52v78h39v-78q90 0 151 -59
-q63 -62 61 -158h-63q-1 74 -42 119t-107 45v-288q107 -23 158 -58q73 -51 73 -145q0 -93 -72 -149q-65 -52 -159 -51v-85h-39v86q-116 11 -169.5 69t-58.5 173z" />
-    <glyph glyph-name="percent" unicode="%" horiz-adv-x="889" 
-d="M228 661q-57 0 -84 -48q-22 -38 -22 -97q0 -58 22 -95q27 -48 84 -48t85 48q21 37 21 95q0 59 -21 97q-28 48 -85 48zM190 -34l453 759h47l-452 -759h-48zM667 363q81 0 122 -54q37 -49 37 -135t-37 -135q-41 -54 -122 -54t-122 54q-37 49 -37 135t37 135q41 54 122 54z
-M667 318q-57 0 -84 -48q-22 -38 -22 -97q0 -58 22 -95q27 -48 84 -48t85 48q21 37 21 95q0 59 -21 97q-28 48 -85 48zM228 706q81 0 122 -54q37 -49 37 -135t-37 -135q-41 -54 -122 -54t-122 54q-37 49 -37 135t37 135q41 54 122 54z" />
-    <glyph glyph-name="ampersand" unicode="&#x26;" horiz-adv-x="611" 
-d="M426 140l-178 216q-146 -79 -146 -177q0 -61 46 -101t109 -40q97 0 169 102zM541 0l-76 93q-86 -108 -210 -108q-95 0 -154 52q-62 54 -62 148q0 118 175 212q-51 66 -59 79q-29 48 -29 93q0 64 45.5 104.5t112.5 40.5t112.5 -40.5t45.5 -104.5q0 -60 -43 -109
-q-30 -34 -97 -76l156 -188q21 43 21 120h63q0 -93 -42 -171l120 -145h-79zM284 661q-41 0 -68 -24.5t-27 -67.5q0 -46 80 -144q110 68 110 144q0 43 -27 67.5t-68 24.5z" />
-    <glyph glyph-name="quotesingle" unicode="'" horiz-adv-x="278" 
-d="M108 471v243h63v-243h-63z" />
-    <glyph glyph-name="parenleft" unicode="(" horiz-adv-x="241" 
-d="M242 -191h-50q-141 233 -141 457q0 244 141 463h50q-128 -204 -128 -486q0 -107 36 -225q34 -114 92 -209z" />
-    <glyph glyph-name="parenright" unicode=")" horiz-adv-x="241" 
-d="M-1 729h50q141 -233 141 -457q0 -244 -141 -463h-50q128 204 128 486q0 107 -35 225q-35 114 -93 209z" />
-    <glyph glyph-name="asterisk" unicode="*" horiz-adv-x="352" 
-d="M142 556l-115 39l14 37l115 -41v123h39v-123l115 41l15 -37l-118 -39l72 -98l-32 -23l-73 100l-73 -100l-32 23z" />
-    <glyph glyph-name="plus" unicode="+" horiz-adv-x="600" 
-d="M274 278v225h53v-225h225v-53h-225v-225h-53v225h-225v53h225z" />
-    <glyph glyph-name="comma" unicode="," horiz-adv-x="278" 
-d="M97 106h84v-107q0 -47 -21 -83q-24 -41 -67 -53v43q23 8 36 39q12 27 10 55h-42v106z" />
-    <glyph glyph-name="hyphen" unicode="-" horiz-adv-x="370" 
-d="M310 246h-250v58h250v-58z" />
-    <glyph glyph-name="hyphen" unicode="&#x2010;" horiz-adv-x="370" 
-d="M310 246h-250v58h250v-58z" />
-    <glyph glyph-name="period" unicode="." horiz-adv-x="278" 
-d="M181 0h-84v106h84v-106z" />
-    <glyph glyph-name="slash" unicode="/" horiz-adv-x="333" 
-d="M41 -15h-54l308 744h51z" />
-    <glyph glyph-name="zero" unicode="0" 
-d="M278 653q-181 0 -181 -307q0 -308 181 -308t181 308q0 307 -181 307zM278 706q244 0 244 -360q0 -361 -244 -361t-244 360q0 361 244 361z" />
-    <glyph glyph-name="one" unicode="1" 
-d="M97 520v45q100 4 133 21q51 26 64 114h51v-700h-63v520h-185z" />
-    <glyph glyph-name="two" unicode="2" 
-d="M497 58v-58h-461q2 93 63 164q39 45 135 109q103 69 139 105q58 58 58 132q0 66 -47 106q-44 37 -110 37q-81 0 -123 -59q-37 -52 -35 -136h-63q-2 112 54 178q59 70 170 70q95 0 154 -50q63 -53 63 -147q0 -80 -54 -146q-40 -49 -122 -103q-120 -79 -124 -83
-q-75 -59 -86 -119h389z" />
-    <glyph glyph-name="three" unicode="3" 
-d="M33 223h63q-3 -84 44 -134.5t131 -50.5q73 0 124 40q55 43 55 115q0 71 -49 111q-46 37 -119 37h-53v53h53q55 0 95 31q47 35 47 96q0 66 -45 101q-41 31 -108 31q-77 0 -119 -48.5t-40 -128.5h-63q1 101 61 165.5t160 64.5q90 0 150 -42q67 -49 67 -136
-q0 -58 -31.5 -100t-86.5 -55v-2q68 -11 106 -57.5t38 -115.5q0 -99 -72 -158q-67 -55 -169 -55q-112 0 -179 64t-60 174z" />
-    <glyph glyph-name="four" unicode="4" 
-d="M30 174v61l324 465h57v-473h105v-53h-105v-174h-58v174h-323zM87 227h266v381h-2z" />
-    <glyph glyph-name="five" unicode="5" 
-d="M36 200h63q2 -72 50.5 -117t121.5 -45q79 0 127 56t48 135q0 78 -50.5 130t-128.5 52q-103 0 -156 -83h-54l68 363h350v-58h-306l-45 -232l2 -2q55 65 148 65q107 0 171 -67.5t64 -174.5q0 -102 -70.5 -169.5t-172.5 -67.5q-99 0 -163.5 58.5t-66.5 156.5z" />
-    <glyph glyph-name="six" unicode="6" 
-d="M290 403q-80 0 -127 -52.5t-47 -132.5q0 -79 42 -127q46 -53 133 -53q73 0 122 54q46 51 46 126q0 79 -44 131q-47 54 -125 54zM505 523h-63q-8 59 -47 94.5t-99 35.5q-109 0 -158 -109q-35 -79 -35 -204h2q25 52 76 84t110 32q104 0 167.5 -65t63.5 -169
-q0 -102 -67 -169.5t-170 -67.5q-129 0 -187 85t-58 287q0 117 39 206q63 143 211 143q193 0 215 -183z" />
-    <glyph glyph-name="seven" unicode="7" 
-d="M51 633v58h450v-58q-266 -278 -283 -633h-68q13 179 78.5 323.5t209.5 309.5h-387z" />
-    <glyph glyph-name="eight" unicode="8" 
-d="M278 38q78 0 126 39q52 42 52 119q0 72 -54 114q-49 38 -124 38q-77 0 -125 -37q-53 -41 -53 -115q0 -76 52 -118q49 -40 126 -40zM490 531q0 -57 -30 -97.5t-84 -57.5v-1q67 -13 105 -60.5t38 -118.5q0 -101 -71 -158q-66 -53 -170 -53t-170 53q-72 57 -71 158
-q1 69 39.5 118t102.5 61v2q-114 34 -114 154q0 84 67 133q60 42 145 42q86 0 146 -42q67 -49 67 -133zM278 401q63 0 103 31q46 35 46 99q0 58 -41.5 90t-107.5 32q-62 0 -104 -31q-46 -34 -46 -91q0 -66 47 -100q41 -30 103 -30z" />
-    <glyph glyph-name="nine" unicode="9" 
-d="M266 288q80 0 128 54q46 51 46 131q0 79 -42 127q-46 53 -133 53q-73 0 -120.5 -52.5t-47.5 -127.5q0 -79 45.5 -132t123.5 -53zM51 168h63q8 -59 47 -94.5t99 -35.5q109 0 158 109q35 79 35 204h-2q-25 -52 -76 -84t-110 -32q-104 0 -167.5 65t-63.5 169q0 102 67 169.5
-t170 67.5q129 0 187 -85t58 -287q0 -117 -39 -206q-63 -143 -211 -143q-192 0 -215 183z" />
-    <glyph glyph-name="colon" unicode=":" horiz-adv-x="278" 
-d="M181 394h-84v106h84v-106zM181 0h-84v106h84v-106z" />
-    <glyph glyph-name="semicolon" unicode=";" horiz-adv-x="278" 
-d="M97 106h84v-107q0 -47 -21 -83q-24 -41 -67 -53v43q23 8 36 39q12 27 10 55h-42v106zM181 394h-84v106h84v-106z" />
-    <glyph glyph-name="less" unicode="&#x3c;" horiz-adv-x="600" 
-d="M554 47v-55l-508 234v54l508 234v-55l-446 -206z" />
-    <glyph glyph-name="equal" unicode="=" horiz-adv-x="600" 
-d="M552 378v-53h-503v53h503zM552 181v-53h-503v53h503z" />
-    <glyph glyph-name="greater" unicode="&#x3e;" horiz-adv-x="600" 
-d="M46 47l446 206l-446 206v55l508 -234v-54l-508 -234v55z" />
-    <glyph glyph-name="question" unicode="?" horiz-adv-x="537" 
-d="M117 494h-63q1 108 60.5 171.5t166.5 63.5q86 0 143 -49q60 -52 60 -136q0 -85 -87 -170q-70 -68 -78 -83q-13 -25 -15 -112h-63q-2 85 25 133q19 33 90 100q65 61 65 129q0 59 -41.5 97t-101.5 38q-79 0 -121 -50.5t-40 -131.5zM231 0v106h84v-106h-84z" />
-    <glyph glyph-name="at" unicode="@" horiz-adv-x="800" 
-d="M426 510q-71 0 -123 -80q-47 -73 -47 -151q0 -44 23 -70.5t60 -26.5q65 0 118 85q48 76 48 150q0 35 -25 64t-54 29zM614 544l-96 -288q-11 -34 -11 -54q0 -29 24 -29q65 0 118 82q51 79 51 166q0 120 -84 193q-80 70 -200 70q-138 0 -227 -95.5t-89 -235.5
-q0 -137 91 -230t227 -93q74 0 144.5 35.5t113.5 97.5h51q-45 -82 -129.5 -130t-179.5 -48q-160 0 -265.5 108t-105.5 267q0 156 107 262.5t260 106.5q146 0 242.5 -87t96.5 -224q0 -117 -72.5 -203.5t-159.5 -86.5q-61 0 -67 67h-2q-62 -66 -127 -66q-60 0 -96 42t-36 103
-q0 110 65 196q70 93 172 93q75 0 106 -84l25 65h53z" />
-    <glyph glyph-name="A" unicode="A" horiz-adv-x="630" 
-d="M-7 0l288 714h76l280 -714h-73l-87 222h-325l-86 -222h-73zM316 646l-143 -366h281z" />
-    <glyph glyph-name="B" unicode="B" horiz-adv-x="667" 
-d="M140 343v-285h234q188 0 188 147q0 138 -188 138h-234zM72 0v714h302q101 0 160 -38q69 -44 69 -137q0 -57 -34.5 -102.5t-89.5 -57.5v-2q69 -9 110 -56.5t41 -117.5q0 -203 -256 -203h-302zM140 656v-255h234q78 0 121 35q40 33 40 86q0 134 -161 134h-234z" />
-    <glyph glyph-name="C" unicode="C" horiz-adv-x="704" 
-d="M662 498h-68q-18 81 -78.5 127t-140.5 46q-133 0 -205 -98q-64 -87 -64 -216t64 -215q72 -99 205 -99q96 0 159 67q60 63 70 162h68q-14 -132 -93.5 -209.5t-203.5 -77.5q-161 0 -253 113q-84 103 -84 259t84 259q92 113 253 113q107 0 187 -59q87 -65 100 -172z" />
-    <glyph glyph-name="D" unicode="D" horiz-adv-x="685" 
-d="M140 656v-598h145q155 0 224.5 70t69.5 229t-69.5 229t-224.5 70h-145zM72 0v714h247q161 0 244.5 -90.5t83.5 -266.5t-83.5 -266.5t-244.5 -90.5h-247z" />
-    <glyph glyph-name="E" unicode="E" horiz-adv-x="593" 
-d="M72 0v714h493v-58h-425v-258h398v-58h-398v-282h430v-58h-498z" />
-    <glyph glyph-name="F" unicode="F" horiz-adv-x="537" 
-d="M72 0v714h453v-58h-385v-258h342v-58h-342v-340h-68z" />
-    <glyph glyph-name="G" unicode="G" horiz-adv-x="741" 
-d="M682 363v-363h-49l-9 130h-2q-68 -145 -247 -145q-161 0 -253 113q-84 103 -84 259t84 259q92 113 253 113q116 0 196 -60q83 -63 102 -177h-68q-7 68 -67 120q-67 59 -163 59q-133 0 -205 -98q-64 -87 -64 -216t64 -215q72 -99 205 -99q116 0 184 74q65 72 63 188h-243
-v58h303z" />
-    <glyph glyph-name="H" unicode="H" horiz-adv-x="704" 
-d="M72 0v714h68v-310h424v310h68v-714h-68v346h-424v-346h-68z" />
-    <glyph glyph-name="I" unicode="I" horiz-adv-x="222" 
-d="M77 0v714h68v-714h-68z" />
-    <glyph glyph-name="J" unicode="J" horiz-adv-x="500" 
-d="M360 228v486h68v-515q0 -111 -46 -161q-49 -53 -162 -53q-201 0 -199 244h68q3 -83 16 -116q30 -70 121 -70q77 0 107 43q27 41 27 142z" />
-    <glyph glyph-name="K" unicode="K" horiz-adv-x="648" 
-d="M72 0v714h68v-384l415 384h92l-319 -295l334 -419h-88l-297 372l-137 -127v-245h-68z" />
-    <glyph glyph-name="L" unicode="L" horiz-adv-x="537" 
-d="M72 0v714h68v-656h400v-58h-468z" />
-    <glyph glyph-name="M" unicode="M" horiz-adv-x="833" 
-d="M70 0v714h100l247 -626l246 626h100v-714h-68v617h-2l-244 -617h-64l-245 617h-2v-617h-68z" />
-    <glyph glyph-name="N" unicode="N" horiz-adv-x="704" 
-d="M71 0v714h76l416 -604h2v604h68v-714h-76l-416 604h-2v-604h-68z" />
-    <glyph glyph-name="O" unicode="O" horiz-adv-x="741" 
-d="M34 357q0 156 84 259q92 113 253 113q160 0 252 -113q84 -103 84 -259t-84 -259q-92 -113 -252 -113q-161 0 -253 113q-84 103 -84 259zM102 357q0 -129 64 -215q72 -99 205 -99q132 0 205 99q63 86 63 215t-63 216q-73 98 -205 98q-133 0 -205 -98q-64 -87 -64 -216z
-" />
-    <glyph glyph-name="P" unicode="P" horiz-adv-x="630" 
-d="M72 0v714h317q97 0 154.5 -54t57.5 -150t-57.5 -150t-154.5 -54h-249v-306h-68zM140 656v-292h234q73 0 116 37.5t43 108.5t-44 110q-42 36 -115 36h-234z" />
-    <glyph glyph-name="Q" unicode="Q" horiz-adv-x="741" 
-d="M444 192l104 -79q91 87 91 244q0 129 -63 216q-73 98 -205 98q-133 0 -205 -98q-64 -87 -64 -216t64 -215q72 -99 205 -99q77 0 132 35l-94 71zM701 -4l-34 -42l-111 84q-76 -53 -185 -53q-161 0 -253 113q-84 103 -84 259t84 259q92 113 253 113q160 0 252 -113
-q84 -103 84 -259q0 -179 -108 -283z" />
-    <glyph glyph-name="R" unicode="R" horiz-adv-x="667" 
-d="M72 0v714h327q95 0 153 -45q64 -49 64 -141q0 -70 -35 -118q-37 -51 -104 -64v-2q116 -15 125 -161q5 -86 6 -96q8 -64 27 -87h-75q-14 22 -19 111q-6 101 -25 140q-32 66 -118 66h-258v-317h-68zM140 656v-281h254q65 0 109.5 38t44.5 101q0 67 -39.5 104.5t-114.5 37.5
-h-254z" />
-    <glyph glyph-name="S" unicode="S" horiz-adv-x="630" 
-d="M36 235h68q-4 -192 233 -192q68 0 123 34q66 40 66 111q0 93 -147 129l-159 39q-161 39 -161 172q0 72 48 127q65 74 198 74q112 0 185 -56q80 -61 79 -166h-68q-3 78 -61 123q-54 41 -134 41q-78 0 -124 -31q-55 -37 -55 -111q0 -82 108 -109l174 -43q185 -46 185 -188
-q0 -69 -42 -120q-68 -84 -231 -84q-134 0 -209 60q-82 65 -76 190z" />
-    <glyph glyph-name="T" unicode="T" 
-d="M-5 656v58h566v-58h-249v-656h-68v656h-249z" />
-    <glyph glyph-name="U" unicode="U" horiz-adv-x="685" 
-d="M65 257v457h68v-442q0 -229 209 -229q210 0 210 229v442h68v-457q0 -120 -64 -191q-73 -81 -214 -81q-140 0 -213 81q-64 72 -64 191z" />
-    <glyph glyph-name="V" unicode="V" horiz-adv-x="593" 
-d="M258 0l-265 714h73l232 -639h2l230 639h72l-265 -714h-79z" />
-    <glyph glyph-name="W" unicode="W" horiz-adv-x="907" 
-d="M198 0l-198 714h73l162 -625h2l174 625h85l175 -625h2l163 625h68l-197 -714h-72l-181 639h-2l-180 -639h-74z" />
-    <glyph glyph-name="X" unicode="X" horiz-adv-x="574" 
-d="M245 368l-242 346h80l202 -299l209 299h76l-244 -348l258 -366h-82l-217 315l-220 -315h-77z" />
-    <glyph glyph-name="Y" unicode="Y" horiz-adv-x="611" 
-d="M271 0v294l-281 420h81l235 -362l234 362h81l-282 -420v-294h-68z" />
-    <glyph glyph-name="Z" unicode="Z" horiz-adv-x="574" 
-d="M37 656v58h514v-60l-470 -596h480v-58h-556v60l470 596h-438z" />
-    <glyph glyph-name="bracketleft" unicode="[" horiz-adv-x="241" 
-d="M237 -191h-161v920h161v-53h-98v-814h98v-53z" />
-    <glyph glyph-name="backslash" unicode="\" horiz-adv-x="333" 
-d="M346 -15h-54l-305 744h51z" />
-    <glyph glyph-name="bracketright" unicode="]" horiz-adv-x="241" 
-d="M4 729h161v-920h-161v53h98v814h-98v53z" />
-    <glyph glyph-name="asciicircum" unicode="^" horiz-adv-x="600" 
-d="M102 238h-58l230 453h52l230 -453h-57l-200 390z" />
-    <glyph glyph-name="underscore" unicode="_" horiz-adv-x="500" 
-d="M500 -125h-500v50h500v-50z" />
-    <glyph glyph-name="grave" unicode="`" horiz-adv-x="185" 
-d="M101 588l-132 141h78l105 -141h-51z" />
-    <glyph glyph-name="a" unicode="a" horiz-adv-x="519" 
-d="M390 200v82h-2q-8 -15 -53 -23q-55 -8 -110 -17q-62 -11 -92 -30q-38 -26 -38 -72q0 -45 34.5 -73.5t80.5 -28.5q75 0 127.5 44.5t52.5 117.5zM117 358h-63q4 87 63 132q53 41 146 41q190 0 190 -166v-274q0 -38 37 -38q10 0 20 4v-53q-19 -4 -41 -4q-45 0 -61 25
-q-13 20 -13 65h-2q-36 -55 -73 -78q-45 -27 -115 -27q-76 0 -122 37q-51 40 -51 114q0 96 87 131q43 18 190 35q45 5 60 16q21 15 21 54q0 106 -135 106t-138 -120z" />
-    <glyph glyph-name="b" unicode="b" horiz-adv-x="574" 
-d="M473 258q0 90 -39 150q-46 70 -130 70q-93 0 -141 -70q-40 -59 -40 -150t40 -150q48 -70 141 -70q84 0 130 70q39 60 39 150zM60 0v714h63v-296h2q20 54 71 84q48 29 108 29q112 0 175 -82q57 -74 57 -191t-57 -191q-63 -82 -175 -82q-63 0 -112.5 30.5t-71.5 82.5h-2
-v-98h-58z" />
-    <glyph glyph-name="c" unicode="c" horiz-adv-x="519" 
-d="M487 354h-63q-27 124 -146 124q-86 0 -136 -69q-45 -62 -45 -151t45 -151q50 -69 136 -69q59 0 101.5 41.5t48.5 105.5h63q-13 -93 -70 -146.5t-143 -53.5q-114 0 -181 81q-63 76 -63 192t63 192q67 81 181 81q85 0 141 -44q58 -47 68 -133z" />
-    <glyph glyph-name="d" unicode="d" horiz-adv-x="574" 
-d="M101 258q0 -90 40 -150q45 -70 129 -70q93 0 141 70q40 59 40 150t-40 150q-48 70 -141 70q-84 0 -129 -70q-40 -60 -40 -150zM514 714v-714h-58v98h-2q-20 -49 -73 -81t-111 -32q-112 0 -175 82q-57 74 -57 191t57 191q63 82 175 82q60 0 109.5 -29.5t69.5 -83.5h2v296
-h63z" />
-    <glyph glyph-name="e" unicode="e" horiz-adv-x="519" 
-d="M494 241h-397q1 -80 40 -136q47 -67 130 -67q128 0 162 132h63q-43 -185 -225 -185q-116 0 -178 81q-55 73 -55 192q0 113 55 188q63 85 178 85q117 0 178 -91q53 -79 49 -199zM97 294h334q-2 76 -45 128q-46 56 -119 56t-120 -57q-42 -50 -50 -127z" />
-    <glyph glyph-name="f" unicode="f" horiz-adv-x="259" 
-d="M259 516v-53h-102v-463h-63v463h-88v53h88v47q0 74 25 109q29 42 100 42q26 0 53 -4v-54q-28 5 -46 5q-47 0 -61 -29q-8 -17 -8 -71v-45h102z" />
-    <glyph glyph-name="g" unicode="g" 
-d="M267 59q78 0 124 63q42 58 42 142q0 86 -38 145q-45 69 -128 69q-84 0 -130 -67q-40 -58 -40 -147q0 -85 42 -141q46 -64 128 -64zM496 516v-474q0 -248 -229 -248q-85 0 -142 37q-66 42 -72 119h63q19 -103 151 -103q166 0 166 195v70h-2q-22 -48 -65.5 -77t-98.5 -29
-q-118 0 -178 77q-55 70 -55 188q0 111 61 183q64 77 172 77q55 0 100 -30t64 -74h2v89h63z" />
-    <glyph glyph-name="h" unicode="h" horiz-adv-x="537" 
-d="M59 0v714h63v-287h2q17 46 63.5 75t101.5 29q104 0 149 -53q40 -48 40 -146v-332h-63v322q0 156 -131 156q-74 0 -118 -50t-44 -127v-301h-63z" />
-    <glyph glyph-name="i" unicode="i" horiz-adv-x="185" 
-d="M61 0v516h63v-516h-63zM61 613v101h63v-101h-63z" />
-    <glyph glyph-name="j" unicode="j" horiz-adv-x="185" 
-d="M61 -63v579h63v-564q0 -143 -110 -143q-7 0 -22 1t-22 1v52q22 -2 34 -1q57 0 57 75zM61 613v101h63v-101h-63z" />
-    <glyph glyph-name="k" unicode="k" horiz-adv-x="500" 
-d="M60 0v714h63v-448l286 250h84l-220 -191l235 -325h-79l-204 281l-102 -85v-196h-63z" />
-    <glyph glyph-name="l" unicode="l" horiz-adv-x="185" 
-d="M61 0v714h63v-714h-63z" />
-    <glyph glyph-name="m" unicode="m" horiz-adv-x="833" 
-d="M60 0v516h58v-87h3q54 102 169 102q53 0 93.5 -26t54.5 -75q46 101 164 101q171 0 171 -181v-350h-63v347q0 131 -122 131q-140 0 -140 -186v-292h-63v350q0 128 -108 128q-78 0 -118 -54q-36 -50 -36 -136v-288h-63z" />
-    <glyph glyph-name="n" unicode="n" horiz-adv-x="537" 
-d="M59 0v516h63v-89h2q17 46 63.5 75t101.5 29q104 0 149 -53q40 -48 40 -146v-332h-63v322q0 156 -131 156q-74 0 -118 -50t-44 -127v-301h-63z" />
-    <glyph glyph-name="o" unicode="o" 
-d="M278 531q114 0 181 -81q63 -76 63 -192t-63 -192q-67 -81 -181 -81t-181 81q-63 76 -63 192t63 192q67 81 181 81zM278 478q-86 0 -136 -69q-45 -62 -45 -151t45 -151q50 -69 136 -69t136 69q45 62 45 151t-45 151q-50 69 -136 69z" />
-    <glyph glyph-name="p" unicode="p" horiz-adv-x="574" 
-d="M473 258q0 90 -39 150q-46 70 -130 70q-181 0 -181 -220q0 -91 40 -150q48 -70 141 -70q84 0 130 70q39 60 39 150zM60 -191v707h58v-98h2q22 52 71.5 82.5t112.5 30.5q112 0 175 -82q57 -74 57 -191t-57 -191q-63 -82 -175 -82q-60 0 -108 29q-51 30 -71 84h-2v-289h-63
-z" />
-    <glyph glyph-name="q" unicode="q" horiz-adv-x="574" 
-d="M101 258q0 -90 40 -150q45 -70 129 -70q93 0 141 70q40 59 40 150t-40 150q-48 70 -141 70q-84 0 -129 -70q-40 -60 -40 -150zM514 516v-707h-63v289h-2q-20 -54 -69.5 -83.5t-109.5 -29.5q-112 0 -175 82q-57 74 -57 191t57 191q63 82 175 82q58 0 111 -32t73 -81h2v98
-h58z" />
-    <glyph glyph-name="r" unicode="r" horiz-adv-x="315" 
-d="M60 0v516h58v-121h2q24 63 75 97t120 31v-63q-84 4 -138 -49.5t-54 -135.5v-275h-63z" />
-    <glyph glyph-name="s" unicode="s" horiz-adv-x="481" 
-d="M432 365h-63q-2 56 -42 86q-36 27 -93 27q-49 0 -82 -19q-40 -24 -40 -70q0 -63 107 -87l84 -19q147 -33 147 -143q0 -77 -67 -119q-58 -36 -140 -36q-198 0 -212 182h63q4 -62 45.5 -95.5t106.5 -33.5q53 0 93 22q48 27 48 76q0 69 -113 94l-81 18q-70 16 -103 43
-q-41 34 -41 95q0 73 63 112q53 33 131 33q81 0 132 -41q54 -44 57 -125z" />
-    <glyph glyph-name="t" unicode="t" horiz-adv-x="296" 
-d="M160 671v-155h105v-53h-105v-348q0 -34 8 -47q11 -19 43 -21q19 -1 54 3v-54q-9 0 -27.5 -1t-28.5 -1q-63 0 -87.5 25t-24.5 91v353h-90v53h90v155h63z" />
-    <glyph glyph-name="u" unicode="u" horiz-adv-x="537" 
-d="M478 516v-516h-58v93h-2q-24 -51 -72 -79.5t-106 -28.5q-181 0 -181 196v335h63v-336q0 -142 138 -142q76 0 118 64q37 57 37 143v271h63z" />
-    <glyph glyph-name="v" unicode="v" horiz-adv-x="463" 
-d="M201 0l-199 516h70l162 -453h2l160 453h65l-193 -516h-67z" />
-    <glyph glyph-name="w" unicode="w" horiz-adv-x="741" 
-d="M174 0l-166 516h67l132 -442h2l126 442h71l126 -442h2l132 442h67l-166 -516h-69l-127 434h-2l-126 -434h-69z" />
-    <glyph glyph-name="x" unicode="x" horiz-adv-x="481" 
-d="M1 0l200 268l-185 248h79l147 -196l144 196h79l-186 -247l201 -269h-80l-162 217l-159 -217h-78z" />
-    <glyph glyph-name="y" unicode="y" horiz-adv-x="463" 
-d="M208 2l-206 514h67l170 -440l159 440h63l-224 -595q-25 -68 -51 -89q-27 -24 -85 -23q-36 0 -48 5v53q25 -5 43 -5q38 0 59 22q15 16 31 59z" />
-    <glyph glyph-name="z" unicode="z" horiz-adv-x="463" 
-d="M432 471l-341 -418h355v-53h-431v51l337 412h-314v53h394v-45z" />
-    <glyph glyph-name="braceleft" unicode="{" horiz-adv-x="333" 
-d="M293 -191h-50q-42 0 -65 33.5t-23 91.5v148q0 160 -72 160v53q72 0 72 160v149q0 58 23 91.5t65 33.5h50v-53h-20q-55 0 -55 -103v-141q0 -70 -30 -119q-19 -32 -40 -43v-2q21 -11 40 -43q30 -49 30 -119v-141q0 -103 55 -103h20v-53z" />
-    <glyph glyph-name="bar" unicode="|" horiz-adv-x="222" 
-d="M85 -15v744h53v-744h-53z" />
-    <glyph glyph-name="braceright" unicode="}" horiz-adv-x="333" 
-d="M40 729h50q42 0 65 -33.5t23 -91.5v-148q0 -160 72 -160v-53q-72 0 -72 -160v-149q0 -58 -23 -91.5t-65 -33.5h-50v53h20q55 0 55 103v141q0 70 30 119q19 32 40 43v2q-21 11 -40 43q-30 49 -30 119v141q0 103 -55 103h-20v53z" />
-    <glyph glyph-name="asciitilde" unicode="~" horiz-adv-x="600" 
-d="M495 315v0l36 -36q-30 -42 -50 -59q-34 -27 -75 -27q-40 0 -109 32.5t-108 32.5q-52 0 -84 -69l-36 36q45 86 125 86q42 0 115 -32.5t98 -32.5q43 0 88 69z" />
-    <glyph glyph-name="exclamdown" unicode="&#xa1;" horiz-adv-x="241" 
-d="M140 338l15 -323v-206h-68v206l15 323h38zM79 516h84v-106h-84v106z" />
-    <glyph glyph-name="cent" unicode="&#xa2;" 
-d="M270 39v437q-73 -11 -115 -77q-38 -59 -38 -141q0 -81 38 -141q42 -67 115 -78zM507 354h-63q-26 120 -135 124v-440q55 3 94 44.5t45 102.5h63q-12 -90 -66 -143.5t-136 -56.5v-101h-39v102q-102 10 -160 89q-56 74 -56 183t56 183q59 79 160 88v89h39v-87
-q80 -2 134 -48t64 -129z" />
-    <glyph glyph-name="sterling" unicode="&#xa3;" 
-d="M46 330v32h63q-55 90 -55 161q0 101 71 157q62 49 157 49q110 0 172 -63.5t60 -173.5h-63q0 83 -42 132q-45 52 -127 52q-69 0 -114 -38q-51 -43 -51 -119q0 -42 20 -81t39 -76h160v-32h-142q24 -38 24 -97q0 -94 -110 -182l2 -2q58 30 97 30q41 0 99.5 -20.5t88.5 -20.5
-q64 0 115 56l32 -48q-69 -61 -146 -61q-51 0 -106 19q-64 22 -101 22q-33 0 -59 -12q-6 -3 -51 -29l-36 54q112 88 112 186q0 58 -28 105h-81z" />
-    <glyph glyph-name="currency" unicode="&#xa4;" 
-d="M29 136l57 57q-55 66 -55 153q0 86 54 154l-56 56l38 38l56 -56q68 56 156 56q87 0 153 -55l55 55l38 -38l-54 -54q56 -67 56 -156t-56 -156l54 -54l-38 -38l-55 55q-64 -55 -153 -55t-156 56l-56 -56zM94 346q0 -80 54 -135t131 -55t131 55t54 135t-54 135t-131 55
-t-131 -55t-54 -135z" />
-    <glyph glyph-name="yen" unicode="&#xa5;" 
-d="M110 304v45h106l-210 365h72l199 -365l201 365h69l-211 -365h109v-45h-131l-6 -10v-73h137v-45h-137v-176h-63v176h-135v45h135v73l-6 10h-129z" />
-    <glyph glyph-name="brokenbar" unicode="&#xa6;" horiz-adv-x="222" 
-d="M85 -15v275h53v-275h-53zM85 454v275h53v-275h-53z" />
-    <glyph glyph-name="section" unicode="&#xa7;" 
-d="M448 230q0 66 -97 126q-81 46 -161 91q-82 -34 -82 -108q0 -63 96 -124q80 -47 160 -93q84 35 84 108zM511 231q0 -47 -31 -84.5t-78 -55.5q50 -44 50 -109t-53 -106q-49 -37 -116 -37q-85 0 -135 46.5t-50 130.5h63q-2 -52 31 -88t85 -36q47 0 77 21q35 24 35 68
-q0 62 -89 116l-166 94q-89 63 -89 146q0 47 32 86q30 36 77 54q-50 44 -50 109t53 106q49 37 116 37q85 0 135 -46.5t50 -130.5h-63q2 52 -31 88t-85 36q-47 0 -77 -21q-35 -24 -35 -68q0 -62 89 -116l166 -94q89 -63 89 -146z" />
-    <glyph glyph-name="dieresis" unicode="&#xa8;" horiz-adv-x="185" 
-d="M215 605h-68v101h68v-101zM38 605h-68v101h68v-101z" />
-    <glyph glyph-name="copyright" unicode="&#xa9;" horiz-adv-x="800" 
-d="M772 357q0 -157 -107.5 -264.5t-264.5 -107.5t-264.5 107.5t-107.5 264.5t107.5 264.5t264.5 107.5t264.5 -107.5t107.5 -264.5zM714 357q0 134 -90 226.5t-224 92.5t-224 -92.5t-90 -226.5t90 -226.5t224 -92.5t224 92.5t90 226.5zM594 435h-51q-9 46 -44.5 76t-88.5 30
-q-78 0 -121.5 -51.5t-43.5 -132.5t42 -131q45 -53 123 -53q53 0 89 31q35 28 44 75h51q-9 -71 -60.5 -111t-124.5 -40q-100 0 -158.5 64t-58.5 165t58.5 165t158.5 64q73 0 124.5 -40t60.5 -111z" />
-    <glyph glyph-name="ordfeminine" unicode="&#xaa;" horiz-adv-x="311" 
-d="M228 502v47h-2q-4 -9 -34 -14q-97 -16 -105 -18q-45 -14 -45 -51q0 -25 22 -42t47 -17q48 0 82.5 26t34.5 69zM54 597h-47q6 109 140 109q128 0 128 -100v-172q0 -23 25 -23q7 0 13 3v-34q-21 -3 -27 -3q-29 0 -40 15q-8 12 -8 41h-2q-41 -65 -123 -65q-118 0 -118 96
-q0 61 57 84q28 11 123 21q53 6 53 31q0 67 -88 67q-85 0 -86 -70z" />
-    <glyph glyph-name="guillemotleft" unicode="&#xab;" horiz-adv-x="426" 
-d="M354 119l-133 125v62l133 125v-67l-94 -89l94 -89v-67zM193 119l-133 125v62l133 125v-67l-94 -89l94 -89v-67z" />
-    <glyph glyph-name="logicalnot" unicode="&#xac;" horiz-adv-x="600" 
-d="M499 111v214h-450v53h503v-267h-53z" />
-    <glyph glyph-name="minus" unicode="&#xad;" horiz-adv-x="600" 
-d="M552 278v-53h-503v53h503z" />
-    <glyph glyph-name="minus" unicode="&#x2212;" horiz-adv-x="600" 
-d="M552 278v-53h-503v53h503z" />
-    <glyph glyph-name="registered" unicode="&#xae;" horiz-adv-x="800" 
-d="M400 729q157 0 264.5 -107.5t107.5 -264.5t-107.5 -264.5t-264.5 -107.5t-264.5 107.5t-107.5 264.5t107.5 264.5t264.5 107.5zM400 676q-134 0 -224 -92.5t-90 -226.5t90 -226.5t224 -92.5t224 92.5t90 226.5t-90 226.5t-224 92.5zM318 333v-191h-53v430h165
-q140 0 140 -119q0 -101 -108 -120l128 -191h-62l-121 191h-89zM318 378h70q63 0 89 10q40 16 40 65q0 74 -102 74h-97v-149z" />
-    <glyph glyph-name="macron" unicode="&#xaf;" horiz-adv-x="185" 
-d="M257 634h-329v45h329v-45z" />
-    <glyph glyph-name="macron" unicode="&#x2c9;" horiz-adv-x="185" 
-d="M257 634h-329v45h329v-45z" />
-    <glyph glyph-name="degree" unicode="&#xb0;" horiz-adv-x="400" 
-d="M56 562q0 60 42 102t102 42t102 -42t42 -102t-42 -102t-102 -42t-102 42t-42 102zM95 562q0 -43 31 -74t74 -31q44 0 74.5 31t30.5 74q0 44 -30.5 74.5t-74.5 30.5q-43 0 -74 -30.5t-31 -74.5z" />
-    <glyph glyph-name="plusminus" unicode="&#xb1;" horiz-adv-x="600" 
-d="M274 323v180h53v-180h225v-53h-225v-180h-53v180h-224v53h224zM49 0v53h503v-53h-503z" />
-    <glyph glyph-name="twosuperior" unicode="&#xb2;" horiz-adv-x="333" 
-d="M71 553h-47q-2 71 36 112t114 41q66 0 106 -32t40 -93q0 -79 -115 -149q-39 -23 -78 -47q-47 -33 -58 -72h248v-39h-302q1 58 41 101q26 29 88 67q67 42 90 63q39 36 39 80q0 37 -30 59.5t-73 22.5q-50 0 -76 -35q-24 -31 -23 -79z" />
-    <glyph glyph-name="threesuperior" unicode="&#xb3;" horiz-adv-x="333" 
-d="M135 478v39h35q35 0 62 18q31 21 31 56q0 37 -29 58q-27 18 -69 18q-48 0 -74.5 -29t-25.5 -74h-47q1 64 40 103t107 39q61 0 101 -26q44 -30 44 -85q0 -76 -77 -93v-2q44 -6 69 -33.5t25 -69.5q0 -62 -46 -97t-115 -35q-75 0 -119 39t-39 108h47q-2 -48 28 -78t82 -30
-q46 0 80.5 24.5t34.5 65.5q0 40 -31.5 62t-78.5 22h-35z" />
-    <glyph glyph-name="acute" unicode="&#xb4;" horiz-adv-x="185" 
-d="M216 729l-132 -141h-51l105 141h78z" />
-    <glyph glyph-name="mu" unicode="&#xb5;" horiz-adv-x="537" 
-d="M478 516v-516h-58v93h-2q-24 -51 -72 -79.5t-106 -28.5q-70 0 -117 35v-211h-64v707h63v-336q0 -142 138 -142q76 0 118 64q37 57 37 143v271h63z" />
-    <glyph glyph-name="mu" unicode="&#x3bc;" horiz-adv-x="537" 
-d="M478 516v-516h-58v93h-2q-24 -51 -72 -79.5t-106 -28.5q-70 0 -117 35v-211h-64v707h63v-336q0 -142 138 -142q76 0 118 64q37 57 37 143v271h63z" />
-    <glyph glyph-name="paragraph" unicode="&#xb6;" horiz-adv-x="600" 
-d="M273 -177v505q-87 0 -147 56t-60 134q0 94 58.5 145t164.5 51h230v-891h-63v838h-120v-838h-63z" />
-    <glyph glyph-name="periodcentered" unicode="&#xb7;" horiz-adv-x="278" 
-d="M139 375q24 0 41.5 -18t17.5 -42q0 -23 -18 -40.5t-41 -17.5q-24 0 -41.5 18t-17.5 42q0 22 18 40t41 18z" />
-    <glyph glyph-name="periodcentered" unicode="&#x2219;" horiz-adv-x="278" 
-d="M139 375q24 0 41.5 -18t17.5 -42q0 -23 -18 -40.5t-41 -17.5q-24 0 -41.5 18t-17.5 42q0 22 18 40t41 18z" />
-    <glyph glyph-name="cedilla" unicode="&#xb8;" horiz-adv-x="185" 
-d="M91 0h35l-44 -58q15 3 31 3q85 0 85 -65q0 -42 -36 -66q-31 -20 -74 -20q-50 0 -94 21l14 32q34 -14 79 -14q53 0 53 43q0 38 -43 38q-35 0 -51 -8l-16 15z" />
-    <glyph glyph-name="onesuperior" unicode="&#xb9;" horiz-adv-x="333" 
-d="M35 584v39q63 2 86 12q33 15 42 62h40v-423h-47v310h-121z" />
-    <glyph glyph-name="ordmasculine" unicode="&#xba;" horiz-adv-x="334" 
-d="M162 368q-75 0 -117 48t-42 121t42 121t117 48q81 0 127 -49q42 -46 42 -120t-42 -120q-46 -49 -127 -49zM167 407q55 0 87 41q30 37 30 89t-30 89q-32 41 -87 41t-87 -41q-30 -37 -30 -89t30 -89q32 -41 87 -41z" />
-    <glyph glyph-name="guillemotright" unicode="&#xbb;" horiz-adv-x="426" 
-d="M233 119v67l94 89l-94 89v67l133 -125v-62zM72 119v67l94 89l-94 89v67l133 -125v-62z" />
-    <glyph glyph-name="onequarter" unicode="&#xbc;" horiz-adv-x="834" 
-d="M133 -34l453 759h47l-452 -759h-48zM48 584v39q63 2 86 12q33 15 42 62h40v-423h-47v310h-121zM642 0v100h-210v39l213 284h44v-284h68v-39h-68v-100h-47zM640 355l-162 -216h164v216h-2z" />
-    <glyph glyph-name="onehalf" unicode="&#xbd;" horiz-adv-x="834" 
-d="M133 -34l453 759h47l-452 -759h-48zM48 584v39q63 2 86 12q33 15 42 62h40v-423h-47v310h-121zM536 279h-47q-2 71 36 112t114 41q66 0 106 -32t40 -93q0 -79 -115 -149q-39 -23 -78 -47q-47 -33 -58 -72h248v-39h-302q1 58 41 101q26 29 88 67q67 42 90 63q39 36 39 80
-q0 37 -30 59.5t-73 22.5q-50 0 -76 -35q-24 -31 -23 -79z" />
-    <glyph glyph-name="threequarters" unicode="&#xbe;" horiz-adv-x="834" 
-d="M135 478v39h35q35 0 62 18q31 21 31 56q0 37 -29 58q-27 18 -69 18q-48 0 -74.5 -29t-25.5 -74h-47q1 64 40 103t107 39q61 0 101 -26q44 -30 44 -85q0 -76 -77 -93v-2q44 -6 69 -33.5t25 -69.5q0 -62 -46 -97t-115 -35q-75 0 -119 39t-39 108h47q-2 -48 28 -78t82 -30
-q46 0 80.5 24.5t34.5 65.5q0 40 -31.5 62t-78.5 22h-35zM173 -34l453 759h47l-452 -759h-48zM642 0v100h-210v39l213 284h44v-284h68v-39h-68v-100h-47zM640 355l-162 -216h164v216h-2z" />
-    <glyph glyph-name="questiondown" unicode="&#xbf;" horiz-adv-x="537" 
-d="M421 29h63q-1 -108 -59 -171q-61 -64 -168 -64q-86 0 -143 49q-60 52 -60 136q0 85 87 170q70 68 78 83q13 25 15 112h63q2 -85 -25 -133q-19 -33 -90 -100q-65 -61 -65 -129q0 -59 41.5 -97t101.5 -38q79 0 122 52q41 49 39 130zM307 516v-106h-84v106h84z" />
-    <glyph glyph-name="Agrave" unicode="&#xc0;" horiz-adv-x="630" 
-d="M-7 0l288 714h76l280 -714h-73l-87 222h-325l-86 -222h-73zM316 646l-143 -366h281zM324 771l-132 141h78l105 -141h-51z" />
-    <glyph glyph-name="Aacute" unicode="&#xc1;" horiz-adv-x="630" 
-d="M-7 0l288 714h76l280 -714h-73l-87 222h-325l-86 -222h-73zM316 646l-143 -366h281zM439 912l-132 -141h-51l105 141h78z" />
-    <glyph glyph-name="Acircumflex" unicode="&#xc2;" horiz-adv-x="630" 
-d="M-7 0l288 714h76l280 -714h-73l-87 222h-325l-86 -222h-73zM316 646l-143 -366h281zM350 912l114 -141h-65l-86 102l-88 -102h-58l114 141h69z" />
-    <glyph glyph-name="Atilde" unicode="&#xc3;" horiz-adv-x="630" 
-d="M-7 0l288 714h76l280 -714h-73l-87 222h-325l-86 -222h-73zM316 646l-143 -366h281zM434 891v0h38q-6 -39 -27 -65q-24 -30 -60 -30q-30 0 -76 23t-59 23q-53 0 -53 -51h-39q5 39 28.5 67.5t60.5 28.5q30 0 74.5 -22t60.5 -22q37 0 52 48z" />
-    <glyph glyph-name="Adieresis" unicode="&#xc4;" horiz-adv-x="630" 
-d="M-7 0l288 714h76l280 -714h-73l-87 222h-325l-86 -222h-73zM316 646l-143 -366h281zM438 788h-68v101h68v-101zM261 788h-68v101h68v-101z" />
-    <glyph glyph-name="Aring" unicode="&#xc5;" horiz-adv-x="630" 
-d="M-7 0l288 714h76l280 -714h-73l-87 222h-325l-86 -222h-73zM316 646l-143 -366h281zM209 860q0 44 31.5 75.5t75.5 31.5t75.5 -31.5t31.5 -75.5t-31.5 -75.5t-75.5 -31.5t-75.5 31.5t-31.5 75.5zM248 860q0 -28 20 -48t48 -20q29 0 48.5 20t19.5 48q0 29 -19.5 48.5
-t-48.5 19.5q-28 0 -48 -19.5t-20 -48.5z" />
-    <glyph glyph-name="AE" unicode="&#xc6;" horiz-adv-x="907" 
-d="M513 340v-282h371v-58h-439v226h-259l-116 -226h-77l374 714h513v-58h-367v-258h344v-58h-344zM216 284h229v372h-40z" />
-    <glyph glyph-name="Ccedilla" unicode="&#xc7;" horiz-adv-x="704" 
-d="M604 272h68q-14 -132 -93.5 -209.5t-203.5 -77.5h-1l-33 -43q15 3 31 3q85 0 85 -65q0 -42 -36 -66q-31 -20 -74 -20q-50 0 -94 21l14 32q34 -14 79 -14q53 0 53 43q0 38 -43 38q-35 0 -51 -8l-16 15l51 66q-145 12 -227 122q-75 101 -75 248q0 156 84 259
-q92 113 253 113q107 0 187 -59q87 -65 100 -172h-68q-18 81 -78.5 127t-140.5 46q-133 0 -205 -98q-64 -87 -64 -216t64 -215q72 -99 205 -99q96 0 159 67q60 63 70 162z" />
-    <glyph glyph-name="Egrave" unicode="&#xc8;" horiz-adv-x="593" 
-d="M72 0v714h493v-58h-425v-258h398v-58h-398v-282h430v-58h-498zM305 771l-132 141h78l105 -141h-51z" />
-    <glyph glyph-name="Eacute" unicode="&#xc9;" horiz-adv-x="593" 
-d="M72 0v714h493v-58h-425v-258h398v-58h-398v-282h430v-58h-498zM420 912l-132 -141h-51l105 141h78z" />
-    <glyph glyph-name="Ecircumflex" unicode="&#xca;" horiz-adv-x="593" 
-d="M72 0v714h493v-58h-425v-258h398v-58h-398v-282h430v-58h-498zM331 912l114 -141h-65l-86 102l-88 -102h-58l114 141h69z" />
-    <glyph glyph-name="Edieresis" unicode="&#xcb;" horiz-adv-x="593" 
-d="M72 0v714h493v-58h-425v-258h398v-58h-398v-282h430v-58h-498zM419 788h-68v101h68v-101zM242 788h-68v101h68v-101z" />
-    <glyph glyph-name="Igrave" unicode="&#xcc;" horiz-adv-x="222" 
-d="M77 0v714h68v-714h-68zM120 771l-132 141h78l105 -141h-51z" />
-    <glyph glyph-name="Iacute" unicode="&#xcd;" horiz-adv-x="222" 
-d="M77 0v714h68v-714h-68zM235 912l-132 -141h-51l105 141h78z" />
-    <glyph glyph-name="Icircumflex" unicode="&#xce;" horiz-adv-x="222" 
-d="M77 0v714h68v-714h-68zM146 912l114 -141h-65l-86 102l-88 -102h-58l114 141h69z" />
-    <glyph glyph-name="Idieresis" unicode="&#xcf;" horiz-adv-x="222" 
-d="M77 0v714h68v-714h-68zM234 788h-68v101h68v-101zM57 788h-68v101h68v-101z" />
-    <glyph glyph-name="Eth" unicode="&#xd0;" horiz-adv-x="685" 
-d="M140 656v-263h236v-45h-236v-290h145q155 0 224.5 70t69.5 229t-69.5 229t-224.5 70h-145zM72 393v321h247q161 0 244.5 -90.5t83.5 -266.5t-83.5 -266.5t-244.5 -90.5h-247v348h-72v45h72z" />
-    <glyph glyph-name="Ntilde" unicode="&#xd1;" horiz-adv-x="704" 
-d="M71 0v714h76l416 -604h2v604h68v-714h-76l-416 604h-2v-604h-68zM471 891v0h38q-6 -39 -27 -65q-24 -30 -60 -30q-30 0 -76 23t-59 23q-53 0 -53 -51h-39q5 39 28.5 67.5t60.5 28.5q30 0 74.5 -22t60.5 -22q37 0 52 48z" />
-    <glyph glyph-name="Ograve" unicode="&#xd2;" horiz-adv-x="741" 
-d="M34 357q0 156 84 259q92 113 253 113q160 0 252 -113q84 -103 84 -259t-84 -259q-92 -113 -252 -113q-161 0 -253 113q-84 103 -84 259zM102 357q0 -129 64 -215q72 -99 205 -99q132 0 205 99q63 86 63 215t-63 216q-73 98 -205 98q-133 0 -205 -98q-64 -87 -64 -216z
-M379 771l-132 141h78l105 -141h-51z" />
-    <glyph glyph-name="Oacute" unicode="&#xd3;" horiz-adv-x="741" 
-d="M34 357q0 156 84 259q92 113 253 113q160 0 252 -113q84 -103 84 -259t-84 -259q-92 -113 -252 -113q-161 0 -253 113q-84 103 -84 259zM102 357q0 -129 64 -215q72 -99 205 -99q132 0 205 99q63 86 63 215t-63 216q-73 98 -205 98q-133 0 -205 -98q-64 -87 -64 -216z
-M494 912l-132 -141h-51l105 141h78z" />
-    <glyph glyph-name="Ocircumflex" unicode="&#xd4;" horiz-adv-x="741" 
-d="M34 357q0 156 84 259q92 113 253 113q160 0 252 -113q84 -103 84 -259t-84 -259q-92 -113 -252 -113q-161 0 -253 113q-84 103 -84 259zM102 357q0 -129 64 -215q72 -99 205 -99q132 0 205 99q63 86 63 215t-63 216q-73 98 -205 98q-133 0 -205 -98q-64 -87 -64 -216z
-M405 912l114 -141h-65l-86 102l-88 -102h-58l114 141h69z" />
-    <glyph glyph-name="Otilde" unicode="&#xd5;" horiz-adv-x="741" 
-d="M34 357q0 156 84 259q92 113 253 113q160 0 252 -113q84 -103 84 -259t-84 -259q-92 -113 -252 -113q-161 0 -253 113q-84 103 -84 259zM102 357q0 -129 64 -215q72 -99 205 -99q132 0 205 99q63 86 63 215t-63 216q-73 98 -205 98q-133 0 -205 -98q-64 -87 -64 -216z
-M489 891v0h38q-6 -39 -27 -65q-24 -30 -60 -30q-30 0 -76 23t-59 23q-53 0 -53 -51h-39q5 39 28.5 67.5t60.5 28.5q30 0 74.5 -22t60.5 -22q37 0 52 48z" />
-    <glyph glyph-name="Odieresis" unicode="&#xd6;" horiz-adv-x="741" 
-d="M34 357q0 156 84 259q92 113 253 113q160 0 252 -113q84 -103 84 -259t-84 -259q-92 -113 -252 -113q-161 0 -253 113q-84 103 -84 259zM102 357q0 -129 64 -215q72 -99 205 -99q132 0 205 99q63 86 63 215t-63 216q-73 98 -205 98q-133 0 -205 -98q-64 -87 -64 -216z
-M493 788h-68v101h68v-101zM316 788h-68v101h68v-101z" />
-    <glyph glyph-name="multiply" unicode="&#xd7;" horiz-adv-x="600" 
-d="M262 251l-188 189l37 37l189 -189l189 189l37 -37l-189 -189l189 -189l-37 -37l-189 188l-189 -188l-37 37z" />
-    <glyph glyph-name="Oslash" unicode="&#xd8;" horiz-adv-x="741" 
-d="M580 565l-397 -443q71 -79 188 -79q132 0 205 99q63 86 63 215q0 126 -59 208zM160 150l398 443q-71 78 -187 78q-133 0 -205 -98q-64 -87 -64 -216q0 -124 58 -207zM672 719l26 -23l-73 -81q82 -101 82 -258q0 -156 -84 -259q-92 -113 -252 -113q-142 0 -231 89l-76 -86
-l-27 25l79 88q-82 103 -82 256q0 156 84 259q92 113 253 113q140 0 230 -88z" />
-    <glyph glyph-name="Ugrave" unicode="&#xd9;" horiz-adv-x="685" 
-d="M65 257v457h68v-442q0 -229 209 -229q210 0 210 229v442h68v-457q0 -120 -64 -191q-73 -81 -214 -81q-140 0 -213 81q-64 72 -64 191zM351 771l-132 141h78l105 -141h-51z" />
-    <glyph glyph-name="Uacute" unicode="&#xda;" horiz-adv-x="685" 
-d="M65 257v457h68v-442q0 -229 209 -229q210 0 210 229v442h68v-457q0 -120 -64 -191q-73 -81 -214 -81q-140 0 -213 81q-64 72 -64 191zM466 912l-132 -141h-51l105 141h78z" />
-    <glyph glyph-name="Ucircumflex" unicode="&#xdb;" horiz-adv-x="685" 
-d="M65 257v457h68v-442q0 -229 209 -229q210 0 210 229v442h68v-457q0 -120 -64 -191q-73 -81 -214 -81q-140 0 -213 81q-64 72 -64 191zM377 912l114 -141h-65l-86 102l-88 -102h-58l114 141h69z" />
-    <glyph glyph-name="Udieresis" unicode="&#xdc;" horiz-adv-x="685" 
-d="M65 257v457h68v-442q0 -229 209 -229q210 0 210 229v442h68v-457q0 -120 -64 -191q-73 -81 -214 -81q-140 0 -213 81q-64 72 -64 191zM465 788h-68v101h68v-101zM288 788h-68v101h68v-101z" />
-    <glyph glyph-name="Yacute" unicode="&#xdd;" horiz-adv-x="611" 
-d="M271 0v294l-281 420h81l235 -362l234 362h81l-282 -420v-294h-68zM429 912l-132 -141h-51l105 141h78z" />
-    <glyph glyph-name="Thorn" unicode="&#xde;" horiz-adv-x="630" 
-d="M72 0v714h68v-117h249q97 0 154.5 -54t57.5 -150t-57.5 -150t-154.5 -54h-249v-189h-68zM140 539v-292h234q73 0 116 37.5t43 108.5t-44 110q-42 36 -115 36h-234z" />
-    <glyph glyph-name="germandbls" unicode="&#xdf;" horiz-adv-x="537" 
-d="M61 0v535q0 96 55 147q52 47 144 47q84 0 138 -45q58 -49 58 -131q0 -56 -29 -96q-31 -43 -84 -54v-2q70 -6 111 -61q38 -52 38 -124q0 -107 -63 -166q-59 -55 -159 -58q-27 -1 -50 2v55q20 -4 47 -4q68 -1 112 37q50 42 50 123q0 97 -58 134q-47 29 -151 29v53
-q173 0 173 130q0 54 -38.5 89.5t-93.5 35.5q-137 0 -137 -146v-530h-63z" />
-    <glyph glyph-name="agrave" unicode="&#xe0;" horiz-adv-x="519" 
-d="M390 200v82h-2q-8 -15 -53 -23q-55 -8 -110 -17q-62 -11 -92 -30q-38 -26 -38 -72q0 -45 34.5 -73.5t80.5 -28.5q75 0 127.5 44.5t52.5 117.5zM117 358h-63q4 87 63 132q53 41 146 41q190 0 190 -166v-274q0 -38 37 -38q10 0 20 4v-53q-19 -4 -41 -4q-45 0 -61 25
-q-13 20 -13 65h-2q-36 -55 -73 -78q-45 -27 -115 -27q-76 0 -122 37q-51 40 -51 114q0 96 87 131q43 18 190 35q45 5 60 16q21 15 21 54q0 106 -135 106t-138 -120zM268 588l-132 141h78l105 -141h-51z" />
-    <glyph glyph-name="aacute" unicode="&#xe1;" horiz-adv-x="519" 
-d="M390 200v82h-2q-8 -15 -53 -23q-55 -8 -110 -17q-62 -11 -92 -30q-38 -26 -38 -72q0 -45 34.5 -73.5t80.5 -28.5q75 0 127.5 44.5t52.5 117.5zM117 358h-63q4 87 63 132q53 41 146 41q190 0 190 -166v-274q0 -38 37 -38q10 0 20 4v-53q-19 -4 -41 -4q-45 0 -61 25
-q-13 20 -13 65h-2q-36 -55 -73 -78q-45 -27 -115 -27q-76 0 -122 37q-51 40 -51 114q0 96 87 131q43 18 190 35q45 5 60 16q21 15 21 54q0 106 -135 106t-138 -120zM383 729l-132 -141h-51l105 141h78z" />
-    <glyph glyph-name="acircumflex" unicode="&#xe2;" horiz-adv-x="519" 
-d="M390 200v82h-2q-8 -15 -53 -23q-55 -8 -110 -17q-62 -11 -92 -30q-38 -26 -38 -72q0 -45 34.5 -73.5t80.5 -28.5q75 0 127.5 44.5t52.5 117.5zM117 358h-63q4 87 63 132q53 41 146 41q190 0 190 -166v-274q0 -38 37 -38q10 0 20 4v-53q-19 -4 -41 -4q-45 0 -61 25
-q-13 20 -13 65h-2q-36 -55 -73 -78q-45 -27 -115 -27q-76 0 -122 37q-51 40 -51 114q0 96 87 131q43 18 190 35q45 5 60 16q21 15 21 54q0 106 -135 106t-138 -120zM294 729l114 -141h-65l-86 102l-88 -102h-58l114 141h69z" />
-    <glyph glyph-name="atilde" unicode="&#xe3;" horiz-adv-x="519" 
-d="M390 200v82h-2q-8 -15 -53 -23q-55 -8 -110 -17q-62 -11 -92 -30q-38 -26 -38 -72q0 -45 34.5 -73.5t80.5 -28.5q75 0 127.5 44.5t52.5 117.5zM117 358h-63q4 87 63 132q53 41 146 41q190 0 190 -166v-274q0 -38 37 -38q10 0 20 4v-53q-19 -4 -41 -4q-45 0 -61 25
-q-13 20 -13 65h-2q-36 -55 -73 -78q-45 -27 -115 -27q-76 0 -122 37q-51 40 -51 114q0 96 87 131q43 18 190 35q45 5 60 16q21 15 21 54q0 106 -135 106t-138 -120zM378 708v0h38q-6 -39 -27 -65q-24 -30 -60 -30q-30 0 -76 23t-59 23q-53 0 -53 -51h-39q5 39 28.5 67.5
-t60.5 28.5q30 0 74.5 -22t60.5 -22q37 0 52 48z" />
-    <glyph glyph-name="adieresis" unicode="&#xe4;" horiz-adv-x="519" 
-d="M390 200v82h-2q-8 -15 -53 -23q-55 -8 -110 -17q-62 -11 -92 -30q-38 -26 -38 -72q0 -45 34.5 -73.5t80.5 -28.5q75 0 127.5 44.5t52.5 117.5zM117 358h-63q4 87 63 132q53 41 146 41q190 0 190 -166v-274q0 -38 37 -38q10 0 20 4v-53q-19 -4 -41 -4q-45 0 -61 25
-q-13 20 -13 65h-2q-36 -55 -73 -78q-45 -27 -115 -27q-76 0 -122 37q-51 40 -51 114q0 96 87 131q43 18 190 35q45 5 60 16q21 15 21 54q0 106 -135 106t-138 -120zM382 605h-68v101h68v-101zM205 605h-68v101h68v-101z" />
-    <glyph glyph-name="aring" unicode="&#xe5;" horiz-adv-x="519" 
-d="M390 200v82h-2q-8 -15 -53 -23q-55 -8 -110 -17q-62 -11 -92 -30q-38 -26 -38 -72q0 -45 34.5 -73.5t80.5 -28.5q75 0 127.5 44.5t52.5 117.5zM117 358h-63q4 87 63 132q53 41 146 41q190 0 190 -166v-274q0 -38 37 -38q10 0 20 4v-53q-19 -4 -41 -4q-45 0 -61 25
-q-13 20 -13 65h-2q-36 -55 -73 -78q-45 -27 -115 -27q-76 0 -122 37q-51 40 -51 114q0 96 87 131q43 18 190 35q45 5 60 16q21 15 21 54q0 106 -135 106t-138 -120zM153 677q0 44 31.5 75.5t75.5 31.5t75.5 -31.5t31.5 -75.5t-31.5 -75.5t-75.5 -31.5t-75.5 31.5t-31.5 75.5
-zM192 677q0 -28 20 -48t48 -20q29 0 48.5 20t19.5 48q0 29 -19.5 48.5t-48.5 19.5q-28 0 -48 -19.5t-20 -48.5z" />
-    <glyph glyph-name="ae" unicode="&#xe6;" horiz-adv-x="870" 
-d="M453 294h325q0 74 -44 128q-47 56 -119 56q-76 0 -121 -56q-43 -52 -41 -128zM390 200v82h-2q-8 -15 -53 -23q-55 -8 -110 -17q-62 -11 -92 -30q-38 -26 -38 -72q0 -45 34.5 -73.5t80.5 -28.5q75 0 127.5 44.5t52.5 117.5zM841 241h-388q-6 -87 36 -143q45 -60 132 -60
-q124 0 153 129h63q-20 -87 -78 -134.5t-147 -47.5q-142 0 -189 130h-2q-11 -60 -80 -97q-61 -33 -136 -33q-76 0 -122 37q-51 40 -51 114q0 96 87 131q43 18 190 35q45 5 60 16q21 15 21 54q0 106 -135 106t-138 -120h-63q4 87 63 132q53 41 146 41q157 0 172 -107h2
-q20 52 73 81q48 26 108 26q118 0 175 -89q48 -74 48 -201z" />
-    <glyph glyph-name="ccedilla" unicode="&#xe7;" horiz-adv-x="519" 
-d="M428 185h63q-13 -92 -68.5 -145t-140.5 -55l-33 -43q15 3 31 3q85 0 85 -65q0 -42 -36 -66q-31 -20 -74 -20q-50 0 -94 21l14 32q34 -14 79 -14q53 0 53 43q0 38 -43 38q-35 0 -51 -8l-16 15l51 66q-101 11 -159 89q-55 74 -55 182q0 116 63 192q67 81 181 81
-q85 0 141 -44q58 -47 68 -133h-63q-27 124 -146 124q-86 0 -136 -69q-45 -62 -45 -151t45 -151q50 -69 136 -69q59 0 101.5 41.5t48.5 105.5z" />
-    <glyph glyph-name="egrave" unicode="&#xe8;" horiz-adv-x="519" 
-d="M494 241h-397q1 -80 40 -136q47 -67 130 -67q128 0 162 132h63q-43 -185 -225 -185q-116 0 -178 81q-55 73 -55 192q0 113 55 188q63 85 178 85q117 0 178 -91q53 -79 49 -199zM97 294h334q-2 76 -45 128q-46 56 -119 56t-120 -57q-42 -50 -50 -127zM268 588l-132 141h78
-l105 -141h-51z" />
-    <glyph glyph-name="eacute" unicode="&#xe9;" horiz-adv-x="519" 
-d="M494 241h-397q1 -80 40 -136q47 -67 130 -67q128 0 162 132h63q-43 -185 -225 -185q-116 0 -178 81q-55 73 -55 192q0 113 55 188q63 85 178 85q117 0 178 -91q53 -79 49 -199zM97 294h334q-2 76 -45 128q-46 56 -119 56t-120 -57q-42 -50 -50 -127zM383 729l-132 -141
-h-51l105 141h78z" />
-    <glyph glyph-name="ecircumflex" unicode="&#xea;" horiz-adv-x="519" 
-d="M494 241h-397q1 -80 40 -136q47 -67 130 -67q128 0 162 132h63q-43 -185 -225 -185q-116 0 -178 81q-55 73 -55 192q0 113 55 188q63 85 178 85q117 0 178 -91q53 -79 49 -199zM97 294h334q-2 76 -45 128q-46 56 -119 56t-120 -57q-42 -50 -50 -127zM294 729l114 -141
-h-65l-86 102l-88 -102h-58l114 141h69z" />
-    <glyph glyph-name="edieresis" unicode="&#xeb;" horiz-adv-x="519" 
-d="M494 241h-397q1 -80 40 -136q47 -67 130 -67q128 0 162 132h63q-43 -185 -225 -185q-116 0 -178 81q-55 73 -55 192q0 113 55 188q63 85 178 85q117 0 178 -91q53 -79 49 -199zM97 294h334q-2 76 -45 128q-46 56 -119 56t-120 -57q-42 -50 -50 -127zM382 605h-68v101h68
-v-101zM205 605h-68v101h68v-101z" />
-    <glyph glyph-name="igrave" unicode="&#xec;" horiz-adv-x="185" 
-d="M124 0h-63v516h63v-516zM101 588l-132 141h78l105 -141h-51z" />
-    <glyph glyph-name="iacute" unicode="&#xed;" horiz-adv-x="185" 
-d="M124 0h-63v516h63v-516zM216 729l-132 -141h-51l105 141h78z" />
-    <glyph glyph-name="icircumflex" unicode="&#xee;" horiz-adv-x="185" 
-d="M124 0h-63v516h63v-516zM127 729l114 -141h-65l-86 102l-88 -102h-58l114 141h69z" />
-    <glyph glyph-name="idieresis" unicode="&#xef;" horiz-adv-x="185" 
-d="M124 0h-63v516h63v-516zM215 605h-68v101h68v-101zM38 605h-68v101h68v-101z" />
-    <glyph glyph-name="eth" unicode="&#xf0;" 
-d="M276 453q-91 0 -138 -61q-41 -54 -41 -144q0 -89 42 -146q47 -64 135 -64q86 0 134 60q45 56 45 152q0 84 -41 139q-47 64 -136 64zM135 596l100 56q-51 34 -93 55l36 35q61 -31 104 -64l114 64l33 -32l-109 -61q196 -161 196 -391q0 -119 -60 -193q-65 -80 -185 -80
-q-116 0 -179 76q-58 71 -58 188q0 109 62 181q65 76 175 76q96 0 133 -40h2q-46 86 -133 157l-105 -59z" />
-    <glyph glyph-name="ntilde" unicode="&#xf1;" horiz-adv-x="537" 
-d="M59 0v516h63v-89h2q17 46 63.5 75t101.5 29q104 0 149 -53q40 -48 40 -146v-332h-63v322q0 156 -131 156q-74 0 -118 -50t-44 -127v-301h-63zM387 708v0h38q-6 -39 -27 -65q-24 -30 -60 -30q-30 0 -76 23t-59 23q-53 0 -53 -51h-39q5 39 28.5 67.5t60.5 28.5
-q30 0 74.5 -22t60.5 -22q37 0 52 48z" />
-    <glyph glyph-name="ograve" unicode="&#xf2;" 
-d="M278 531q114 0 181 -81q63 -76 63 -192t-63 -192q-67 -81 -181 -81t-181 81q-63 76 -63 192t63 192q67 81 181 81zM278 478q-86 0 -136 -69q-45 -62 -45 -151t45 -151q50 -69 136 -69t136 69q45 62 45 151t-45 151q-50 69 -136 69zM287 588l-132 141h78l105 -141h-51z
-" />
-    <glyph glyph-name="oacute" unicode="&#xf3;" 
-d="M278 531q114 0 181 -81q63 -76 63 -192t-63 -192q-67 -81 -181 -81t-181 81q-63 76 -63 192t63 192q67 81 181 81zM278 478q-86 0 -136 -69q-45 -62 -45 -151t45 -151q50 -69 136 -69t136 69q45 62 45 151t-45 151q-50 69 -136 69zM402 729l-132 -141h-51l105 141h78z
-" />
-    <glyph glyph-name="ocircumflex" unicode="&#xf4;" 
-d="M278 531q114 0 181 -81q63 -76 63 -192t-63 -192q-67 -81 -181 -81t-181 81q-63 76 -63 192t63 192q67 81 181 81zM278 478q-86 0 -136 -69q-45 -62 -45 -151t45 -151q50 -69 136 -69t136 69q45 62 45 151t-45 151q-50 69 -136 69zM313 729l114 -141h-65l-86 102
-l-88 -102h-58l114 141h69z" />
-    <glyph glyph-name="otilde" unicode="&#xf5;" 
-d="M278 531q114 0 181 -81q63 -76 63 -192t-63 -192q-67 -81 -181 -81t-181 81q-63 76 -63 192t63 192q67 81 181 81zM278 478q-86 0 -136 -69q-45 -62 -45 -151t45 -151q50 -69 136 -69t136 69q45 62 45 151t-45 151q-50 69 -136 69zM397 708v0h38q-6 -39 -27 -65
-q-24 -30 -60 -30q-30 0 -76 23t-59 23q-53 0 -53 -51h-39q5 39 28.5 67.5t60.5 28.5q30 0 74.5 -22t60.5 -22q37 0 52 48z" />
-    <glyph glyph-name="odieresis" unicode="&#xf6;" 
-d="M278 531q114 0 181 -81q63 -76 63 -192t-63 -192q-67 -81 -181 -81t-181 81q-63 76 -63 192t63 192q67 81 181 81zM278 478q-86 0 -136 -69q-45 -62 -45 -151t45 -151q50 -69 136 -69t136 69q45 62 45 151t-45 151q-50 69 -136 69zM401 605h-68v101h68v-101zM224 605h-68
-v101h68v-101z" />
-    <glyph glyph-name="divide" unicode="&#xf7;" horiz-adv-x="600" 
-d="M552 225h-503v53h503v-53zM244 453q0 23 16.5 39.5t39.5 16.5t39.5 -16.5t16.5 -39.5q0 -21 -16.5 -38.5t-38.5 -17.5q-23 0 -40 16.5t-17 39.5zM244 50q0 23 16.5 39.5t39.5 16.5t39.5 -16.5t16.5 -39.5q0 -21 -16.5 -38.5t-38.5 -17.5q-23 0 -40 16.5t-17 39.5z" />
-    <glyph glyph-name="oslash" unicode="&#xf8;" 
-d="M133 123l273 295q-48 60 -128 60q-86 0 -136 -69q-45 -62 -45 -151q0 -77 36 -135zM424 392l-273 -295q47 -59 127 -59q86 0 136 69q45 62 45 151q0 77 -35 134zM500 519l22 -20l-54 -59q54 -75 54 -182q0 -116 -63 -192q-67 -81 -181 -81q-105 0 -168 68l-59 -64l-22 20
-l61 67q-56 73 -56 182q0 116 63 192q67 81 181 81q105 0 170 -68z" />
-    <glyph glyph-name="ugrave" unicode="&#xf9;" horiz-adv-x="537" 
-d="M478 516v-516h-58v93h-2q-24 -51 -72 -79.5t-106 -28.5q-181 0 -181 196v335h63v-336q0 -142 138 -142q76 0 118 64q37 57 37 143v271h63zM277 588l-132 141h78l105 -141h-51z" />
-    <glyph glyph-name="uacute" unicode="&#xfa;" horiz-adv-x="537" 
-d="M478 516v-516h-58v93h-2q-24 -51 -72 -79.5t-106 -28.5q-181 0 -181 196v335h63v-336q0 -142 138 -142q76 0 118 64q37 57 37 143v271h63zM392 729l-132 -141h-51l105 141h78z" />
-    <glyph glyph-name="ucircumflex" unicode="&#xfb;" horiz-adv-x="537" 
-d="M478 516v-516h-58v93h-2q-24 -51 -72 -79.5t-106 -28.5q-181 0 -181 196v335h63v-336q0 -142 138 -142q76 0 118 64q37 57 37 143v271h63zM303 729l114 -141h-65l-86 102l-88 -102h-58l114 141h69z" />
-    <glyph glyph-name="udieresis" unicode="&#xfc;" horiz-adv-x="537" 
-d="M478 516v-516h-58v93h-2q-24 -51 -72 -79.5t-106 -28.5q-181 0 -181 196v335h63v-336q0 -142 138 -142q76 0 118 64q37 57 37 143v271h63zM391 605h-68v101h68v-101zM214 605h-68v101h68v-101z" />
-    <glyph glyph-name="yacute" unicode="&#xfd;" horiz-adv-x="463" 
-d="M208 2l-206 514h67l170 -440l159 440h63l-224 -595q-25 -68 -51 -89q-27 -24 -85 -23q-36 0 -48 5v53q25 -5 43 -5q38 0 59 22q15 16 31 59zM355 729l-132 -141h-51l105 141h78z" />
-    <glyph glyph-name="thorn" unicode="&#xfe;" horiz-adv-x="574" 
-d="M473 258q0 90 -39 150q-46 70 -130 70q-93 0 -141 -70q-40 -59 -40 -150t40 -150q48 -70 141 -70q84 0 130 70q39 60 39 150zM60 -191v905h63v-296h2q20 54 71 84q48 29 108 29q112 0 175 -82q57 -74 57 -191t-57 -191q-63 -82 -175 -82q-60 0 -108 29q-51 30 -71 84h-2
-v-289h-63z" />
-    <glyph glyph-name="ydieresis" unicode="&#xff;" horiz-adv-x="463" 
-d="M208 2l-206 514h67l170 -440l159 440h63l-224 -595q-25 -68 -51 -89q-27 -24 -85 -23q-36 0 -48 5v53q25 -5 43 -5q38 0 59 22q15 16 31 59zM354 605h-68v101h68v-101zM177 605h-68v101h68v-101z" />
-    <glyph glyph-name="dotlessi" unicode="&#x131;" horiz-adv-x="185" 
-d="M124 0h-63v516h63v-516z" />
-    <glyph glyph-name="Lslash" unicode="&#x141;" horiz-adv-x="537" 
-d="M72 306v408h68v-361l243 168v-56l-243 -167v-240h400v-58h-468v251l-82 -57v56z" />
-    <glyph glyph-name="lslash" unicode="&#x142;" horiz-adv-x="185" 
-d="M61 395v319h63v-267l67 55v-55l-67 -55v-392h-63v340l-67 -55v55z" />
-    <glyph glyph-name="OE" unicode="&#x152;" horiz-adv-x="1074" 
-d="M569 177v332q0 46 -5 66q-16 43 -78 71q-54 25 -111 25q-126 0 -200 -94q-70 -87 -70 -220t70 -220q74 -94 200 -94q113 0 176 79q18 23 18 55zM637 656v-258h368v-58h-368v-282h409v-58h-477v70q-34 -41 -89 -63q-51 -22 -105 -22q-154 0 -249 111q-89 105 -89 261
-t89 261q95 111 249 111q129 0 194 -74v59h474v-58h-406z" />
-    <glyph glyph-name="oe" unicode="&#x153;" horiz-adv-x="889" 
-d="M265 478q-89 0 -132 -75q-34 -60 -34 -157q0 -86 38 -143q44 -65 127 -65q152 0 152 224q0 216 -151 216zM799 166h63q-21 -86 -80.5 -133.5t-147.5 -47.5q-66 0 -116 34q-53 35 -68 97h-2q-37 -131 -185 -131q-114 0 -174 82q-53 73 -53 191q0 115 56 190q62 83 174 83
-q144 0 184 -127h2q21 57 72 92t118 35q221 0 221 -290h-384q0 -83 39 -139q43 -64 122 -64q122 0 159 128zM479 294h321q0 77 -41 128q-44 56 -119 56q-76 0 -120 -55q-41 -51 -41 -129z" />
-    <glyph glyph-name="Scaron" unicode="&#x160;" horiz-adv-x="630" 
-d="M36 235h68q-4 -192 233 -192q68 0 123 34q66 40 66 111q0 93 -147 129l-159 39q-161 39 -161 172q0 72 48 127q65 74 198 74q112 0 185 -56q80 -61 79 -166h-68q-3 78 -61 123q-54 41 -134 41q-78 0 -124 -31q-55 -37 -55 -111q0 -82 108 -109l174 -43q185 -46 185 -188
-q0 -69 -42 -120q-68 -84 -231 -84q-134 0 -209 60q-82 65 -76 190zM281 771l-114 141h65l86 -102l88 102h58l-114 -141h-69z" />
-    <glyph glyph-name="scaron" unicode="&#x161;" horiz-adv-x="481" 
-d="M432 365h-63q-2 56 -42 86q-36 27 -93 27q-49 0 -82 -19q-40 -24 -40 -70q0 -63 107 -87l84 -19q147 -33 147 -143q0 -77 -67 -119q-58 -36 -140 -36q-198 0 -212 182h63q4 -62 45.5 -95.5t106.5 -33.5q53 0 93 22q48 27 48 76q0 69 -113 94l-81 18q-70 16 -103 43
-q-41 34 -41 95q0 73 63 112q53 33 131 33q81 0 132 -41q54 -44 57 -125zM206 588l-114 141h65l86 -102l88 102h58l-114 -141h-69z" />
-    <glyph glyph-name="Ydieresis" unicode="&#x178;" horiz-adv-x="611" 
-d="M271 0v294l-281 420h81l235 -362l234 362h81l-282 -420v-294h-68zM428 788h-68v101h68v-101zM251 788h-68v101h68v-101z" />
-    <glyph glyph-name="Zcaron" unicode="&#x17d;" horiz-adv-x="574" 
-d="M37 656v58h514v-60l-470 -596h480v-58h-556v60l470 596h-438zM253 771l-114 141h65l86 -102l88 102h58l-114 -141h-69z" />
-    <glyph glyph-name="zcaron" unicode="&#x17e;" horiz-adv-x="463" 
-d="M432 471l-341 -418h355v-53h-431v51l337 412h-314v53h394v-45zM197 588l-114 141h65l86 -102l88 102h58l-114 -141h-69z" />
-    <glyph glyph-name="florin" unicode="&#x192;" 
-d="M7 -168l9 54q28 -7 52 -6q71 2 92 112l75 394h-100l9 53h101l13 69q20 104 43 143q39 63 127 63q27 0 62 -5l-12 -52q-32 4 -46 4q-56 -1 -82 -52q-11 -24 -28 -103l-14 -67h113l-9 -53h-113l-61 -323q-24 -128 -44 -164q-41 -74 -142 -72q-22 0 -45 5z" />
-    <glyph glyph-name="circumflex" unicode="&#x2c6;" horiz-adv-x="185" 
-d="M127 729l114 -141h-65l-86 102l-88 -102h-58l114 141h69z" />
-    <glyph glyph-name="caron" unicode="&#x2c7;" horiz-adv-x="185" 
-d="M58 588l-114 141h65l86 -102l88 102h58l-114 -141h-69z" />
-    <glyph glyph-name="breve" unicode="&#x2d8;" horiz-adv-x="185" 
-d="M-51 722h39q2 -38 34 -61q29 -21 69 -21t70.5 22t35.5 60h39q-7 -58 -46.5 -92.5t-98.5 -34.5q-61 0 -99.5 33.5t-42.5 93.5z" />
-    <glyph glyph-name="dotaccent" unicode="&#x2d9;" horiz-adv-x="185" 
-d="M127 605h-68v101h68v-101z" />
-    <glyph glyph-name="ring" unicode="&#x2da;" horiz-adv-x="185" 
-d="M-14 677q0 44 31.5 75.5t75.5 31.5t75.5 -31.5t31.5 -75.5t-31.5 -75.5t-75.5 -31.5t-75.5 31.5t-31.5 75.5zM25 677q0 -28 20 -48t48 -20q29 0 48.5 20t19.5 48q0 29 -19.5 48.5t-48.5 19.5q-28 0 -48 -19.5t-20 -48.5z" />
-    <glyph glyph-name="ogonek" unicode="&#x2db;" horiz-adv-x="185" 
-d="M151 2h38q-89 -71 -89 -127q0 -47 44 -47q32 0 55 41l29 -13q-40 -67 -95 -67q-96 0 -96 82q0 63 114 131z" />
-    <glyph glyph-name="tilde" unicode="&#x2dc;" horiz-adv-x="185" 
-d="M211 708v0h38q-6 -39 -27 -65q-24 -30 -60 -30q-30 0 -76 23t-59 23q-53 0 -53 -51h-39q5 39 28.5 67.5t60.5 28.5q30 0 74.5 -22t60.5 -22q37 0 52 48z" />
-    <glyph glyph-name="hungarumlaut" unicode="&#x2dd;" horiz-adv-x="185" 
-d="M18 588h-51l101 141h76zM174 588h-51l101 141h76z" />
-    <glyph glyph-name="endash" unicode="&#x2013;" horiz-adv-x="500" 
-d="M500 246h-500v58h500v-58z" />
-    <glyph glyph-name="emdash" unicode="&#x2014;" horiz-adv-x="1000" 
-d="M1000 246h-1000v58h1000v-58z" />
-    <glyph glyph-name="quoteleft" unicode="&#x2018;" horiz-adv-x="278" 
-d="M179 471h-84v107q0 47 22 83q24 41 67 53v-43q-24 -8 -37 -39q-12 -27 -10 -55h42v-106z" />
-    <glyph glyph-name="quoteright" unicode="&#x2019;" horiz-adv-x="278" 
-d="M99 714h84v-107q0 -47 -22 -83q-24 -41 -67 -53v43q24 8 37 39q12 27 10 55h-42v106z" />
-    <glyph glyph-name="quotesinglbase" unicode="&#x201a;" horiz-adv-x="278" 
-d="M99 106h84v-107q0 -47 -22 -83q-24 -41 -67 -53v43q24 8 37 39q12 27 10 55h-42v106z" />
-    <glyph glyph-name="quotedblleft" unicode="&#x201c;" horiz-adv-x="370" 
-d="M302 471h-84v107q0 47 22 83q24 41 67 53v-43q-24 -8 -37 -39q-12 -27 -10 -55h42v-106zM148 471h-84v107q0 47 22 83q24 41 67 53v-43q-24 -8 -37 -39q-12 -27 -10 -55h42v-106z" />
-    <glyph glyph-name="quotedblright" unicode="&#x201d;" horiz-adv-x="370" 
-d="M222 714h84v-107q0 -47 -22 -83q-24 -41 -67 -53v43q24 8 37 39q12 27 10 55h-42v106zM68 714h84v-107q0 -47 -22 -83q-24 -41 -67 -53v43q24 8 37 39q12 27 10 55h-42v106z" />
-    <glyph glyph-name="quotedblbase" unicode="&#x201e;" horiz-adv-x="370" 
-d="M222 106h84v-107q0 -47 -22 -83q-24 -41 -67 -53v43q24 8 37 39q12 27 10 55h-42v106zM68 106h84v-107q0 -47 -22 -83q-24 -41 -67 -53v43q24 8 37 39q12 27 10 55h-42v106z" />
-    <glyph glyph-name="dagger" unicode="&#x2020;" 
-d="M246 440h-198v53h198v221h63v-221h199v-53h-199v-591h-63v591z" />
-    <glyph glyph-name="daggerdbl" unicode="&#x2021;" 
-d="M246 47h-198v53h198v363h-198v53h198v198h63v-198h199v-53h-199v-363h199v-53h-199v-198h-63v198z" />
-    <glyph glyph-name="bullet" unicode="&#x2022;" horiz-adv-x="500" 
-d="M72 357q0 74 52 126t126 52t126.5 -52t52.5 -126t-52.5 -126.5t-126.5 -52.5t-126 52.5t-52 126.5z" />
-    <glyph glyph-name="ellipsis" unicode="&#x2026;" horiz-adv-x="1000" 
-d="M209 0h-84v106h84v-106zM542 0h-84v106h84v-106zM875 0h-84v106h84v-106z" />
-    <glyph glyph-name="perthousand" unicode="&#x2030;" horiz-adv-x="1111" 
-d="M129 -34l453 759h47l-452 -759h-48zM547 323q149 0 149 -169t-149 -169t-149 169t149 169zM547 278q-96 0 -96 -124t96 -124t96 124t-96 124zM901 323q149 0 149 -169t-149 -169t-149 169t149 169zM901 278q-96 0 -96 -124t96 -124t96 124t-96 124zM214 706
-q149 0 149 -169t-149 -169t-149 169t149 169zM214 661q-96 0 -96 -124t96 -124t96 124t-96 124z" />
-    <glyph glyph-name="guilsinglleft" unicode="&#x2039;" horiz-adv-x="259" 
-d="M186 119l-133 125v62l133 125v-67l-94 -89l94 -89v-67z" />
-    <glyph glyph-name="guilsinglright" unicode="&#x203a;" horiz-adv-x="259" 
-d="M73 119v67l94 89l-94 89v67l133 -125v-62z" />
-    <glyph glyph-name="fraction" unicode="&#x2044;" horiz-adv-x="167" 
-d="M-166 -34l453 759h47l-452 -759h-48z" />
-    <glyph glyph-name="trademark" unicode="&#x2122;" horiz-adv-x="990" 
-d="M269 302h-53v367h-134v45h321v-45h-134v-367zM529 302h-53v412h85l133 -334l129 334h85v-412h-53v364h-2l-146 -364h-30l-146 364h-2v-364z" />
-    <glyph glyph-name="fi" unicode="&#xf001;" horiz-adv-x="444" 
-d="M320 0v516h63v-516h-63zM94 0v463h-88v53h88v47q0 74 25 109q29 42 100 42q26 0 53 -4v-54q-28 5 -46 5q-47 0 -61 -29q-8 -17 -8 -71v-45h102v-53h-102v-463h-63zM320 613v101h63v-101h-63z" />
-    <glyph glyph-name="fl" unicode="&#xf002;" horiz-adv-x="444" 
-d="M94 0v463h-88v53h88v47q0 74 25 109q29 42 100 42q26 0 53 -4v-54q-28 5 -46 5q-47 0 -61 -29q-8 -17 -8 -71v-45h102v-53h-102v-463h-63zM320 0v714h63v-714h-63z" />
-    <glyph glyph-name="glyph231" horiz-adv-x="278" 
- />
-  </font>
-</defs></svg>

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/assets/fonts/helveticaneue_lt.ttf
----------------------------------------------------------------------
diff --git a/website/assets/fonts/helveticaneue_lt.ttf b/website/assets/fonts/helveticaneue_lt.ttf
deleted file mode 100644
index 3b56b91..0000000
Binary files a/website/assets/fonts/helveticaneue_lt.ttf and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/assets/fonts/helveticaneue_lt.woff
----------------------------------------------------------------------
diff --git a/website/assets/fonts/helveticaneue_lt.woff b/website/assets/fonts/helveticaneue_lt.woff
deleted file mode 100644
index 1d3dbae..0000000
Binary files a/website/assets/fonts/helveticaneue_lt.woff and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/assets/fonts/opensans-light-webfont.eot
----------------------------------------------------------------------
diff --git a/website/assets/fonts/opensans-light-webfont.eot b/website/assets/fonts/opensans-light-webfont.eot
deleted file mode 100644
index 6860f95..0000000
Binary files a/website/assets/fonts/opensans-light-webfont.eot and /dev/null differ


[05/52] [abbrv] kylin git commit: KYLIN-1343 fix some Job unit test

Posted by li...@apache.org.
KYLIN-1343 fix some Job unit test


Project: http://git-wip-us.apache.org/repos/asf/kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/2bd3b4ed
Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/2bd3b4ed
Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/2bd3b4ed

Branch: refs/heads/1.x-HBase1.1.3
Commit: 2bd3b4ed8175da0aa95c3af86cbbc5cee6c2184b
Parents: d0f610f
Author: Yang Li <li...@apache.org>
Authored: Sun Jan 24 08:30:28 2016 +0800
Committer: Yang Li <li...@apache.org>
Committed: Sun Jan 24 08:30:28 2016 +0800

----------------------------------------------------------------------
 .../java/org/apache/kylin/job/hadoop/cube/BaseCuboidJobTest.java    | 1 +
 .../org/apache/kylin/job/hadoop/cube/KeyDistributionJobTest.java    | 1 +
 .../java/org/apache/kylin/job/hadoop/cube/MergeCuboidJobTest.java   | 1 +
 .../test/java/org/apache/kylin/job/hadoop/cube/NDCuboidJobTest.java | 1 +
 .../apache/kylin/job/hadoop/cube/RangeKeyDistributionJobTest.java   | 1 +
 .../java/org/apache/kylin/job/hadoop/hbase/CreateHTableTest.java    | 1 +
 .../java/org/apache/kylin/job/tools/ColumnCardinalityJobTest.java   | 1 +
 7 files changed, 7 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/2bd3b4ed/job/src/test/java/org/apache/kylin/job/hadoop/cube/BaseCuboidJobTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/org/apache/kylin/job/hadoop/cube/BaseCuboidJobTest.java b/job/src/test/java/org/apache/kylin/job/hadoop/cube/BaseCuboidJobTest.java
index 088409d..1775eae 100644
--- a/job/src/test/java/org/apache/kylin/job/hadoop/cube/BaseCuboidJobTest.java
+++ b/job/src/test/java/org/apache/kylin/job/hadoop/cube/BaseCuboidJobTest.java
@@ -39,6 +39,7 @@ public class BaseCuboidJobTest extends LocalFileMetadataTestCase {
         conf = new Configuration();
         conf.set("fs.default.name", "file:///");
         conf.set("mapred.job.tracker", "local");
+        conf.set("mapreduce.application.framework.path", "");
 
         // for local runner out-of-memory issue
         conf.set("mapreduce.task.io.sort.mb", "10");

http://git-wip-us.apache.org/repos/asf/kylin/blob/2bd3b4ed/job/src/test/java/org/apache/kylin/job/hadoop/cube/KeyDistributionJobTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/org/apache/kylin/job/hadoop/cube/KeyDistributionJobTest.java b/job/src/test/java/org/apache/kylin/job/hadoop/cube/KeyDistributionJobTest.java
index e7b5238..f5b3b7d 100644
--- a/job/src/test/java/org/apache/kylin/job/hadoop/cube/KeyDistributionJobTest.java
+++ b/job/src/test/java/org/apache/kylin/job/hadoop/cube/KeyDistributionJobTest.java
@@ -62,6 +62,7 @@ package org.apache.kylin.job.hadoop.cube;
 //        conf = new Configuration();
 //        conf.set("fs.default.name", "file:///");
 //        conf.set("mapred.job.tracker", "local");
+//        conf.set("mapreduce.application.framework.path", "");
 //    }
 //
 //    @Test

http://git-wip-us.apache.org/repos/asf/kylin/blob/2bd3b4ed/job/src/test/java/org/apache/kylin/job/hadoop/cube/MergeCuboidJobTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/org/apache/kylin/job/hadoop/cube/MergeCuboidJobTest.java b/job/src/test/java/org/apache/kylin/job/hadoop/cube/MergeCuboidJobTest.java
index cc8a756..012b3a1 100644
--- a/job/src/test/java/org/apache/kylin/job/hadoop/cube/MergeCuboidJobTest.java
+++ b/job/src/test/java/org/apache/kylin/job/hadoop/cube/MergeCuboidJobTest.java
@@ -40,6 +40,7 @@ public class MergeCuboidJobTest extends LocalFileMetadataTestCase {
         conf = new Configuration();
         conf.set("fs.default.name", "file:///");
         conf.set("mapred.job.tracker", "local");
+        conf.set("mapreduce.application.framework.path", "");
 
         // for local runner out-of-memory issue
         conf.set("mapreduce.task.io.sort.mb", "10");

http://git-wip-us.apache.org/repos/asf/kylin/blob/2bd3b4ed/job/src/test/java/org/apache/kylin/job/hadoop/cube/NDCuboidJobTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/org/apache/kylin/job/hadoop/cube/NDCuboidJobTest.java b/job/src/test/java/org/apache/kylin/job/hadoop/cube/NDCuboidJobTest.java
index eb5ef21..5ba3ec6 100644
--- a/job/src/test/java/org/apache/kylin/job/hadoop/cube/NDCuboidJobTest.java
+++ b/job/src/test/java/org/apache/kylin/job/hadoop/cube/NDCuboidJobTest.java
@@ -39,6 +39,7 @@ public class NDCuboidJobTest extends LocalFileMetadataTestCase {
         conf = new Configuration();
         conf.set("fs.default.name", "file:///");
         conf.set("mapred.job.tracker", "local");
+        conf.set("mapreduce.application.framework.path", "");
 
         // for local runner out-of-memory issue
         conf.set("mapreduce.task.io.sort.mb", "10");

http://git-wip-us.apache.org/repos/asf/kylin/blob/2bd3b4ed/job/src/test/java/org/apache/kylin/job/hadoop/cube/RangeKeyDistributionJobTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/org/apache/kylin/job/hadoop/cube/RangeKeyDistributionJobTest.java b/job/src/test/java/org/apache/kylin/job/hadoop/cube/RangeKeyDistributionJobTest.java
index 4de7504..586bdf0 100644
--- a/job/src/test/java/org/apache/kylin/job/hadoop/cube/RangeKeyDistributionJobTest.java
+++ b/job/src/test/java/org/apache/kylin/job/hadoop/cube/RangeKeyDistributionJobTest.java
@@ -43,6 +43,7 @@ public class RangeKeyDistributionJobTest extends LocalFileMetadataTestCase {
         conf = new Configuration();
         conf.set("fs.default.name", "file:///");
         conf.set("mapred.job.tracker", "local");
+        conf.set("mapreduce.application.framework.path", "");
 
         // for local runner out-of-memory issue
         conf.set("mapreduce.task.io.sort.mb", "10");

http://git-wip-us.apache.org/repos/asf/kylin/blob/2bd3b4ed/job/src/test/java/org/apache/kylin/job/hadoop/hbase/CreateHTableTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/org/apache/kylin/job/hadoop/hbase/CreateHTableTest.java b/job/src/test/java/org/apache/kylin/job/hadoop/hbase/CreateHTableTest.java
index 28d83b7..a87c66b 100644
--- a/job/src/test/java/org/apache/kylin/job/hadoop/hbase/CreateHTableTest.java
+++ b/job/src/test/java/org/apache/kylin/job/hadoop/hbase/CreateHTableTest.java
@@ -41,6 +41,7 @@ public class CreateHTableTest extends LocalFileMetadataTestCase {
         conf = new Configuration();
         conf.set("fs.default.name", "file:///");
         conf.set("mapred.job.tracker", "local");
+        conf.set("mapreduce.application.framework.path", "");
         this.createTestMetadata();
 
     }

http://git-wip-us.apache.org/repos/asf/kylin/blob/2bd3b4ed/job/src/test/java/org/apache/kylin/job/tools/ColumnCardinalityJobTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/org/apache/kylin/job/tools/ColumnCardinalityJobTest.java b/job/src/test/java/org/apache/kylin/job/tools/ColumnCardinalityJobTest.java
index 8032546..9d95a88 100644
--- a/job/src/test/java/org/apache/kylin/job/tools/ColumnCardinalityJobTest.java
+++ b/job/src/test/java/org/apache/kylin/job/tools/ColumnCardinalityJobTest.java
@@ -45,6 +45,7 @@ public class ColumnCardinalityJobTest {
         conf = new Configuration();
         conf.set("fs.default.name", "file:///");
         conf.set("mapred.job.tracker", "local");
+        conf.set("mapreduce.application.framework.path", "");
     }
 
     @Test


[16/52] [abbrv] kylin git commit: KYLIN-1372 fix CI

Posted by li...@apache.org.
KYLIN-1372 fix CI


Project: http://git-wip-us.apache.org/repos/asf/kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/555a2e56
Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/555a2e56
Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/555a2e56

Branch: refs/heads/1.x-HBase1.1.3
Commit: 555a2e5667837d6bdf16460a34c88f3f5130bf31
Parents: 75970c3
Author: honma <ho...@ebay.com>
Authored: Mon Feb 1 13:48:06 2016 +0800
Committer: honma <ho...@ebay.com>
Committed: Mon Feb 1 13:48:06 2016 +0800

----------------------------------------------------------------------
 query/src/test/resources/query/sql_dynamic/query03.sql | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/555a2e56/query/src/test/resources/query/sql_dynamic/query03.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_dynamic/query03.sql b/query/src/test/resources/query/sql_dynamic/query03.sql
index 31d5d0b..3187c33 100644
--- a/query/src/test/resources/query/sql_dynamic/query03.sql
+++ b/query/src/test/resources/query/sql_dynamic/query03.sql
@@ -17,7 +17,7 @@
 --
 
 select test_cal_dt.week_beg_dt,sum(test_kylin_fact.price) as GMV
- , count(1) as TRANS_CNT, count(distinct leaf_categ_id) as LEAF_CATEG_CNT
+ , count(1) as TRANS_CNT
  from test_kylin_fact
  inner JOIN edw.test_cal_dt as test_cal_dt
  ON test_kylin_fact.cal_dt = test_cal_dt.cal_dt


[25/52] [abbrv] kylin git commit: document, enhance How to Release a bit

Posted by li...@apache.org.
document, enhance How to Release a bit


Project: http://git-wip-us.apache.org/repos/asf/kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/fc66cdb5
Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/fc66cdb5
Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/fc66cdb5

Branch: refs/heads/1.x-HBase1.1.3
Commit: fc66cdb50116e06172d9d0943dfc181a8b2d1f1c
Parents: 7d3c4f4
Author: Yang Li <li...@apache.org>
Authored: Tue Feb 9 22:10:17 2016 +0800
Committer: Yang Li <li...@apache.org>
Committed: Tue Feb 9 22:10:17 2016 +0800

----------------------------------------------------------------------
 website/_dev/howto_release.md | 31 +++++++++++++------------------
 1 file changed, 13 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/fc66cdb5/website/_dev/howto_release.md
----------------------------------------------------------------------
diff --git a/website/_dev/howto_release.md b/website/_dev/howto_release.md
index f11425b..cad3901 100644
--- a/website/_dev/howto_release.md
+++ b/website/_dev/howto_release.md
@@ -34,8 +34,9 @@ Verify your key:
 Get the fingerprint of your key:
 `gpg --fingerprint YOUR_NAME`
 
-It will display the fingerprint like "Key fingerprint = XXXX XXXX ...", then add the fingerprint to your apache account at https://id.apache.org/ in "OpenPGP Public Key Primary Fingerprint" field; wait for a while the key will added to https://people.apache.org/keys/, for example:  
-[https://people.apache.org/keys/committer/lukehan.asc](https://people.apache.org/keys/committer/lukehan.asc)  
+It will display the fingerprint like "Key fingerprint = XXXX XXXX ...", then add the fingerprint to your apache account at [https://id.apache.org/](https://id.apache.org/) in "OpenPGP Public Key Primary Fingerprint" field; wait for a few hours the key will added to [https://people.apache.org/keys/](https://people.apache.org/keys/), for example:
+[https://people.apache.org/keys/committer/lukehan.asc](https://people.apache.org/keys/committer/lukehan.asc)
+
 Generate ASCII Amromed Key:  
 `gpg -a --export YOUR_MAIL_ADDRESS > YOUR_NAME.asc &`
 
@@ -43,10 +44,10 @@ Upload key to public server:
 `gpg --send-keys YOUR_KEY_HASH`
 
 or Submit key via web:  
-Open and Submit to [http://pool.sks-keyservers.net:11371](http://pool.sks-keyservers.net:11371) (you can pickup any avaliable public key server)  
+Open and Submit to [http://pool.sks-keyservers.net:11371](http://pool.sks-keyservers.net:11371) (you can pickup any avaliable public key server)
+
 Once your key submitted to server, you can verify using following command:  
 `gpg --recv-keys YOUR_KEY_HASH`
-
 for example:  
 `gpg --recv-keys 027DC364`
 
@@ -65,14 +66,6 @@ __Before you start:__
 * Make sure you are working on right release version number.
 * Make sure that every “resolved” JIRA case (including duplicates) has a fix version assigned.
 
-__Verify licenses__  
-Run Apache RAT to check licenses issue:  
-{% highlight bash %}
-mvn -Papache-release clean rat:rat
-{% endhighlight %}
-
-Fix license issue if any.
-
 __Configure Apache repository server in Maven__
 If you're the first time to do release, you need update the server authentication information in ~/.m2/settings.xml; If this file doesn't exist, copy a template from $M2_HOME/conf/settings.xml;
 
@@ -119,7 +112,7 @@ In the "servers" section, make sure the following servers be added, and replace
   </servers>
 {% endhighlight %}
 
-__Making a snapshot__  
+__Fix license issues and make a snapshot__
 {% highlight bash %}
 # Set passphrase variable without putting it into shell history
 $ read -s GPG_PASSPHRASE
@@ -128,9 +121,11 @@ $ read -s GPG_PASSPHRASE
 $ git clean -xn
 $ mvn clean
 
+# Fix any license issues as prompted
 $ mvn -Papache-release -Dgpg.passphrase=${GPG_PASSPHRASE} install
 {% endhighlight %}
-When the dry-run has succeeded, change install to deploy:
+
+Optionally, when the dry-run has succeeded, change install to deploy:
 {% highlight bash %}
 $ mvn -Papache-release -Dgpg.passphrase=${GPG_PASSPHRASE} deploy
 {% endhighlight %}
@@ -187,14 +182,14 @@ $ mvn -DskipTests -DreleaseVersion=X.Y.Z -DdevelopmentVersion=(X.Y.Z+1)-SNAPSHOT
 $ mvn -DskipTests -Papache-release -Darguments="-Dgpg.passphrase=${GPG_PASSPHRASE} -DskipTests" release:perform
 {% endhighlight %}
 
-__Verify the staged artifacts in the Nexus repository:__  
+__Close the staged artifacts in the Nexus repository:__
+
 * Go to [https://repository.apache.org/](https://repository.apache.org/) and login
 * Under `Build Promotion`, click `Staging Repositories`
 * In the `Staging Repositories` tab there should be a line with profile `org.apache.kylin`
 * Navigate through the artifact tree and make sure the .jar, .pom, .asc files are present
-* Check the box on in the first column of the row,
-  and press the 'Close' button to publish the repository at
-  https://repository.apache.org/content/repositories/orgapachekylin-1006
+* Check the box on in the first column of the row, and press the 'Close' button to publish the repository at
+  [https://repository.apache.org/content/repositories/orgapachekylin-1006](https://repository.apache.org/content/repositories/orgapachekylin-1006)
   (or a similar URL)
 
 __Upload to staging area:__  


[30/52] [abbrv] kylin git commit: KYLIN-1416 remove website from coding branch

Posted by li...@apache.org.
http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/assets/fonts/opensans-regular-webfont.svg
----------------------------------------------------------------------
diff --git a/website/assets/fonts/opensans-regular-webfont.svg b/website/assets/fonts/opensans-regular-webfont.svg
deleted file mode 100644
index a169e01..0000000
--- a/website/assets/fonts/opensans-regular-webfont.svg
+++ /dev/null
@@ -1,1824 +0,0 @@
-<?xml version="1.0" standalone="no"?>
-<!DOCTYPE svg PUBLIC "-//W3C//DTD SVG 1.1//EN" "http://www.w3.org/Graphics/SVG/1.1/DTD/svg11.dtd" >
-<svg xmlns="http://www.w3.org/2000/svg">
-<metadata></metadata>
-<defs>
-<font id="open_sansregular" horiz-adv-x="1171" >
-<font-face units-per-em="2048" ascent="1638" descent="-410" />
-<missing-glyph horiz-adv-x="532" />
-<glyph unicode="&#xfb01;" horiz-adv-x="1212" d="M29 967v75l196 60v61q0 404 353 404q87 0 204 -35l-43 -133q-96 31 -164 31q-94 0 -139 -62.5t-45 -200.5v-71h279v-129h-279v-967h-166v967h-196zM856 1393q0 57 28 83.5t70 26.5q40 0 69 -27t29 -83t-29 -83.5t-69 -27.5q-42 0 -70 27.5t-28 83.5zM870 0v1096h166 v-1096h-166z" />
-<glyph unicode="&#xfb02;" horiz-adv-x="1212" d="M29 967v75l196 60v61q0 404 353 404q87 0 204 -35l-43 -133q-96 31 -164 31q-94 0 -139 -62.5t-45 -200.5v-71h279v-129h-279v-967h-166v967h-196zM870 0v1556h166v-1556h-166z" />
-<glyph unicode="&#xfb03;" horiz-adv-x="1909" d="M29 967v75l196 60v61q0 404 353 404q87 0 204 -35l-43 -133q-96 31 -164 31q-94 0 -139 -62.5t-45 -200.5v-71h279v-129h-279v-967h-166v967h-196zM717 967v75l196 60v61q0 404 353 404q87 0 204 -35l-43 -133q-96 31 -164 31q-94 0 -139 -62.5t-45 -200.5v-71h279v-129 h-279v-967h-166v967h-196zM1551 1393q0 57 28 83.5t70 26.5q40 0 69 -27t29 -83t-29 -83.5t-69 -27.5q-42 0 -70 27.5t-28 83.5zM1565 0v1096h166v-1096h-166z" />
-<glyph unicode="&#xfb04;" horiz-adv-x="1909" d="M29 967v75l196 60v61q0 404 353 404q87 0 204 -35l-43 -133q-96 31 -164 31q-94 0 -139 -62.5t-45 -200.5v-71h279v-129h-279v-967h-166v967h-196zM717 967v75l196 60v61q0 404 353 404q87 0 204 -35l-43 -133q-96 31 -164 31q-94 0 -139 -62.5t-45 -200.5v-71h279v-129 h-279v-967h-166v967h-196zM1565 0v1556h166v-1556h-166z" />
-<glyph horiz-adv-x="0" />
-<glyph unicode="&#xd;" horiz-adv-x="1044" />
-<glyph unicode=" "  horiz-adv-x="532" />
-<glyph unicode="&#x09;" horiz-adv-x="532" />
-<glyph unicode="&#xa0;" horiz-adv-x="532" />
-<glyph unicode="!" horiz-adv-x="547" d="M152 106q0 136 120 136q58 0 89.5 -35t31.5 -101q0 -64 -32 -99.5t-89 -35.5q-52 0 -86 31.5t-34 103.5zM170 1462h207l-51 -1059h-105z" />
-<glyph unicode="&#x22;" horiz-adv-x="821" d="M133 1462h186l-40 -528h-105zM502 1462h186l-41 -528h-104z" />
-<glyph unicode="#" horiz-adv-x="1323" d="M51 430v129h287l68 340h-277v127h299l82 436h139l-82 -436h305l84 436h134l-84 -436h264v-127h-289l-66 -340h283v-129h-307l-84 -430h-137l84 430h-303l-82 -430h-136l80 430h-262zM475 559h303l66 340h-303z" />
-<glyph unicode="$" d="M131 170v156q83 -37 191.5 -60.5t197.5 -23.5v440q-205 65 -287.5 151t-82.5 222q0 131 101.5 215t268.5 102v182h129v-180q184 -5 355 -74l-52 -131q-149 59 -303 70v-434q157 -50 235 -97.5t115 -109t37 -149.5q0 -136 -102 -224.5t-285 -111.5v-232h-129v223 q-112 0 -217 17.5t-172 48.5zM319 1057q0 -76 45 -122t156 -87v387q-99 -16 -150 -62.5t-51 -115.5zM649 252q217 30 217 184q0 72 -44.5 116.5t-172.5 88.5v-389z" />
-<glyph unicode="%" horiz-adv-x="1686" d="M104 1026q0 227 74.5 342t220.5 115q145 0 223 -119t78 -338q0 -228 -76.5 -344.5t-224.5 -116.5q-140 0 -217.5 119t-77.5 342zM242 1026q0 -170 37 -255t120 -85q164 0 164 340q0 338 -164 338q-83 0 -120 -84t-37 -254zM365 0l811 1462h147l-811 -1462h-147zM985 440 q0 227 74.5 342t220.5 115q142 0 221.5 -117.5t79.5 -339.5q0 -227 -76.5 -343.5t-224.5 -116.5q-142 0 -218.5 119t-76.5 341zM1122 440q0 -171 37 -255.5t121 -84.5t124 83.5t40 256.5q0 171 -40 253.5t-124 82.5t-121 -82.5t-37 -253.5z" />
-<glyph unicode="&#x26;" horiz-adv-x="1495" d="M113 379q0 130 69.5 230t249.5 202q-85 95 -115.5 144t-48.5 102t-18 110q0 150 98 234t273 84q162 0 255 -83.5t93 -232.5q0 -107 -68 -197.5t-225 -183.5l407 -391q56 62 89.5 145.5t56.5 182.5h168q-68 -286 -205 -434l299 -291h-229l-185 178q-118 -106 -240 -152 t-272 -46q-215 0 -333.5 106t-118.5 293zM285 383q0 -117 77.5 -185.5t206.5 -68.5q241 0 400 154l-437 424q-111 -68 -157 -112.5t-68 -95.5t-22 -116zM414 1171q0 -69 36 -131.5t123 -150.5q129 75 179.5 138.5t50.5 146.5q0 77 -51.5 125.5t-137.5 48.5q-89 0 -144.5 -48 t-55.5 -129z" />
-<glyph unicode="'" horiz-adv-x="453" d="M133 1462h186l-40 -528h-105z" />
-<glyph unicode="(" horiz-adv-x="606" d="M82 561q0 265 77.5 496t223.5 405h162q-144 -193 -216.5 -424t-72.5 -475q0 -240 74 -469t213 -418h-160q-147 170 -224 397t-77 488z" />
-<glyph unicode=")" horiz-adv-x="606" d="M61 1462h162q147 -175 224 -406.5t77 -494.5t-77.5 -490t-223.5 -395h-160q139 188 213 417.5t74 469.5q0 244 -72.5 475t-216.5 424z" />
-<glyph unicode="*" horiz-adv-x="1130" d="M86 1090l29 182l391 -111l-43 395h194l-43 -395l398 111l26 -182l-381 -31l248 -326l-172 -94l-176 362l-160 -362l-176 94l242 326z" />
-<glyph unicode="+" d="M104 653v138h410v428h139v-428h412v-138h-412v-426h-139v426h-410z" />
-<glyph unicode="," horiz-adv-x="502" d="M63 -264q27 104 59.5 257t45.5 245h182l15 -23q-26 -100 -75 -232.5t-102 -246.5h-125z" />
-<glyph unicode="-" horiz-adv-x="659" d="M84 473v152h491v-152h-491z" />
-<glyph unicode="." horiz-adv-x="545" d="M152 106q0 67 30.5 101.5t87.5 34.5q58 0 90.5 -34.5t32.5 -101.5q0 -65 -33 -100t-90 -35q-51 0 -84.5 31.5t-33.5 103.5z" />
-<glyph unicode="/" horiz-adv-x="752" d="M20 0l545 1462h166l-545 -1462h-166z" />
-<glyph unicode="0" d="M102 733q0 382 119 567t363 185q238 0 361.5 -193t123.5 -559q0 -379 -119.5 -566t-365.5 -187q-236 0 -359 191.5t-123 561.5zM270 733q0 -319 75 -464.5t239 -145.5q166 0 240.5 147.5t74.5 462.5t-74.5 461.5t-240.5 146.5q-164 0 -239 -144.5t-75 -463.5z" />
-<glyph unicode="1" d="M188 1163l387 299h140v-1462h-162v1042q0 130 8 246q-21 -21 -47 -44t-238 -195z" />
-<glyph unicode="2" d="M100 0v143l385 387q176 178 232 254t84 148t28 155q0 117 -71 185.5t-197 68.5q-91 0 -172.5 -30t-181.5 -109l-88 113q202 168 440 168q206 0 323 -105.5t117 -283.5q0 -139 -78 -275t-292 -344l-320 -313v-8h752v-154h-961z" />
-<glyph unicode="3" d="M94 59v158q95 -47 202.5 -71.5t203.5 -24.5q379 0 379 297q0 266 -418 266h-144v143h146q171 0 271 75.5t100 209.5q0 107 -73.5 168t-199.5 61q-96 0 -181 -26t-194 -96l-84 112q90 71 207.5 111.5t247.5 40.5q213 0 331 -97.5t118 -267.5q0 -140 -78.5 -229 t-222.5 -119v-8q176 -22 261 -112t85 -236q0 -209 -145 -321.5t-412 -112.5q-116 0 -212.5 17.5t-187.5 61.5z" />
-<glyph unicode="4" d="M43 336v145l694 989h176v-983h217v-151h-217v-336h-159v336h-711zM209 487h545v486q0 143 10 323h-8q-48 -96 -90 -159z" />
-<glyph unicode="5" d="M133 59v160q70 -45 174 -70.5t205 -25.5q176 0 273.5 83t97.5 240q0 306 -375 306q-95 0 -254 -29l-86 55l55 684h727v-153h-585l-37 -439q115 23 229 23q231 0 363.5 -114.5t132.5 -313.5q0 -227 -144.5 -356t-398.5 -129q-247 0 -377 79z" />
-<glyph unicode="6" d="M117 625q0 431 167.5 644.5t495.5 213.5q113 0 178 -19v-143q-77 25 -176 25q-235 0 -359 -146.5t-136 -460.5h12q110 172 348 172q197 0 310.5 -119t113.5 -323q0 -228 -124.5 -358.5t-336.5 -130.5q-227 0 -360 170.5t-133 474.5zM287 506q0 -103 40 -192t113.5 -141 t167.5 -52q142 0 220.5 89.5t78.5 258.5q0 145 -73 228t-218 83q-90 0 -165 -37t-119.5 -102t-44.5 -135z" />
-<glyph unicode="7" d="M94 1309v153h973v-133l-598 -1329h-184l606 1309h-797z" />
-<glyph unicode="8" d="M104 373q0 251 306 391q-138 78 -198 168.5t-60 202.5q0 159 117.5 253.5t314.5 94.5q200 0 317 -93t117 -257q0 -108 -67 -197t-214 -162q178 -85 253 -178.5t75 -216.5q0 -182 -127 -290.5t-348 -108.5q-234 0 -360 102.5t-126 290.5zM268 369q0 -120 83.5 -187 t234.5 -67q149 0 232 70t83 192q0 97 -78 172.5t-272 146.5q-149 -64 -216 -141.5t-67 -185.5zM315 1128q0 -92 59 -158t218 -132q143 60 202.5 129t59.5 161q0 101 -72.5 160.5t-199.5 59.5q-125 0 -196 -60t-71 -160z" />
-<glyph unicode="9" d="M106 991q0 228 127.5 360t335.5 132q149 0 260.5 -76.5t171.5 -223t60 -345.5q0 -858 -664 -858q-116 0 -184 20v143q80 -26 182 -26q240 0 362.5 148.5t133.5 455.5h-12q-55 -83 -146 -126.5t-205 -43.5q-194 0 -308 116t-114 324zM270 993q0 -144 72 -226.5t219 -82.5 q91 0 167.5 37t120.5 101t44 134q0 105 -41 194t-114.5 140t-168.5 51q-143 0 -221 -92t-78 -256z" />
-<glyph unicode=":" horiz-adv-x="545" d="M152 106q0 67 30.5 101.5t87.5 34.5q58 0 90.5 -34.5t32.5 -101.5q0 -65 -33 -100t-90 -35q-51 0 -84.5 31.5t-33.5 103.5zM152 989q0 135 118 135q123 0 123 -135q0 -65 -33 -100t-90 -35q-51 0 -84.5 31.5t-33.5 103.5z" />
-<glyph unicode=";" horiz-adv-x="545" d="M63 -264q27 104 59.5 257t45.5 245h182l15 -23q-26 -100 -75 -232.5t-102 -246.5h-125zM147 989q0 135 119 135q123 0 123 -135q0 -65 -33 -100t-90 -35q-58 0 -88.5 35t-30.5 100z" />
-<glyph unicode="&#x3c;" d="M104 664v98l961 479v-149l-782 -371l782 -328v-151z" />
-<glyph unicode="=" d="M119 449v137h930v-137h-930zM119 858v137h930v-137h-930z" />
-<glyph unicode="&#x3e;" d="M104 242v151l783 326l-783 373v149l961 -479v-98z" />
-<glyph unicode="?" horiz-adv-x="879" d="M27 1384q189 99 395 99q191 0 297 -94t106 -265q0 -73 -19.5 -128.5t-57.5 -105t-164 -159.5q-101 -86 -133.5 -143t-32.5 -152v-33h-129v54q0 117 36 192.5t134 159.5q136 115 171.5 173t35.5 140q0 102 -65.5 157.5t-188.5 55.5q-79 0 -154 -18.5t-172 -67.5zM240 106 q0 136 120 136q58 0 89.5 -35t31.5 -101q0 -64 -32 -99.5t-89 -35.5q-52 0 -86 31.5t-34 103.5z" />
-<glyph unicode="@" horiz-adv-x="1841" d="M121 571q0 260 107 463t305 314.5t454 111.5q215 0 382.5 -90.5t259 -257t91.5 -383.5q0 -142 -44 -260t-124 -183t-184 -65q-86 0 -145 52t-70 133h-8q-40 -87 -114.5 -136t-176.5 -49q-150 0 -234.5 102.5t-84.5 278.5q0 204 118 331.5t310 127.5q68 0 154 -12.5 t155 -34.5l-25 -470v-22q0 -178 133 -178q91 0 148 107.5t57 279.5q0 181 -74 317t-210.5 209.5t-313.5 73.5q-223 0 -388 -92.5t-252 -264t-87 -396.5q0 -305 161 -469t464 -164q210 0 436 86v-133q-192 -84 -436 -84q-363 0 -563.5 199.5t-200.5 557.5zM686 598 q0 -254 195 -254q207 0 225 313l14 261q-72 20 -157 20q-130 0 -203.5 -90t-73.5 -250z" />
-<glyph unicode="A" horiz-adv-x="1296" d="M0 0l578 1468h143l575 -1468h-176l-182 465h-586l-180 -465h-172zM412 618h473l-170 453q-33 86 -68 211q-22 -96 -63 -211z" />
-<glyph unicode="B" horiz-adv-x="1327" d="M201 0v1462h413q291 0 421 -87t130 -275q0 -130 -72.5 -214.5t-211.5 -109.5v-10q333 -57 333 -350q0 -196 -132.5 -306t-370.5 -110h-510zM371 145h305q177 0 266.5 68.5t89.5 214.5q0 136 -91.5 200t-278.5 64h-291v-547zM371 836h280q180 0 259 56.5t79 190.5 q0 123 -88 177.5t-280 54.5h-250v-479z" />
-<glyph unicode="C" horiz-adv-x="1292" d="M125 733q0 226 84.5 396t244 262t375.5 92q230 0 402 -84l-72 -146q-166 78 -332 78q-241 0 -380.5 -160.5t-139.5 -439.5q0 -287 134.5 -443.5t383.5 -156.5q153 0 349 55v-149q-152 -57 -375 -57q-323 0 -498.5 196t-175.5 557z" />
-<glyph unicode="D" horiz-adv-x="1493" d="M201 0v1462h448q341 0 530 -189t189 -528q0 -362 -196.5 -553.5t-565.5 -191.5h-405zM371 147h207q304 0 457 149.5t153 442.5q0 286 -143.5 431t-426.5 145h-247v-1168z" />
-<glyph unicode="E" horiz-adv-x="1139" d="M201 0v1462h815v-151h-645v-471h606v-150h-606v-538h645v-152h-815z" />
-<glyph unicode="F" horiz-adv-x="1057" d="M201 0v1462h815v-151h-645v-535h606v-151h-606v-625h-170z" />
-<glyph unicode="G" horiz-adv-x="1491" d="M125 731q0 228 91.5 399.5t263.5 262t403 90.5q234 0 436 -86l-66 -150q-198 84 -381 84q-267 0 -417 -159t-150 -441q0 -296 144.5 -449t424.5 -153q152 0 297 35v450h-327v152h497v-711q-116 -37 -236 -56t-278 -19q-332 0 -517 197.5t-185 553.5z" />
-<glyph unicode="H" horiz-adv-x="1511" d="M201 0v1462h170v-622h770v622h170v-1462h-170v688h-770v-688h-170z" />
-<glyph unicode="I" horiz-adv-x="571" d="M201 0v1462h170v-1462h-170z" />
-<glyph unicode="J" horiz-adv-x="547" d="M-160 -213q71 -20 148 -20q99 0 150.5 60t51.5 173v1462h170v-1448q0 -190 -96 -294.5t-276 -104.5q-94 0 -148 27v145z" />
-<glyph unicode="K" horiz-adv-x="1257" d="M201 0v1462h170v-725l663 725h201l-588 -635l610 -827h-200l-533 709l-153 -136v-573h-170z" />
-<glyph unicode="L" horiz-adv-x="1063" d="M201 0v1462h170v-1308h645v-154h-815z" />
-<glyph unicode="M" horiz-adv-x="1849" d="M201 0v1462h256l463 -1206h8l467 1206h254v-1462h-170v942q0 162 14 352h-8l-500 -1294h-137l-496 1296h-8q14 -154 14 -366v-930h-157z" />
-<glyph unicode="N" horiz-adv-x="1544" d="M201 0v1462h192l797 -1222h8q-2 28 -9 174q-5 114 -5 177v32v839h159v-1462h-194l-799 1227h-8q16 -216 16 -396v-831h-157z" />
-<glyph unicode="O" horiz-adv-x="1595" d="M125 735q0 357 176 553.5t500 196.5q315 0 492 -200t177 -552q0 -351 -177.5 -552t-493.5 -201q-323 0 -498.5 197.5t-175.5 557.5zM305 733q0 -297 126.5 -450.5t367.5 -153.5q243 0 367 153t124 451q0 295 -123.5 447.5t-365.5 152.5q-243 0 -369.5 -153.5 t-126.5 -446.5z" />
-<glyph unicode="P" horiz-adv-x="1233" d="M201 0v1462h379q548 0 548 -426q0 -222 -151.5 -341.5t-433.5 -119.5h-172v-575h-170zM371 721h153q226 0 327 73t101 234q0 145 -95 216t-296 71h-190v-594z" />
-<glyph unicode="Q" horiz-adv-x="1595" d="M125 735q0 357 176 553.5t500 196.5q315 0 492 -200t177 -552q0 -281 -113 -467t-319 -252l348 -362h-247l-285 330l-55 -2q-323 0 -498.5 197.5t-175.5 557.5zM305 733q0 -297 126.5 -450.5t367.5 -153.5q243 0 367 153t124 451q0 295 -123.5 447.5t-365.5 152.5 q-243 0 -369.5 -153.5t-126.5 -446.5z" />
-<glyph unicode="R" horiz-adv-x="1266" d="M201 0v1462h401q269 0 397.5 -103t128.5 -310q0 -290 -294 -392l397 -657h-201l-354 608h-305v-608h-170zM371 754h233q180 0 264 71.5t84 214.5q0 145 -85.5 209t-274.5 64h-221v-559z" />
-<glyph unicode="S" horiz-adv-x="1124" d="M106 47v164q90 -38 196 -60t210 -22q170 0 256 64.5t86 179.5q0 76 -30.5 124.5t-102 89.5t-217.5 93q-204 73 -291.5 173t-87.5 261q0 169 127 269t336 100q218 0 401 -80l-53 -148q-181 76 -352 76q-135 0 -211 -58t-76 -161q0 -76 28 -124.5t94.5 -89t203.5 -89.5 q230 -82 316.5 -176t86.5 -244q0 -193 -140 -301t-380 -108q-260 0 -400 67z" />
-<glyph unicode="T" horiz-adv-x="1133" d="M18 1311v151h1096v-151h-463v-1311h-170v1311h-463z" />
-<glyph unicode="U" horiz-adv-x="1491" d="M186 520v942h170v-954q0 -183 100 -281t294 -98q185 0 285 98.5t100 282.5v952h170v-946q0 -250 -151 -393t-415 -143t-408.5 144t-144.5 396z" />
-<glyph unicode="V" horiz-adv-x="1219" d="M0 1462h180l336 -946q58 -163 92 -317q36 162 94 323l334 940h183l-527 -1462h-168z" />
-<glyph unicode="W" horiz-adv-x="1896" d="M27 1462h180l231 -903q48 -190 70 -344q27 183 80 358l262 889h180l275 -897q48 -155 81 -350q19 142 72 346l230 901h180l-391 -1462h-168l-295 979q-21 65 -47 164t-27 119q-22 -132 -70 -289l-286 -973h-168z" />
-<glyph unicode="X" horiz-adv-x="1182" d="M8 0l486 764l-453 698h188l363 -579l366 579h181l-453 -692l488 -770h-193l-393 643l-400 -643h-180z" />
-<glyph unicode="Y" horiz-adv-x="1147" d="M0 1462h186l387 -731l390 731h184l-488 -895v-567h-172v559z" />
-<glyph unicode="Z" horiz-adv-x="1169" d="M82 0v133l776 1176h-752v153h959v-133l-776 -1175h798v-154h-1005z" />
-<glyph unicode="[" horiz-adv-x="674" d="M166 -324v1786h457v-141h-289v-1503h289v-142h-457z" />
-<glyph unicode="\" horiz-adv-x="752" d="M23 1462h163l547 -1462h-166z" />
-<glyph unicode="]" horiz-adv-x="674" d="M51 -182h289v1503h-289v141h457v-1786h-457v142z" />
-<glyph unicode="^" horiz-adv-x="1110" d="M49 551l434 922h99l477 -922h-152l-372 745l-334 -745h-152z" />
-<glyph unicode="_" horiz-adv-x="918" d="M-4 -184h926v-131h-926v131z" />
-<glyph unicode="`" horiz-adv-x="1182" d="M393 1548v21h203q32 -69 89 -159.5t101 -143.5v-25h-110q-65 52 -154 148t-129 159z" />
-<glyph unicode="a" horiz-adv-x="1139" d="M94 303q0 332 531 348l186 6v68q0 129 -55.5 190.5t-177.5 61.5q-137 0 -310 -84l-51 127q81 44 177.5 69t193.5 25q196 0 290.5 -87t94.5 -279v-748h-123l-33 156h-8q-82 -103 -163.5 -139.5t-203.5 -36.5q-163 0 -255.5 84t-92.5 239zM268 301q0 -90 54.5 -137 t152.5 -47q155 0 243.5 85t88.5 238v99l-166 -7q-198 -7 -285.5 -61.5t-87.5 -169.5z" />
-<glyph unicode="b" horiz-adv-x="1255" d="M176 0v1556h166v-378q0 -127 -8 -228h8q116 164 344 164q216 0 335.5 -147.5t119.5 -417.5t-120.5 -419.5t-334.5 -149.5q-107 0 -195.5 39.5t-148.5 121.5h-12l-35 -141h-119zM342 549q0 -231 77 -330.5t247 -99.5q153 0 228 111.5t75 320.5q0 214 -75 319t-232 105 q-170 0 -245 -97.5t-75 -328.5z" />
-<glyph unicode="c" horiz-adv-x="975" d="M115 541q0 275 132.5 425t377.5 150q79 0 158 -17t124 -40l-51 -141q-55 22 -120 36.5t-115 14.5q-334 0 -334 -426q0 -202 81.5 -310t241.5 -108q137 0 281 59v-147q-110 -57 -277 -57q-238 0 -368.5 146.5t-130.5 414.5z" />
-<glyph unicode="d" horiz-adv-x="1255" d="M115 545q0 271 120 421t334 150q223 0 342 -162h13l-7 79l-4 77v446h166v-1556h-135l-22 147h-9q-115 -167 -344 -167q-215 0 -334.5 147t-119.5 418zM287 543q0 -210 77 -317t226 -107q170 0 246.5 92.5t76.5 298.5v35q0 233 -77.5 332.5t-247.5 99.5 q-146 0 -223.5 -113.5t-77.5 -320.5z" />
-<glyph unicode="e" horiz-adv-x="1149" d="M115 539q0 265 130.5 421t350.5 156q206 0 326 -135.5t120 -357.5v-105h-755q5 -193 97.5 -293t260.5 -100q177 0 350 74v-148q-88 -38 -166.5 -54.5t-189.5 -16.5q-243 0 -383.5 148t-140.5 411zM291 653h573q0 157 -70 240.5t-200 83.5q-132 0 -210.5 -86t-92.5 -238z " />
-<glyph unicode="f" horiz-adv-x="694" d="M29 967v75l196 60v61q0 404 353 404q87 0 204 -35l-43 -133q-96 31 -164 31q-94 0 -139 -62.5t-45 -200.5v-71h279v-129h-279v-967h-166v967h-196z" />
-<glyph unicode="g" horiz-adv-x="1122" d="M39 -186q0 100 64 173t180 99q-42 19 -70.5 59t-28.5 93q0 60 32 105t101 87q-85 35 -138.5 119t-53.5 192q0 180 108 277.5t306 97.5q86 0 155 -20h379v-105l-203 -24q28 -35 50 -91.5t22 -127.5q0 -161 -110 -257t-302 -96q-49 0 -92 8q-106 -56 -106 -141 q0 -45 37 -66.5t127 -21.5h194q178 0 273.5 -75t95.5 -218q0 -182 -146 -277.5t-426 -95.5q-215 0 -331.5 80t-116.5 226zM199 -184q0 -89 75 -135t215 -46q209 0 309.5 62.5t100.5 169.5q0 89 -55 123.5t-207 34.5h-199q-113 0 -176 -54t-63 -155zM289 745q0 -115 65 -174 t181 -59q243 0 243 236q0 247 -246 247q-117 0 -180 -63t-63 -187z" />
-<glyph unicode="h" horiz-adv-x="1257" d="M176 0v1556h166v-471q0 -85 -8 -141h10q49 79 139.5 124.5t206.5 45.5q201 0 301.5 -95.5t100.5 -303.5v-715h-166v709q0 134 -61 200t-191 66q-173 0 -252.5 -94t-79.5 -308v-573h-166z" />
-<glyph unicode="i" horiz-adv-x="518" d="M162 1393q0 57 28 83.5t70 26.5q40 0 69 -27t29 -83t-29 -83.5t-69 -27.5q-42 0 -70 27.5t-28 83.5zM176 0v1096h166v-1096h-166z" />
-<glyph unicode="j" horiz-adv-x="518" d="M-111 -332q69 -20 136 -20q78 0 114.5 42.5t36.5 129.5v1276h166v-1264q0 -324 -299 -324q-95 0 -154 25v135zM162 1393q0 57 28 83.5t70 26.5q40 0 69 -27t29 -83t-29 -83.5t-69 -27.5q-42 0 -70 27.5t-28 83.5z" />
-<glyph unicode="k" horiz-adv-x="1075" d="M176 0v1556h164v-825q0 -55 -8 -170h8q43 61 131 160l354 375h197l-444 -467l475 -629h-201l-387 518l-125 -108v-410h-164z" />
-<glyph unicode="l" horiz-adv-x="518" d="M176 0v1556h166v-1556h-166z" />
-<glyph unicode="m" horiz-adv-x="1905" d="M176 0v1096h135l27 -150h8q47 80 132.5 125t191.5 45q257 0 336 -186h8q49 86 142 136t212 50q186 0 278.5 -95.5t92.5 -305.5v-715h-166v713q0 131 -56 196.5t-174 65.5q-155 0 -229 -89t-74 -274v-612h-166v713q0 131 -56 196.5t-175 65.5q-156 0 -228.5 -93.5 t-72.5 -306.5v-575h-166z" />
-<glyph unicode="n" horiz-adv-x="1257" d="M176 0v1096h135l27 -150h8q51 81 143 125.5t205 44.5q198 0 298 -95.5t100 -305.5v-715h-166v709q0 134 -61 200t-191 66q-172 0 -252 -93t-80 -307v-575h-166z" />
-<glyph unicode="o" horiz-adv-x="1237" d="M115 549q0 268 134 417.5t372 149.5q230 0 365.5 -153t135.5 -414q0 -268 -135 -418.5t-373 -150.5q-147 0 -261 69t-176 198t-62 302zM287 549q0 -210 84 -320t247 -110t247.5 109.5t84.5 320.5q0 209 -84.5 317.5t-249.5 108.5q-163 0 -246 -107t-83 -319z" />
-<glyph unicode="p" horiz-adv-x="1255" d="M176 -492v1588h135l23 -150h8q64 90 149 130t195 40q218 0 336.5 -149t118.5 -418q0 -270 -120.5 -419.5t-334.5 -149.5q-107 0 -195.5 39.5t-148.5 121.5h-12q12 -96 12 -182v-451h-166zM342 549q0 -231 77 -330.5t247 -99.5q142 0 222.5 115t80.5 317 q0 205 -80.5 314.5t-226.5 109.5q-168 0 -243 -93t-77 -296v-37z" />
-<glyph unicode="q" horiz-adv-x="1255" d="M115 545q0 269 120 420t334 151q225 0 346 -170h9l24 150h131v-1588h-166v469q0 100 11 170h-13q-115 -167 -346 -167q-212 0 -331 149t-119 416zM287 543q0 -207 76.5 -315.5t226.5 -108.5q166 0 242 89t81 300v37q0 230 -78 331t-247 101q-146 0 -223.5 -113.5 t-77.5 -320.5z" />
-<glyph unicode="r" horiz-adv-x="836" d="M176 0v1096h137l19 -203h8q61 107 147 165t189 58q73 0 131 -12l-23 -154q-68 15 -120 15q-133 0 -227.5 -108t-94.5 -269v-588h-166z" />
-<glyph unicode="s" horiz-adv-x="977" d="M106 827q0 134 109 211.5t299 77.5q177 0 346 -72l-59 -135q-165 68 -299 68q-118 0 -178 -37t-60 -102q0 -44 22.5 -75t72.5 -59t192 -81q195 -71 263.5 -143t68.5 -181q0 -153 -114 -236t-320 -83q-218 0 -340 69v154q79 -40 169.5 -63t174.5 -23q130 0 200 41.5 t70 126.5q0 64 -55.5 109.5t-216.5 107.5q-153 57 -217.5 99.5t-96 96.5t-31.5 129z" />
-<glyph unicode="t" horiz-adv-x="723" d="M31 967v80l157 69l70 234h96v-254h318v-129h-318v-645q0 -99 47 -152t129 -53q44 0 85 6.5t65 13.5v-127q-27 -13 -79.5 -21.5t-94.5 -8.5q-318 0 -318 335v652h-157z" />
-<glyph unicode="u" horiz-adv-x="1257" d="M164 379v717h168v-711q0 -134 61 -200t191 -66q172 0 251.5 94t79.5 307v576h166v-1096h-137l-24 147h-9q-51 -81 -141.5 -124t-206.5 -43q-200 0 -299.5 95t-99.5 304z" />
-<glyph unicode="v" horiz-adv-x="1026" d="M0 1096h178l236 -650q80 -228 94 -296h8q11 53 69.5 219.5t262.5 726.5h178l-416 -1096h-194z" />
-<glyph unicode="w" horiz-adv-x="1593" d="M23 1096h174q106 -413 161.5 -629t63.5 -291h8q11 57 35.5 147.5t42.5 143.5l201 629h180l196 -629q56 -172 76 -289h8q4 36 21.5 111t208.5 807h172l-303 -1096h-197l-201 643q-19 59 -71 268h-8q-40 -175 -70 -270l-207 -641h-192z" />
-<glyph unicode="x" horiz-adv-x="1073" d="M39 0l401 561l-381 535h189l289 -420l288 420h187l-381 -535l401 -561h-188l-307 444l-310 -444h-188z" />
-<glyph unicode="y" horiz-adv-x="1032" d="M2 1096h178l240 -625q79 -214 98 -309h8q13 51 54.5 174.5t271.5 759.5h178l-471 -1248q-70 -185 -163.5 -262.5t-229.5 -77.5q-76 0 -150 17v133q55 -12 123 -12q171 0 244 192l61 156z" />
-<glyph unicode="z" horiz-adv-x="958" d="M82 0v113l598 854h-561v129h743v-129l-590 -838h605v-129h-795z" />
-<glyph unicode="{" horiz-adv-x="776" d="M61 498v141q130 2 188 48t58 142v306q0 155 108 241t290 86v-139q-230 -6 -230 -199v-295q0 -215 -223 -254v-12q223 -39 223 -254v-297q0 -102 58.5 -148t171.5 -48v-140q-190 2 -294 87t-104 239v303q0 104 -63 148.5t-183 44.5z" />
-<glyph unicode="|" horiz-adv-x="1128" d="M494 -496v2052h141v-2052h-141z" />
-<glyph unicode="}" horiz-adv-x="776" d="M72 -184q111 2 169 48t58 148v297q0 114 55 174t168 80v12q-223 39 -223 254v295q0 193 -227 199v139q184 0 289.5 -87t105.5 -240v-306q0 -97 59 -142.5t189 -47.5v-141q-122 0 -185 -44.5t-63 -148.5v-303q0 -153 -102.5 -238.5t-292.5 -87.5v140z" />
-<glyph unicode="~" d="M104 592v151q100 109 244 109q68 0 124.5 -14t145.5 -52q66 -28 115 -41.5t96 -13.5q54 0 118 32t118 89v-150q-102 -110 -244 -110q-72 0 -135 16.5t-135 48.5q-75 32 -120 44t-93 12q-53 0 -116.5 -33.5t-117.5 -87.5z" />
-<glyph unicode="&#xa1;" horiz-adv-x="547" d="M152 983q0 63 31.5 99t88.5 36q51 0 86 -32t35 -103q0 -135 -121 -135q-60 0 -90 35.5t-30 99.5zM168 -373l51 1057h105l51 -1057h-207z" />
-<glyph unicode="&#xa2;" d="M190 741q0 508 396 570v172h135v-164q75 -3 146 -19.5t120 -39.5l-49 -140q-133 51 -242 51q-172 0 -253 -105.5t-81 -322.5q0 -212 79.5 -313.5t246.5 -101.5q141 0 283 59v-147q-105 -54 -252 -60v-200h-133v206q-203 32 -299.5 168.5t-96.5 386.5z" />
-<glyph unicode="&#xa3;" d="M63 0v141q205 47 205 291v223h-198v127h198v316q0 178 112 280.5t302 102.5t360 -84l-61 -133q-154 77 -297 77q-123 0 -185.5 -62t-62.5 -202v-295h422v-127h-422v-221q0 -100 -32.5 -168t-106.5 -112h795v-154h-1029z" />
-<glyph unicode="&#xa4;" d="M123 1092l94 92l135 -133q104 73 234 73q127 0 229 -73l137 133l95 -92l-134 -138q74 -113 74 -231q0 -131 -74 -234l131 -135l-92 -92l-137 133q-102 -71 -229 -71q-134 0 -234 73l-135 -133l-92 92l133 136q-74 107 -74 231q0 122 74 229zM313 723q0 -112 78.5 -192 t194.5 -80t195 79.5t79 192.5q0 114 -80 195t-194 81q-116 0 -194.5 -82t-78.5 -194z" />
-<glyph unicode="&#xa5;" d="M31 1462h178l375 -727l379 727h174l-416 -770h262v-127h-317v-170h317v-127h-317v-268h-164v268h-316v127h316v170h-316v127h256z" />
-<glyph unicode="&#xa6;" horiz-adv-x="1128" d="M494 281h141v-777h-141v777zM494 780v776h141v-776h-141z" />
-<glyph unicode="&#xa7;" horiz-adv-x="1057" d="M123 57v148q78 -37 175 -59.5t179 -22.5q134 0 204.5 38t70.5 109q0 46 -24 75t-78 58t-169 72q-142 52 -209 97t-100 102t-33 135q0 86 43 154.5t121 105.5q-74 40 -116 95.5t-42 140.5q0 121 103.5 190.5t300.5 69.5q94 0 173.5 -14.5t176.5 -53.5l-53 -131 q-98 39 -165.5 52.5t-143.5 13.5q-116 0 -174 -29.5t-58 -93.5q0 -60 61.5 -102t215.5 -97q186 -68 261 -143.5t75 -182.5q0 -90 -41 -160.5t-115 -111.5q153 -81 153 -227q0 -140 -117 -216.5t-329 -76.5q-218 0 -346 65zM285 829q0 -77 66 -129.5t233 -113.5l49 -19 q137 80 137 191q0 83 -73.5 139t-258.5 113q-68 -19 -110.5 -69t-42.5 -112z" />
-<glyph unicode="&#xa8;" horiz-adv-x="1182" d="M309 1393q0 52 26.5 75t63.5 23q38 0 65.5 -23t27.5 -75q0 -50 -27.5 -74.5t-65.5 -24.5q-37 0 -63.5 24.5t-26.5 74.5zM690 1393q0 52 26.5 75t63.5 23t64.5 -23t27.5 -75q0 -50 -27.5 -74.5t-64.5 -24.5t-63.5 24.5t-26.5 74.5z" />
-<glyph unicode="&#xa9;" horiz-adv-x="1704" d="M100 731q0 200 100 375t275 276t377 101q200 0 375 -100t276 -275t101 -377q0 -197 -97 -370t-272 -277t-383 -104q-207 0 -382 103.5t-272.5 276.5t-97.5 371zM205 731q0 -173 87 -323.5t237.5 -237t322.5 -86.5q174 0 323 87t236.5 235.5t87.5 324.5q0 174 -87 323 t-235.5 236.5t-324.5 87.5q-174 0 -323 -87t-236.5 -235.5t-87.5 -324.5zM481 731q0 209 110.5 332t301.5 123q128 0 246 -60l-58 -118q-108 51 -188 51q-125 0 -192.5 -87t-67.5 -241q0 -168 63.5 -249t194.5 -81q86 0 211 45v-124q-48 -20 -98.5 -34t-120.5 -14 q-194 0 -298 120.5t-104 336.5z" />
-<glyph unicode="&#xaa;" horiz-adv-x="725" d="M70 989q0 102 77 154.5t242 58.5l117 4v39q0 133 -148 133q-100 0 -204 -51l-43 96q114 56 247 56q130 0 198.5 -52.5t68.5 -173.5v-452h-93l-24 84q-92 -97 -232 -97q-95 0 -150.5 49.5t-55.5 151.5zM193 989q0 -100 112 -100q201 0 201 180v49l-98 -4 q-112 -4 -163.5 -32.5t-51.5 -92.5z" />
-<glyph unicode="&#xab;" horiz-adv-x="1018" d="M82 524v27l342 407l119 -69l-289 -350l289 -351l-119 -71zM477 524v27l344 407l117 -69l-287 -350l287 -351l-117 -71z" />
-<glyph unicode="&#xac;" d="M104 653v138h961v-527h-137v389h-824z" />
-<glyph unicode="&#xad;" horiz-adv-x="659" d="M84 473v152h491v-152h-491z" />
-<glyph unicode="&#xae;" horiz-adv-x="1704" d="M100 731q0 200 100 375t275 276t377 101q200 0 375 -100t276 -275t101 -377q0 -197 -97 -370t-272 -277t-383 -104q-207 0 -382 103.5t-272.5 276.5t-97.5 371zM205 731q0 -173 87 -323.5t237.5 -237t322.5 -86.5q174 0 323 87t236.5 235.5t87.5 324.5q0 174 -87 323 t-235.5 236.5t-324.5 87.5q-174 0 -323 -87t-236.5 -235.5t-87.5 -324.5zM575 285v891h261q166 0 243.5 -65t77.5 -198q0 -80 -42.5 -141.5t-119.5 -91.5l238 -395h-168l-207 354h-135v-354h-148zM723 762h108q80 0 128.5 41.5t48.5 105.5q0 75 -43 107.5t-136 32.5h-106 v-287z" />
-<glyph unicode="&#xaf;" horiz-adv-x="1024" d="M-6 1556v127h1036v-127h-1036z" />
-<glyph unicode="&#xb0;" horiz-adv-x="877" d="M127 1171q0 130 90.5 221t220.5 91t221 -90.5t91 -221.5q0 -84 -41 -155.5t-114 -113.5t-157 -42q-130 0 -220.5 90t-90.5 221zM242 1171q0 -82 58.5 -139t139.5 -57q80 0 137.5 56.5t57.5 139.5q0 84 -56.5 140.5t-138.5 56.5q-83 0 -140.5 -57t-57.5 -140z" />
-<glyph unicode="&#xb1;" d="M104 1v138h961v-138h-961zM104 653v138h410v428h139v-428h412v-138h-412v-426h-139v426h-410z" />
-<glyph unicode="&#xb2;" horiz-adv-x="711" d="M49 586v104l236 230q89 86 130 134.5t57.5 86.5t16.5 92q0 68 -40 102.5t-103 34.5q-52 0 -101 -19t-118 -69l-66 88q131 111 283 111q132 0 205.5 -65t73.5 -177q0 -80 -44.5 -155.5t-191.5 -213.5l-174 -165h440v-119h-604z" />
-<glyph unicode="&#xb3;" horiz-adv-x="711" d="M33 625v123q147 -68 270 -68q211 0 211 162q0 145 -231 145h-117v107h119q103 0 152.5 39.5t49.5 107.5q0 61 -40 95t-107 34q-66 0 -122 -21.5t-112 -56.5l-69 90q63 45 133 72t164 27q136 0 214.5 -59.5t78.5 -166.5q0 -80 -41 -131.5t-109 -74.5q176 -47 176 -209 q0 -128 -92 -199.5t-260 -71.5q-152 0 -268 56z" />
-<glyph unicode="&#xb4;" horiz-adv-x="1182" d="M393 1241v25q48 62 103.5 150t87.5 153h202v-21q-44 -65 -131 -160t-151 -147h-111z" />
-<glyph unicode="&#xb5;" horiz-adv-x="1268" d="M176 -492v1588h166v-715q0 -262 254 -262q171 0 250.5 94.5t79.5 306.5v576h166v-1096h-136l-26 147h-10q-111 -167 -340 -167q-150 0 -238 92h-10q10 -84 10 -244v-320h-166z" />
-<glyph unicode="&#xb6;" horiz-adv-x="1341" d="M113 1042q0 260 109 387t341 127h557v-1816h-114v1712h-213v-1712h-115v819q-62 -18 -146 -18q-216 0 -317.5 125t-101.5 376z" />
-<glyph unicode="&#xb7;" horiz-adv-x="545" d="M152 723q0 66 31 100.5t87 34.5q58 0 90.5 -34.5t32.5 -100.5q0 -65 -33 -100t-90 -35q-51 0 -84.5 31.5t-33.5 103.5z" />
-<glyph unicode="&#xb8;" horiz-adv-x="465" d="M37 -377q45 -8 104 -8q79 0 119.5 20t40.5 74q0 43 -39.5 69.5t-148.5 43.5l88 178h110l-55 -115q180 -39 180 -174q0 -97 -76.5 -150t-226.5 -53q-51 0 -96 9v106z" />
-<glyph unicode="&#xb9;" horiz-adv-x="711" d="M76 1280l262 182h143v-876h-133v579q0 91 6 181q-22 -22 -49 -44.5t-162 -117.5z" />
-<glyph unicode="&#xba;" horiz-adv-x="768" d="M66 1135q0 163 84 253.5t235 90.5q152 0 234.5 -91t82.5 -253q0 -164 -85.5 -255.5t-235.5 -91.5q-146 0 -230.5 93t-84.5 254zM188 1135q0 -122 45.5 -183t149.5 -61q105 0 151 61t46 183q0 123 -46 182t-151 59q-103 0 -149 -59t-46 -182z" />
-<glyph unicode="&#xbb;" horiz-adv-x="1018" d="M80 188l287 351l-287 350l117 69l344 -407v-27l-344 -407zM475 188l287 351l-287 350l117 69l344 -407v-27l-344 -407z" />
-<glyph unicode="&#xbc;" horiz-adv-x="1597" d="M75 1280l262 182h143v-876h-133v579q0 91 6 181q-22 -22 -49 -44.5t-162 -117.5zM252 0l903 1462h143l-903 -1462h-143zM817 203v101l408 579h139v-563h125v-117h-125v-202h-145v202h-402zM957 320h262v195q0 134 6 209q-5 -12 -17 -31.5t-27 -42l-30 -45t-26 -39.5z" />
-<glyph unicode="&#xbd;" horiz-adv-x="1597" d="M46 1280l262 182h143v-876h-133v579q0 91 6 181q-22 -22 -49 -44.5t-162 -117.5zM184 0l903 1462h143l-903 -1462h-143zM895 1v104l236 230q89 86 130 134.5t57.5 86.5t16.5 92q0 68 -40 102.5t-103 34.5q-52 0 -101 -19t-118 -69l-66 88q131 111 283 111 q132 0 205.5 -65t73.5 -177q0 -80 -44.5 -155.5t-191.5 -213.5l-174 -165h440v-119h-604z" />
-<glyph unicode="&#xbe;" horiz-adv-x="1597" d="M26 625v123q147 -68 270 -68q211 0 211 162q0 145 -231 145h-117v107h119q103 0 152.5 39.5t49.5 107.5q0 61 -40 95t-107 34q-66 0 -122 -21.5t-112 -56.5l-69 90q63 45 133 72t164 27q136 0 214.5 -59.5t78.5 -166.5q0 -80 -41 -131.5t-109 -74.5q176 -47 176 -209 q0 -128 -92 -199.5t-260 -71.5q-152 0 -268 56zM344 0l903 1462h143l-903 -1462h-143zM897 203v101l408 579h139v-563h125v-117h-125v-202h-145v202h-402zM1037 320h262v195q0 134 6 209q-5 -12 -17 -31.5t-27 -42l-30 -45t-26 -39.5z" />
-<glyph unicode="&#xbf;" horiz-adv-x="879" d="M51 -37q0 70 17.5 122.5t49.5 97t76.5 85.5t98.5 88q101 88 133.5 146t32.5 151v31h131v-51q0 -122 -37.5 -196t-134.5 -158q-121 -106 -151.5 -143.5t-43 -76t-12.5 -94.5q0 -100 66 -156.5t188 -56.5q80 0 155 19t173 67l59 -135q-197 -96 -395 -96q-190 0 -298 93 t-108 263zM397 983q0 64 33 99.5t88 35.5q51 0 86 -32t35 -103q0 -135 -121 -135q-59 0 -90 34.5t-31 100.5z" />
-<glyph unicode="&#xc0;" horiz-adv-x="1296" d="M0 0l578 1468h143l575 -1468h-176l-182 465h-586l-180 -465h-172zM331 1886v21h203q32 -69 89 -159.5t101 -143.5v-25h-110q-65 52 -154 148t-129 159zM412 618h473l-170 453q-33 86 -68 211q-22 -96 -63 -211z" />
-<glyph unicode="&#xc1;" horiz-adv-x="1296" d="M0 0l578 1468h143l575 -1468h-176l-182 465h-586l-180 -465h-172zM412 618h473l-170 453q-33 86 -68 211q-22 -96 -63 -211zM526 1579v25q48 62 103.5 150t87.5 153h202v-21q-44 -65 -131 -160t-151 -147h-111z" />
-<glyph unicode="&#xc2;" horiz-adv-x="1296" d="M0 0l578 1468h143l575 -1468h-176l-182 465h-586l-180 -465h-172zM303 1579v23q127 136 178 200t74 105h166q22 -42 76.5 -108.5t179.5 -196.5v-23h-119q-88 55 -221 186q-136 -134 -219 -186h-115zM412 618h473l-170 453q-33 86 -68 211q-22 -96 -63 -211z" />
-<glyph unicode="&#xc3;" horiz-adv-x="1296" d="M0 0l578 1468h143l575 -1468h-176l-182 465h-586l-180 -465h-172zM268 1579q13 121 70.5 189.5t148.5 68.5q46 0 89 -18.5t82 -41t75 -41t68 -18.5q49 0 73 29.5t39 91.5h99q-13 -121 -69.5 -189.5t-150.5 -68.5q-43 0 -84 18.5t-80.5 41t-76 41t-70.5 18.5 q-50 0 -75.5 -30t-39.5 -91h-98zM412 618h473l-170 453q-33 86 -68 211q-22 -96 -63 -211z" />
-<glyph unicode="&#xc4;" horiz-adv-x="1296" d="M0 0l578 1468h143l575 -1468h-176l-182 465h-586l-180 -465h-172zM364 1731q0 52 26.5 75t63.5 23q38 0 65.5 -23t27.5 -75q0 -50 -27.5 -74.5t-65.5 -24.5q-37 0 -63.5 24.5t-26.5 74.5zM412 618h473l-170 453q-33 86 -68 211q-22 -96 -63 -211zM745 1731q0 52 26.5 75 t63.5 23t64.5 -23t27.5 -75q0 -50 -27.5 -74.5t-64.5 -24.5t-63.5 24.5t-26.5 74.5z" />
-<glyph unicode="&#xc5;" horiz-adv-x="1296" d="M0 0l578 1468h143l575 -1468h-176l-182 465h-586l-180 -465h-172zM412 618h473l-170 453q-33 86 -68 211q-22 -96 -63 -211zM424 1585q0 98 60.5 155.5t160.5 57.5q101 0 163 -59.5t62 -151.5q0 -98 -61.5 -157.5t-163.5 -59.5q-101 0 -161 58.5t-60 156.5zM528 1585 q0 -56 30 -86.5t87 -30.5q52 0 84.5 30.5t32.5 86.5t-33 86.5t-84 30.5t-84 -30.5t-33 -86.5z" />
-<glyph unicode="&#xc6;" horiz-adv-x="1788" d="M-2 0l698 1462h969v-151h-580v-471h541v-150h-541v-538h580v-152h-750v465h-514l-227 -465h-176zM469 618h446v693h-118z" />
-<glyph unicode="&#xc7;" horiz-adv-x="1292" d="M125 733q0 226 84.5 396t244 262t375.5 92q230 0 402 -84l-72 -146q-166 78 -332 78q-241 0 -380.5 -160.5t-139.5 -439.5q0 -287 134.5 -443.5t383.5 -156.5q153 0 349 55v-149q-152 -57 -375 -57q-323 0 -498.5 196t-175.5 557zM551 -377q45 -8 104 -8q79 0 119.5 20 t40.5 74q0 43 -39.5 69.5t-148.5 43.5l88 178h110l-55 -115q180 -39 180 -174q0 -97 -76.5 -150t-226.5 -53q-51 0 -96 9v106z" />
-<glyph unicode="&#xc8;" horiz-adv-x="1139" d="M201 0v1462h815v-151h-645v-471h606v-150h-606v-538h645v-152h-815zM320 1886v21h203q32 -69 89 -159.5t101 -143.5v-25h-110q-65 52 -154 148t-129 159z" />
-<glyph unicode="&#xc9;" horiz-adv-x="1139" d="M201 0v1462h815v-151h-645v-471h606v-150h-606v-538h645v-152h-815zM456 1579v25q48 62 103.5 150t87.5 153h202v-21q-44 -65 -131 -160t-151 -147h-111z" />
-<glyph unicode="&#xca;" horiz-adv-x="1139" d="M201 0v1462h815v-151h-645v-471h606v-150h-606v-538h645v-152h-815zM263 1579v23q127 136 178 200t74 105h166q22 -42 76.5 -108.5t179.5 -196.5v-23h-119q-88 55 -221 186q-136 -134 -219 -186h-115z" />
-<glyph unicode="&#xcb;" horiz-adv-x="1139" d="M201 0v1462h815v-151h-645v-471h606v-150h-606v-538h645v-152h-815zM327 1731q0 52 26.5 75t63.5 23q38 0 65.5 -23t27.5 -75q0 -50 -27.5 -74.5t-65.5 -24.5q-37 0 -63.5 24.5t-26.5 74.5zM708 1731q0 52 26.5 75t63.5 23t64.5 -23t27.5 -75q0 -50 -27.5 -74.5 t-64.5 -24.5t-63.5 24.5t-26.5 74.5z" />
-<glyph unicode="&#xcc;" horiz-adv-x="571" d="M5 1886v21h203q32 -69 89 -159.5t101 -143.5v-25h-110q-65 52 -154 148t-129 159zM201 0v1462h170v-1462h-170z" />
-<glyph unicode="&#xcd;" horiz-adv-x="571" d="M179 1579v25q48 62 103.5 150t87.5 153h202v-21q-44 -65 -131 -160t-151 -147h-111zM201 0v1462h170v-1462h-170z" />
-<glyph unicode="&#xce;" horiz-adv-x="571" d="M-57 1579v23q127 136 178 200t74 105h166q22 -42 76.5 -108.5t179.5 -196.5v-23h-119q-88 55 -221 186q-136 -134 -219 -186h-115zM201 0v1462h170v-1462h-170z" />
-<glyph unicode="&#xcf;" horiz-adv-x="571" d="M5 1731q0 52 26.5 75t63.5 23q38 0 65.5 -23t27.5 -75q0 -50 -27.5 -74.5t-65.5 -24.5q-37 0 -63.5 24.5t-26.5 74.5zM201 0v1462h170v-1462h-170zM386 1731q0 52 26.5 75t63.5 23t64.5 -23t27.5 -75q0 -50 -27.5 -74.5t-64.5 -24.5t-63.5 24.5t-26.5 74.5z" />
-<glyph unicode="&#xd0;" horiz-adv-x="1479" d="M47 649v150h154v663h434q337 0 527 -187.5t190 -529.5q0 -362 -196.5 -553.5t-565.5 -191.5h-389v649h-154zM371 147h190q610 0 610 592q0 576 -569 576h-231v-516h379v-150h-379v-502z" />
-<glyph unicode="&#xd1;" horiz-adv-x="1544" d="M201 0v1462h192l797 -1222h8q-2 27 -9 173.5t-5 209.5v839h159v-1462h-194l-799 1227h-8q16 -216 16 -396v-831h-157zM411 1579q13 121 70.5 189.5t148.5 68.5q46 0 89 -18.5t82 -41t75 -41t68 -18.5q49 0 73 29.5t39 91.5h99q-13 -121 -69.5 -189.5t-150.5 -68.5 q-43 0 -84 18.5t-80.5 41t-76 41t-70.5 18.5q-50 0 -75.5 -30t-39.5 -91h-98z" />
-<glyph unicode="&#xd2;" horiz-adv-x="1595" d="M125 735q0 357 176 553.5t500 196.5q315 0 492 -200t177 -552q0 -351 -177.5 -552t-493.5 -201q-323 0 -498.5 197.5t-175.5 557.5zM305 733q0 -297 126.5 -450.5t367.5 -153.5q243 0 367 153t124 451q0 295 -123.5 447.5t-365.5 152.5q-243 0 -369.5 -153.5 t-126.5 -446.5zM514 1886v21h203q32 -69 89 -159.5t101 -143.5v-25h-110q-65 52 -154 148t-129 159z" />
-<glyph unicode="&#xd3;" horiz-adv-x="1595" d="M125 735q0 357 176 553.5t500 196.5q315 0 492 -200t177 -552q0 -351 -177.5 -552t-493.5 -201q-323 0 -498.5 197.5t-175.5 557.5zM305 733q0 -297 126.5 -450.5t367.5 -153.5q243 0 367 153t124 451q0 295 -123.5 447.5t-365.5 152.5q-243 0 -369.5 -153.5 t-126.5 -446.5zM659 1579v25q48 62 103.5 150t87.5 153h202v-21q-44 -65 -131 -160t-151 -147h-111z" />
-<glyph unicode="&#xd4;" horiz-adv-x="1595" d="M125 735q0 357 176 553.5t500 196.5q315 0 492 -200t177 -552q0 -351 -177.5 -552t-493.5 -201q-323 0 -498.5 197.5t-175.5 557.5zM305 733q0 -297 126.5 -450.5t367.5 -153.5q243 0 367 153t124 451q0 295 -123.5 447.5t-365.5 152.5q-243 0 -369.5 -153.5 t-126.5 -446.5zM448 1579v23q127 136 178 200t74 105h166q22 -42 76.5 -108.5t179.5 -196.5v-23h-119q-88 55 -221 186q-136 -134 -219 -186h-115z" />
-<glyph unicode="&#xd5;" horiz-adv-x="1595" d="M125 735q0 357 176 553.5t500 196.5q315 0 492 -200t177 -552q0 -351 -177.5 -552t-493.5 -201q-323 0 -498.5 197.5t-175.5 557.5zM305 733q0 -297 126.5 -450.5t367.5 -153.5q243 0 367 153t124 451q0 295 -123.5 447.5t-365.5 152.5q-243 0 -369.5 -153.5 t-126.5 -446.5zM418 1579q13 121 70.5 189.5t148.5 68.5q46 0 89 -18.5t82 -41t75 -41t68 -18.5q49 0 73 29.5t39 91.5h99q-13 -121 -69.5 -189.5t-150.5 -68.5q-43 0 -84 18.5t-80.5 41t-76 41t-70.5 18.5q-50 0 -75.5 -30t-39.5 -91h-98z" />
-<glyph unicode="&#xd6;" horiz-adv-x="1595" d="M125 735q0 357 176 553.5t500 196.5q315 0 492 -200t177 -552q0 -351 -177.5 -552t-493.5 -201q-323 0 -498.5 197.5t-175.5 557.5zM305 733q0 -297 126.5 -450.5t367.5 -153.5q243 0 367 153t124 451q0 295 -123.5 447.5t-365.5 152.5q-243 0 -369.5 -153.5 t-126.5 -446.5zM522 1731q0 52 26.5 75t63.5 23q38 0 65.5 -23t27.5 -75q0 -50 -27.5 -74.5t-65.5 -24.5q-37 0 -63.5 24.5t-26.5 74.5zM903 1731q0 52 26.5 75t63.5 23t64.5 -23t27.5 -75q0 -50 -27.5 -74.5t-64.5 -24.5t-63.5 24.5t-26.5 74.5z" />
-<glyph unicode="&#xd7;" d="M133 1075l100 101l353 -355l354 355l96 -99l-352 -354l350 -352l-96 -99l-354 351l-348 -351l-101 99l350 352z" />
-<glyph unicode="&#xd8;" horiz-adv-x="1595" d="M125 735q0 357 176 553.5t500 196.5q209 0 366 -94l97 135l120 -80l-106 -148q192 -202 192 -565q0 -351 -177.5 -552t-493.5 -201q-235 0 -383 100l-101 -141l-120 79l108 154q-178 198 -178 563zM305 733q0 -262 101 -416l669 943q-106 73 -274 73 q-243 0 -369.5 -153.5t-126.5 -446.5zM508 211q115 -82 291 -82q243 0 367 153t124 451q0 272 -110 426z" />
-<glyph unicode="&#xd9;" horiz-adv-x="1491" d="M186 520v942h170v-954q0 -183 100 -281t294 -98q185 0 285 98.5t100 282.5v952h170v-946q0 -250 -151 -393t-415 -143t-408.5 144t-144.5 396zM463 1886v21h203q32 -69 89 -159.5t101 -143.5v-25h-110q-65 52 -154 148t-129 159z" />
-<glyph unicode="&#xda;" horiz-adv-x="1491" d="M186 520v942h170v-954q0 -183 100 -281t294 -98q185 0 285 98.5t100 282.5v952h170v-946q0 -250 -151 -393t-415 -143t-408.5 144t-144.5 396zM600 1579v25q48 62 103.5 150t87.5 153h202v-21q-44 -65 -131 -160t-151 -147h-111z" />
-<glyph unicode="&#xdb;" horiz-adv-x="1491" d="M186 520v942h170v-954q0 -183 100 -281t294 -98q185 0 285 98.5t100 282.5v952h170v-946q0 -250 -151 -393t-415 -143t-408.5 144t-144.5 396zM393 1579v23q127 136 178 200t74 105h166q22 -42 76.5 -108.5t179.5 -196.5v-23h-119q-88 55 -221 186q-136 -134 -219 -186 h-115z" />
-<glyph unicode="&#xdc;" horiz-adv-x="1491" d="M186 520v942h170v-954q0 -183 100 -281t294 -98q185 0 285 98.5t100 282.5v952h170v-946q0 -250 -151 -393t-415 -143t-408.5 144t-144.5 396zM461 1731q0 52 26.5 75t63.5 23q38 0 65.5 -23t27.5 -75q0 -50 -27.5 -74.5t-65.5 -24.5q-37 0 -63.5 24.5t-26.5 74.5z M842 1731q0 52 26.5 75t63.5 23t64.5 -23t27.5 -75q0 -50 -27.5 -74.5t-64.5 -24.5t-63.5 24.5t-26.5 74.5z" />
-<glyph unicode="&#xdd;" horiz-adv-x="1147" d="M0 1462h186l387 -731l390 731h184l-488 -895v-567h-172v559zM442 1579v25q48 62 103.5 150t87.5 153h202v-21q-44 -65 -131 -160t-151 -147h-111z" />
-<glyph unicode="&#xde;" horiz-adv-x="1251" d="M201 0v1462h170v-256h215q281 0 420 -103.5t139 -318.5q0 -227 -151.5 -346t-438.5 -119h-184v-319h-170zM371 465h168q226 0 327 71.5t101 235.5q0 149 -95 218t-297 69h-204v-594z" />
-<glyph unicode="&#xdf;" horiz-adv-x="1274" d="M176 0v1202q0 178 110 271.5t332 93.5q206 0 318.5 -78.5t112.5 -222.5q0 -135 -143 -250q-88 -70 -116 -103.5t-28 -66.5q0 -32 13.5 -53t49 -49.5t113.5 -79.5q140 -95 191 -173.5t51 -179.5q0 -160 -97 -245.5t-276 -85.5q-188 0 -295 69v154q63 -39 141 -62.5 t150 -23.5q215 0 215 182q0 75 -41.5 128.5t-151.5 123.5q-127 82 -175 143.5t-48 145.5q0 63 34.5 116t105.5 106q75 57 107 102t32 98q0 80 -68 122.5t-195 42.5q-276 0 -276 -223v-1204h-166z" />
-<glyph unicode="&#xe0;" horiz-adv-x="1139" d="M94 303q0 332 531 348l186 6v68q0 129 -55.5 190.5t-177.5 61.5q-137 0 -310 -84l-51 127q81 44 177.5 69t193.5 25q196 0 290.5 -87t94.5 -279v-748h-123l-33 156h-8q-82 -103 -163.5 -139.5t-203.5 -36.5q-163 0 -255.5 84t-92.5 239zM268 301q0 -90 54.5 -137 t152.5 -47q155 0 243.5 85t88.5 238v99l-166 -7q-198 -7 -285.5 -61.5t-87.5 -169.5zM279 1548v21h203q32 -69 89 -159.5t101 -143.5v-25h-110q-65 52 -154 148t-129 159z" />
-<glyph unicode="&#xe1;" horiz-adv-x="1139" d="M94 303q0 332 531 348l186 6v68q0 129 -55.5 190.5t-177.5 61.5q-137 0 -310 -84l-51 127q81 44 177.5 69t193.5 25q196 0 290.5 -87t94.5 -279v-748h-123l-33 156h-8q-82 -103 -163.5 -139.5t-203.5 -36.5q-163 0 -255.5 84t-92.5 239zM268 301q0 -90 54.5 -137 t152.5 -47q155 0 243.5 85t88.5 238v99l-166 -7q-198 -7 -285.5 -61.5t-87.5 -169.5zM436 1241v25q48 62 103.5 150t87.5 153h202v-21q-44 -65 -131 -160t-151 -147h-111z" />
-<glyph unicode="&#xe2;" horiz-adv-x="1139" d="M94 303q0 332 531 348l186 6v68q0 129 -55.5 190.5t-177.5 61.5q-137 0 -310 -84l-51 127q81 44 177.5 69t193.5 25q196 0 290.5 -87t94.5 -279v-748h-123l-33 156h-8q-82 -103 -163.5 -139.5t-203.5 -36.5q-163 0 -255.5 84t-92.5 239zM228 1241v23q127 136 178 200 t74 105h166q22 -42 76.5 -108.5t179.5 -196.5v-23h-119q-88 55 -221 186q-136 -134 -219 -186h-115zM268 301q0 -90 54.5 -137t152.5 -47q155 0 243.5 85t88.5 238v99l-166 -7q-198 -7 -285.5 -61.5t-87.5 -169.5z" />
-<glyph unicode="&#xe3;" horiz-adv-x="1139" d="M94 303q0 332 531 348l186 6v68q0 129 -55.5 190.5t-177.5 61.5q-137 0 -310 -84l-51 127q81 44 177.5 69t193.5 25q196 0 290.5 -87t94.5 -279v-748h-123l-33 156h-8q-82 -103 -163.5 -139.5t-203.5 -36.5q-163 0 -255.5 84t-92.5 239zM197 1241q13 121 70.5 189.5 t148.5 68.5q46 0 89 -18.5t82 -41t75 -41t68 -18.5q49 0 73 29.5t39 91.5h99q-13 -121 -69.5 -189.5t-150.5 -68.5q-43 0 -84 18.5t-80.5 41t-76 41t-70.5 18.5q-50 0 -75.5 -30t-39.5 -91h-98zM268 301q0 -90 54.5 -137t152.5 -47q155 0 243.5 85t88.5 238v99l-166 -7 q-198 -7 -285.5 -61.5t-87.5 -169.5z" />
-<glyph unicode="&#xe4;" horiz-adv-x="1139" d="M94 303q0 332 531 348l186 6v68q0 129 -55.5 190.5t-177.5 61.5q-137 0 -310 -84l-51 127q81 44 177.5 69t193.5 25q196 0 290.5 -87t94.5 -279v-748h-123l-33 156h-8q-82 -103 -163.5 -139.5t-203.5 -36.5q-163 0 -255.5 84t-92.5 239zM268 301q0 -90 54.5 -137 t152.5 -47q155 0 243.5 85t88.5 238v99l-166 -7q-198 -7 -285.5 -61.5t-87.5 -169.5zM279 1393q0 52 26.5 75t63.5 23q38 0 65.5 -23t27.5 -75q0 -50 -27.5 -74.5t-65.5 -24.5q-37 0 -63.5 24.5t-26.5 74.5zM660 1393q0 52 26.5 75t63.5 23t64.5 -23t27.5 -75 q0 -50 -27.5 -74.5t-64.5 -24.5t-63.5 24.5t-26.5 74.5z" />
-<glyph unicode="&#xe5;" horiz-adv-x="1139" d="M94 303q0 332 531 348l186 6v68q0 129 -55.5 190.5t-177.5 61.5q-137 0 -310 -84l-51 127q81 44 177.5 69t193.5 25q196 0 290.5 -87t94.5 -279v-748h-123l-33 156h-8q-82 -103 -163.5 -139.5t-203.5 -36.5q-163 0 -255.5 84t-92.5 239zM268 301q0 -90 54.5 -137 t152.5 -47q155 0 243.5 85t88.5 238v99l-166 -7q-198 -7 -285.5 -61.5t-87.5 -169.5zM358 1456q0 98 60.5 155.5t160.5 57.5q101 0 163 -59.5t62 -151.5q0 -98 -61.5 -157.5t-163.5 -59.5q-101 0 -161 58.5t-60 156.5zM462 1456q0 -56 30 -86.5t87 -30.5q52 0 84.5 30.5 t32.5 86.5t-33 86.5t-84 30.5t-84 -30.5t-33 -86.5z" />
-<glyph unicode="&#xe6;" horiz-adv-x="1757" d="M94 303q0 161 124 250.5t378 97.5l184 6v68q0 129 -58 190.5t-177 61.5q-144 0 -307 -84l-52 127q74 41 173.5 67.5t197.5 26.5q130 0 212.5 -43.5t123.5 -138.5q53 88 138.5 136t195.5 48q192 0 308 -133.5t116 -355.5v-107h-701q8 -395 322 -395q91 0 169.5 17.5 t162.5 56.5v-148q-86 -38 -160.5 -54.5t-175.5 -16.5q-289 0 -414 233q-81 -127 -179.5 -180t-232.5 -53q-163 0 -255.5 85t-92.5 238zM268 301q0 -95 53.5 -139.5t141.5 -44.5q145 0 229 84.5t84 238.5v99l-158 -7q-186 -8 -268 -62.5t-82 -168.5zM954 653h519 q0 156 -64 240t-184 84q-121 0 -190.5 -83t-80.5 -241z" />
-<glyph unicode="&#xe7;" horiz-adv-x="975" d="M115 541q0 275 132.5 425t377.5 150q79 0 158 -17t124 -40l-51 -141q-55 22 -120 36.5t-115 14.5q-334 0 -334 -426q0 -202 81.5 -310t241.5 -108q137 0 281 59v-147q-110 -57 -277 -57q-238 0 -368.5 146.5t-130.5 414.5zM363 -377q45 -8 104 -8q79 0 119.5 20t40.5 74 q0 43 -39.5 69.5t-148.5 43.5l88 178h110l-55 -115q180 -39 180 -174q0 -97 -76.5 -150t-226.5 -53q-51 0 -96 9v106z" />
-<glyph unicode="&#xe8;" horiz-adv-x="1149" d="M115 539q0 265 130.5 421t350.5 156q206 0 326 -135.5t120 -357.5v-105h-755q5 -193 97.5 -293t260.5 -100q177 0 350 74v-148q-88 -38 -166.5 -54.5t-189.5 -16.5q-243 0 -383.5 148t-140.5 411zM291 653h573q0 157 -70 240.5t-200 83.5q-132 0 -210.5 -86t-92.5 -238z M318 1548v21h203q32 -69 89 -159.5t101 -143.5v-25h-110q-65 52 -154 148t-129 159z" />
-<glyph unicode="&#xe9;" horiz-adv-x="1149" d="M115 539q0 265 130.5 421t350.5 156q206 0 326 -135.5t120 -357.5v-105h-755q5 -193 97.5 -293t260.5 -100q177 0 350 74v-148q-88 -38 -166.5 -54.5t-189.5 -16.5q-243 0 -383.5 148t-140.5 411zM291 653h573q0 157 -70 240.5t-200 83.5q-132 0 -210.5 -86t-92.5 -238z M471 1241v25q48 62 103.5 150t87.5 153h202v-21q-44 -65 -131 -160t-151 -147h-111z" />
-<glyph unicode="&#xea;" horiz-adv-x="1149" d="M115 539q0 265 130.5 421t350.5 156q206 0 326 -135.5t120 -357.5v-105h-755q5 -193 97.5 -293t260.5 -100q177 0 350 74v-148q-88 -38 -166.5 -54.5t-189.5 -16.5q-243 0 -383.5 148t-140.5 411zM259 1241v23q127 136 178 200t74 105h166q22 -42 76.5 -108.5 t179.5 -196.5v-23h-119q-88 55 -221 186q-136 -134 -219 -186h-115zM291 653h573q0 157 -70 240.5t-200 83.5q-132 0 -210.5 -86t-92.5 -238z" />
-<glyph unicode="&#xeb;" horiz-adv-x="1149" d="M115 539q0 265 130.5 421t350.5 156q206 0 326 -135.5t120 -357.5v-105h-755q5 -193 97.5 -293t260.5 -100q177 0 350 74v-148q-88 -38 -166.5 -54.5t-189.5 -16.5q-243 0 -383.5 148t-140.5 411zM291 653h573q0 157 -70 240.5t-200 83.5q-132 0 -210.5 -86t-92.5 -238z M319 1393q0 52 26.5 75t63.5 23q38 0 65.5 -23t27.5 -75q0 -50 -27.5 -74.5t-65.5 -24.5q-37 0 -63.5 24.5t-26.5 74.5zM700 1393q0 52 26.5 75t63.5 23t64.5 -23t27.5 -75q0 -50 -27.5 -74.5t-64.5 -24.5t-63.5 24.5t-26.5 74.5z" />
-<glyph unicode="&#xec;" horiz-adv-x="518" d="M-38 1548v21h203q32 -69 89 -159.5t101 -143.5v-25h-110q-65 52 -154 148t-129 159zM176 0v1096h166v-1096h-166z" />
-<glyph unicode="&#xed;" horiz-adv-x="518" d="M169 1241v25q48 62 103.5 150t87.5 153h202v-21q-44 -65 -131 -160t-151 -147h-111zM176 0v1096h166v-1096h-166z" />
-<glyph unicode="&#xee;" horiz-adv-x="518" d="M-77 1241v23q127 136 178 200t74 105h166q22 -42 76.5 -108.5t179.5 -196.5v-23h-119q-88 55 -221 186q-136 -134 -219 -186h-115zM176 0v1096h166v-1096h-166z" />
-<glyph unicode="&#xef;" horiz-adv-x="518" d="M-20 1393q0 52 26.5 75t63.5 23q38 0 65.5 -23t27.5 -75q0 -50 -27.5 -74.5t-65.5 -24.5q-37 0 -63.5 24.5t-26.5 74.5zM176 0v1096h166v-1096h-166zM361 1393q0 52 26.5 75t63.5 23t64.5 -23t27.5 -75q0 -50 -27.5 -74.5t-64.5 -24.5t-63.5 24.5t-26.5 74.5z" />
-<glyph unicode="&#xf0;" horiz-adv-x="1221" d="M113 475q0 230 131.5 361t351.5 131q226 0 326 -121l8 4q-57 214 -262 405l-271 -155l-73 108l233 133q-92 62 -186 111l69 117q156 -73 258 -148l238 138l76 -107l-207 -119q152 -143 234.5 -342t82.5 -428q0 -281 -130.5 -432t-377.5 -151q-222 0 -361.5 134.5 t-139.5 360.5zM281 469q0 -167 87.5 -258.5t249.5 -91.5q175 0 255.5 100.5t80.5 292.5q0 147 -90 232t-246 85q-337 0 -337 -360z" />
-<glyph unicode="&#xf1;" horiz-adv-x="1257" d="M176 0v1096h135l27 -150h8q51 81 143 125.5t205 44.5q198 0 298 -95.5t100 -305.5v-715h-166v709q0 134 -61 200t-191 66q-172 0 -252 -93t-80 -307v-575h-166zM278 1241q13 121 70.5 189.5t148.5 68.5q46 0 89 -18.5t82 -41t75 -41t68 -18.5q49 0 73 29.5t39 91.5h99 q-13 -121 -69.5 -189.5t-150.5 -68.5q-43 0 -84 18.5t-80.5 41t-76 41t-70.5 18.5q-50 0 -75.5 -30t-39.5 -91h-98z" />
-<glyph unicode="&#xf2;" horiz-adv-x="1237" d="M115 549q0 268 134 417.5t372 149.5q230 0 365.5 -153t135.5 -414q0 -268 -135 -418.5t-373 -150.5q-147 0 -261 69t-176 198t-62 302zM287 549q0 -210 84 -320t247 -110t247.5 109.5t84.5 320.5q0 209 -84.5 317.5t-249.5 108.5q-163 0 -246 -107t-83 -319zM349 1548v21 h203q32 -69 89 -159.5t101 -143.5v-25h-110q-65 52 -154 148t-129 159z" />
-<glyph unicode="&#xf3;" horiz-adv-x="1237" d="M115 549q0 268 134 417.5t372 149.5q230 0 365.5 -153t135.5 -414q0 -268 -135 -418.5t-373 -150.5q-147 0 -261 69t-176 198t-62 302zM287 549q0 -210 84 -320t247 -110t247.5 109.5t84.5 320.5q0 209 -84.5 317.5t-249.5 108.5q-163 0 -246 -107t-83 -319zM479 1241v25 q48 62 103.5 150t87.5 153h202v-21q-44 -65 -131 -160t-151 -147h-111z" />
-<glyph unicode="&#xf4;" horiz-adv-x="1237" d="M115 549q0 268 134 417.5t372 149.5q230 0 365.5 -153t135.5 -414q0 -268 -135 -418.5t-373 -150.5q-147 0 -261 69t-176 198t-62 302zM282 1241v23q127 136 178 200t74 105h166q22 -42 76.5 -108.5t179.5 -196.5v-23h-119q-88 55 -221 186q-136 -134 -219 -186h-115z M287 549q0 -210 84 -320t247 -110t247.5 109.5t84.5 320.5q0 209 -84.5 317.5t-249.5 108.5q-163 0 -246 -107t-83 -319z" />
-<glyph unicode="&#xf5;" horiz-adv-x="1237" d="M115 549q0 268 134 417.5t372 149.5q230 0 365.5 -153t135.5 -414q0 -268 -135 -418.5t-373 -150.5q-147 0 -261 69t-176 198t-62 302zM249 1241q13 121 70.5 189.5t148.5 68.5q46 0 89 -18.5t82 -41t75 -41t68 -18.5q49 0 73 29.5t39 91.5h99q-13 -121 -69.5 -189.5 t-150.5 -68.5q-43 0 -84 18.5t-80.5 41t-76 41t-70.5 18.5q-50 0 -75.5 -30t-39.5 -91h-98zM287 549q0 -210 84 -320t247 -110t247.5 109.5t84.5 320.5q0 209 -84.5 317.5t-249.5 108.5q-163 0 -246 -107t-83 -319z" />
-<glyph unicode="&#xf6;" horiz-adv-x="1237" d="M115 549q0 268 134 417.5t372 149.5q230 0 365.5 -153t135.5 -414q0 -268 -135 -418.5t-373 -150.5q-147 0 -261 69t-176 198t-62 302zM287 549q0 -210 84 -320t247 -110t247.5 109.5t84.5 320.5q0 209 -84.5 317.5t-249.5 108.5q-163 0 -246 -107t-83 -319zM336 1393 q0 52 26.5 75t63.5 23q38 0 65.5 -23t27.5 -75q0 -50 -27.5 -74.5t-65.5 -24.5q-37 0 -63.5 24.5t-26.5 74.5zM717 1393q0 52 26.5 75t63.5 23t64.5 -23t27.5 -75q0 -50 -27.5 -74.5t-64.5 -24.5t-63.5 24.5t-26.5 74.5z" />
-<glyph unicode="&#xf7;" d="M104 653v138h961v-138h-961zM471 373q0 60 29.5 90.5t83.5 30.5q52 0 81 -31.5t29 -89.5q0 -57 -29.5 -89t-80.5 -32q-52 0 -82.5 31.5t-30.5 89.5zM471 1071q0 60 29.5 90.5t83.5 30.5q52 0 81 -31.5t29 -89.5q0 -57 -29.5 -89t-80.5 -32q-52 0 -82.5 31.5t-30.5 89.5z " />
-<glyph unicode="&#xf8;" horiz-adv-x="1237" d="M115 549q0 268 134 417.5t372 149.5q154 0 270 -76l84 119l117 -76l-97 -133q127 -152 127 -401q0 -268 -135 -418.5t-373 -150.5q-154 0 -266 69l-84 -117l-114 78l94 131q-129 152 -129 408zM287 549q0 -171 53 -273l465 646q-75 53 -189 53q-163 0 -246 -107t-83 -319 zM434 170q71 -51 184 -51q163 0 247.5 109.5t84.5 320.5q0 164 -51 264z" />
-<glyph unicode="&#xf9;" horiz-adv-x="1257" d="M164 379v717h168v-711q0 -134 61 -200t191 -66q172 0 251.5 94t79.5 307v576h166v-1096h-137l-24 147h-9q-51 -81 -141.5 -124t-206.5 -43q-200 0 -299.5 95t-99.5 304zM333 1548v21h203q32 -69 89 -159.5t101 -143.5v-25h-110q-65 52 -154 148t-129 159z" />
-<glyph unicode="&#xfa;" horiz-adv-x="1257" d="M164 379v717h168v-711q0 -134 61 -200t191 -66q172 0 251.5 94t79.5 307v576h166v-1096h-137l-24 147h-9q-51 -81 -141.5 -124t-206.5 -43q-200 0 -299.5 95t-99.5 304zM506 1241v25q48 62 103.5 150t87.5 153h202v-21q-44 -65 -131 -160t-151 -147h-111z" />
-<glyph unicode="&#xfb;" horiz-adv-x="1257" d="M164 379v717h168v-711q0 -134 61 -200t191 -66q172 0 251.5 94t79.5 307v576h166v-1096h-137l-24 147h-9q-51 -81 -141.5 -124t-206.5 -43q-200 0 -299.5 95t-99.5 304zM286 1241v23q127 136 178 200t74 105h166q22 -42 76.5 -108.5t179.5 -196.5v-23h-119 q-88 55 -221 186q-136 -134 -219 -186h-115z" />
-<glyph unicode="&#xfc;" horiz-adv-x="1257" d="M164 379v717h168v-711q0 -134 61 -200t191 -66q172 0 251.5 94t79.5 307v576h166v-1096h-137l-24 147h-9q-51 -81 -141.5 -124t-206.5 -43q-200 0 -299.5 95t-99.5 304zM342 1393q0 52 26.5 75t63.5 23q38 0 65.5 -23t27.5 -75q0 -50 -27.5 -74.5t-65.5 -24.5 q-37 0 -63.5 24.5t-26.5 74.5zM723 1393q0 52 26.5 75t63.5 23t64.5 -23t27.5 -75q0 -50 -27.5 -74.5t-64.5 -24.5t-63.5 24.5t-26.5 74.5z" />
-<glyph unicode="&#xfd;" horiz-adv-x="1032" d="M2 1096h178l240 -625q79 -214 98 -309h8q13 51 54.5 174.5t271.5 759.5h178l-471 -1248q-70 -185 -163.5 -262.5t-229.5 -77.5q-76 0 -150 17v133q55 -12 123 -12q171 0 244 192l61 156zM411 1241v25q48 62 103.5 150t87.5 153h202v-21q-44 -65 -131 -160t-151 -147h-111 z" />
-<glyph unicode="&#xfe;" horiz-adv-x="1255" d="M176 -492v2048h166v-466q0 -52 -6 -142h8q66 89 151 128.5t191 39.5q215 0 335 -150t120 -417q0 -268 -120.5 -418.5t-334.5 -150.5q-222 0 -344 161h-12l4 -34q8 -77 8 -140v-459h-166zM342 549q0 -231 77 -330.5t247 -99.5q303 0 303 432q0 215 -74 319.5t-231 104.5 q-168 0 -244 -92t-78 -293v-41z" />
-<glyph unicode="&#xff;" horiz-adv-x="1032" d="M2 1096h178l240 -625q79 -214 98 -309h8q13 51 54.5 174.5t271.5 759.5h178l-471 -1248q-70 -185 -163.5 -262.5t-229.5 -77.5q-76 0 -150 17v133q55 -12 123 -12q171 0 244 192l61 156zM234 1393q0 52 26.5 75t63.5 23q38 0 65.5 -23t27.5 -75q0 -50 -27.5 -74.5 t-65.5 -24.5q-37 0 -63.5 24.5t-26.5 74.5zM615 1393q0 52 26.5 75t63.5 23t64.5 -23t27.5 -75q0 -50 -27.5 -74.5t-64.5 -24.5t-63.5 24.5t-26.5 74.5z" />
-<glyph unicode="&#x152;" horiz-adv-x="1890" d="M125 735q0 360 174 555t494 195q102 0 192 -23h782v-151h-589v-471h551v-150h-551v-538h589v-152h-768q-102 -20 -194 -20q-327 0 -503.5 196.5t-176.5 558.5zM305 733q0 -297 128.5 -450.5t375.5 -153.5q112 0 199 33v1141q-87 30 -197 30q-249 0 -377.5 -152.5 t-128.5 -447.5z" />
-<glyph unicode="&#x153;" horiz-adv-x="1929" d="M113 549q0 265 131 415t366 150q131 0 233.5 -59.5t164.5 -173.5q58 112 154 172.5t222 60.5q201 0 320 -132.5t119 -358.5v-105h-729q8 -393 338 -393q94 0 174.5 17.5t167.5 56.5v-148q-88 -39 -164 -55t-180 -16q-293 0 -418 235q-62 -116 -166.5 -175.5t-241.5 -59.5 q-223 0 -357 152.5t-134 416.5zM287 549q0 -211 76 -320.5t243 -109.5q163 0 239.5 106.5t76.5 315.5q0 221 -77.5 327.5t-242.5 106.5q-166 0 -240.5 -108t-74.5 -318zM1098 653h544q0 158 -66 240t-194 82q-127 0 -199.5 -82t-84.5 -240z" />
-<glyph unicode="&#x178;" horiz-adv-x="1147" d="M0 1462h186l387 -731l390 731h184l-488 -895v-567h-172v559zM294 1731q0 52 26.5 75t63.5 23q38 0 65.5 -23t27.5 -75q0 -50 -27.5 -74.5t-65.5 -24.5q-37 0 -63.5 24.5t-26.5 74.5zM675 1731q0 52 26.5 75t63.5 23t64.5 -23t27.5 -75q0 -50 -27.5 -74.5t-64.5 -24.5 t-63.5 24.5t-26.5 74.5z" />
-<glyph unicode="&#x2c6;" horiz-adv-x="1212" d="M268 1241v23q127 136 178 200t74 105h166q22 -42 76.5 -108.5t179.5 -196.5v-23h-119q-88 55 -221 186q-136 -134 -219 -186h-115z" />
-<glyph unicode="&#x2dc;" horiz-adv-x="1212" d="M264 1241q13 121 70.5 189.5t148.5 68.5q46 0 89 -18.5t82 -41t75 -41t68 -18.5q49 0 73 29.5t39 91.5h99q-13 -121 -69.5 -189.5t-150.5 -68.5q-43 0 -84 18.5t-80.5 41t-76 41t-70.5 18.5q-50 0 -75.5 -30t-39.5 -91h-98z" />
-<glyph unicode="&#x2000;" horiz-adv-x="953" />
-<glyph unicode="&#x2001;" horiz-adv-x="1907" />
-<glyph unicode="&#x2002;" horiz-adv-x="953" />
-<glyph unicode="&#x2003;" horiz-adv-x="1907" />
-<glyph unicode="&#x2004;" horiz-adv-x="635" />
-<glyph unicode="&#x2005;" horiz-adv-x="476" />
-<glyph unicode="&#x2006;" horiz-adv-x="317" />
-<glyph unicode="&#x2007;" horiz-adv-x="317" />
-<glyph unicode="&#x2008;" horiz-adv-x="238" />
-<glyph unicode="&#x2009;" horiz-adv-x="381" />
-<glyph unicode="&#x200a;" horiz-adv-x="105" />
-<glyph unicode="&#x2010;" horiz-adv-x="659" d="M84 473v152h491v-152h-491z" />
-<glyph unicode="&#x2011;" horiz-adv-x="659" d="M84 473v152h491v-152h-491z" />
-<glyph unicode="&#x2012;" horiz-adv-x="659" d="M84 473v152h491v-152h-491z" />
-<glyph unicode="&#x2013;" horiz-adv-x="1024" d="M82 473v152h860v-152h-860z" />
-<glyph unicode="&#x2014;" horiz-adv-x="2048" d="M82 473v152h1884v-152h-1884z" />
-<glyph unicode="&#x2018;" horiz-adv-x="348" d="M25 983q22 90 71 224t105 255h123q-66 -254 -103 -501h-184z" />
-<glyph unicode="&#x2019;" horiz-adv-x="348" d="M25 961q70 285 102 501h182l15 -22q-26 -100 -75 -232.5t-102 -246.5h-122z" />
-<glyph unicode="&#x201a;" horiz-adv-x="502" d="M63 -264q27 104 59.5 257t45.5 245h182l15 -23q-26 -100 -75 -232.5t-102 -246.5h-125z" />
-<glyph unicode="&#x201c;" horiz-adv-x="717" d="M25 983q22 90 71 224t105 255h123q-66 -254 -103 -501h-184zM391 983q56 215 178 479h123q-30 -115 -59.5 -259.5t-42.5 -241.5h-184z" />
-<glyph unicode="&#x201d;" horiz-adv-x="717" d="M25 961q70 285 102 501h182l15 -22q-26 -100 -75 -232.5t-102 -246.5h-122zM391 961q26 100 59 254t46 247h182l14 -22q-24 -91 -72 -224t-104 -255h-125z" />
-<glyph unicode="&#x201e;" horiz-adv-x="829" d="M25 -263q70 285 102 501h182l15 -22q-26 -100 -75 -232.5t-102 -246.5h-122zM391 -263q26 100 59 254t46 247h182l14 -22q-24 -91 -72 -224t-104 -255h-125z" />
-<glyph unicode="&#x2022;" horiz-adv-x="770" d="M164 748q0 121 56.5 184t164.5 63q105 0 163 -62t58 -185q0 -119 -57.5 -183.5t-163.5 -64.5q-107 0 -164 65.5t-57 182.5z" />
-<glyph unicode="&#x2026;" horiz-adv-x="1606" d="M152 106q0 67 30.5 101.5t87.5 34.5q58 0 90.5 -34.5t32.5 -101.5q0 -65 -33 -100t-90 -35q-51 0 -84.5 31.5t-33.5 103.5zM682 106q0 67 30.5 101.5t87.5 34.5q58 0 90.5 -34.5t32.5 -101.5q0 -65 -33 -100t-90 -35q-51 0 -84.5 31.5t-33.5 103.5zM1213 106 q0 67 30.5 101.5t87.5 34.5q58 0 90.5 -34.5t32.5 -101.5q0 -65 -33 -100t-90 -35q-51 0 -84.5 31.5t-33.5 103.5z" />
-<glyph unicode="&#x202f;" horiz-adv-x="381" />
-<glyph unicode="&#x2039;" horiz-adv-x="623" d="M82 524v27l342 407l119 -69l-289 -350l289 -351l-119 -71z" />
-<glyph unicode="&#x203a;" horiz-adv-x="623" d="M80 188l287 351l-287 350l117 69l344 -407v-27l-344 -407z" />
-<glyph unicode="&#x205f;" horiz-adv-x="476" />
-<glyph unicode="&#x20ac;" horiz-adv-x="1208" d="M63 506v129h152l-2 42v44l2 80h-152v129h164q39 261 185 407t383 146q201 0 366 -97l-71 -139q-166 86 -295 86q-319 0 -398 -403h510v-129h-524l-2 -57v-64l2 -45h463v-129h-447q37 -180 138.5 -278.5t271.5 -98.5q156 0 309 66v-150q-146 -65 -317 -65 q-237 0 -381.5 134.5t-190.5 391.5h-166z" />
-<glyph unicode="&#x2122;" horiz-adv-x="1589" d="M37 1356v106h543v-106h-211v-615h-123v615h-209zM647 741v721h187l196 -559l203 559h180v-721h-127v420l6 137h-8l-211 -557h-104l-201 559h-8l6 -129v-430h-119z" />
-<glyph unicode="&#x25fc;" horiz-adv-x="1095" d="M0 0v1095h1095v-1095h-1095z" />
-<hkern u1="&#x22;" u2="&#x178;" k="-20" />
-<hkern u1="&#x22;" u2="&#x153;" k="123" />
-<hkern u1="&#x22;" u2="&#xfc;" k="61" />
-<hkern u1="&#x22;" u2="&#xfb;" k="61" />
-<hkern u1="&#x22;" u2="&#xfa;" k="61" />
-<hkern u1="&#x22;" u2="&#xf9;" k="61" />
-<hkern u1="&#x22;" u2="&#xf8;" k="123" />
-<hkern u1="&#x22;" u2="&#xf6;" k="123" />
-<hkern u1="&#x22;" u2="&#xf5;" k="123" />
-<hkern u1="&#x22;" u2="&#xf4;" k="123" />
-<hkern u1="&#x22;" u2="&#xf3;" k="123" />
-<hkern u1="&#x22;" u2="&#xf2;" k="123" />
-<hkern u1="&#x22;" u2="&#xeb;" k="123" />
-<hkern u1="&#x22;" u2="&#xea;" k="123" />
-<hkern u1="&#x22;" u2="&#xe9;" k="123" />
-<hkern u1="&#x22;" u2="&#xe8;" k="123" />
-<hkern u1="&#x22;" u2="&#xe7;" k="123" />
-<hkern u1="&#x22;" u2="&#xe6;" k="82" />
-<hkern u1="&#x22;" u2="&#xe5;" k="82" />
-<hkern u1="&#x22;" u2="&#xe4;" k="82" />
-<hkern u1="&#x22;" u2="&#xe3;" k="82" />
-<hkern u1="&#x22;" u2="&#xe2;" k="82" />
-<hkern u1="&#x22;" u2="&#xe1;" k="82" />
-<hkern u1="&#x22;" u2="&#xe0;" k="123" />
-<hkern u1="&#x22;" u2="&#xdd;" k="-20" />
-<hkern u1="&#x22;" u2="&#xc5;" k="143" />
-<hkern u1="&#x22;" u2="&#xc4;" k="143" />
-<hkern u1="&#x22;" u2="&#xc3;" k="143" />
-<hkern u1="&#x22;" u2="&#xc2;" k="143" />
-<hkern u1="&#x22;" u2="&#xc1;" k="143" />
-<hkern u1="&#x22;" u2="&#xc0;" k="143" />
-<hkern u1="&#x22;" u2="u" k="61" />
-<hkern u1="&#x22;" u2="s" k="61" />
-<hkern u1="&#x22;" u2="r" k="61" />
-<hkern u1="&#x22;" u2="q" k="123" />
-<hkern u1="&#x22;" u2="p" k="61" />
-<hkern u1="&#x22;" u2="o" k="123" />
-<hkern u1="&#x22;" u2="n" k="61" />
-<hkern u1="&#x22;" u2="m" k="61" />
-<hkern u1="&#x22;" u2="g" k="61" />
-<hkern u1="&#x22;" u2="e" k="123" />
-<hkern u1="&#x22;" u2="d" k="123" />
-<hkern u1="&#x22;" u2="c" k="123" />
-<hkern u1="&#x22;" u2="a" k="82" />
-<hkern u1="&#x22;" u2="Y" k="-20" />
-<hkern u1="&#x22;" u2="W" k="-41" />
-<hkern u1="&#x22;" u2="V" k="-41" />
-<hkern u1="&#x22;" u2="T" k="-41" />
-<hkern u1="&#x22;" u2="A" k="143" />
-<hkern u1="&#x27;" u2="&#x178;" k="-20" />
-<hkern u1="&#x27;" u2="&#x153;" k="123" />
-<hkern u1="&#x27;" u2="&#xfc;" k="61" />
-<hkern u1="&#x27;" u2="&#xfb;" k="61" />
-<hkern u1="&#x27;" u2="&#xfa;" k="61" />
-<hkern u1="&#x27;" u2="&#xf9;" k="61" />
-<hkern u1="&#x27;" u2="&#xf8;" k="123" />
-<hkern u1="&#x27;" u2="&#xf6;" k="123" />
-<hkern u1="&#x27;" u2="&#xf5;" k="123" />
-<hkern u1="&#x27;" u2="&#xf4;" k="123" />
-<hkern u1="&#x27;" u2="&#xf3;" k="123" />
-<hkern u1="&#x27;" u2="&#xf2;" k="123" />
-<hkern u1="&#x27;" u2="&#xeb;" k="123" />
-<hkern u1="&#x27;" u2="&#xea;" k="123" />
-<hkern u1="&#x27;" u2="&#xe9;" k="123" />
-<hkern u1="&#x27;" u2="&#xe8;" k="123" />
-<hkern u1="&#x27;" u2="&#xe7;" k="123" />
-<hkern u1="&#x27;" u2="&#xe6;" k="82" />
-<hkern u1="&#x27;" u2="&#xe5;" k="82" />
-<hkern u1="&#x27;" u2="&#xe4;" k="82" />
-<hkern u1="&#x27;" u2="&#xe3;" k="82" />
-<hkern u1="&#x27;" u2="&#xe2;" k="82" />
-<hkern u1="&#x27;" u2="&#xe1;" k="82" />
-<hkern u1="&#x27;" u2="&#xe0;" k="123" />
-<hkern u1="&#x27;" u2="&#xdd;" k="-20" />
-<hkern u1="&#x27;" u2="&#xc5;" k="143" />
-<hkern u1="&#x27;" u2="&#xc4;" k="143" />
-<hkern u1="&#x27;" u2="&#xc3;" k="143" />
-<hkern u1="&#x27;" u2="&#xc2;" k="143" />
-<hkern u1="&#x27;" u2="&#xc1;" k="143" />
-<hkern u1="&#x27;" u2="&#xc0;" k="143" />
-<hkern u1="&#x27;" u2="u" k="61" />
-<hkern u1="&#x27;" u2="s" k="61" />
-<hkern u1="&#x27;" u2="r" k="61" />
-<hkern u1="&#x27;" u2="q" k="123" />
-<hkern u1="&#x27;" u2="p" k="61" />
-<hkern u1="&#x27;" u2="o" k="123" />
-<hkern u1="&#x27;" u2="n" k="61" />
-<hkern u1="&#x27;" u2="m" k="61" />
-<hkern u1="&#x27;" u2="g" k="61" />
-<hkern u1="&#x27;" u2="e" k="123" />
-<hkern u1="&#x27;" u2="d" k="123" />
-<hkern u1="&#x27;" u2="c" k="123" />
-<hkern u1="&#x27;" u2="a" k="82" />
-<hkern u1="&#x27;" u2="Y" k="-20" />
-<hkern u1="&#x27;" u2="W" k="-41" />
-<hkern u1="&#x27;" u2="V" k="-41" />
-<hkern u1="&#x27;" u2="T" k="-41" />
-<hkern u1="&#x27;" u2="A" k="143" />
-<hkern u1="&#x28;" u2="J" k="-184" />
-<hkern u1="&#x2c;" u2="&#x178;" k="123" />
-<hkern u1="&#x2c;" u2="&#x152;" k="102" />
-<hkern u1="&#x2c;" u2="&#xdd;" k="123" />
-<hkern u1="&#x2c;" u2="&#xdc;" k="41" />
-<hkern u1="&#x2c;" u2="&#xdb;" k="41" />
-<hkern u1="&#x2c;" u2="&#xda;" k="41" />
-<hkern u1="&#x2c;" u2="&#xd9;" k="41" />
-<hkern u1="&#x2c;" u2="&#xd8;" k="102" />
-<hkern u1="&#x2c;" u2="&#xd6;" k="102" />
-<hkern u1="&#x2c;" u2="&#xd5;" k="102" />
-<hkern u1="&#x2c;" u2="&#xd4;" k="102" />
-<hkern u1="&#x2c;" u2="&#xd3;" k="102" />
-<hkern u1="&#x2c;" u2="&#xd2;" k="102" />
-<hkern u1="&#x2c;" u2="&#xc7;" k="102" />
-<hkern u1="&#x2c;" u2="Y" k="123" />
-<hkern u1="&#x2c;" u2="W" k="123" />
-<hkern u1="&#x2c;" u2="V" k="123" />
-<hkern u1="&#x2c;" u2="U" k="41" />
-<hkern u1="&#x2c;" u2="T" k="143" />
-<hkern u1="&#x2c;" u2="Q" k="102" />
-<hkern u1="&#x2c;" u2="O" k="102" />
-<hkern u1="&#x2c;" u2="G" k="102" />
-<hkern u1="&#x2c;" u2="C" k="102" />
-<hkern u1="&#x2d;" u2="T" k="82" />
-<hkern u1="&#x2e;" u2="&#x178;" k="123" />
-<hkern u1="&#x2e;" u2="&#x152;" k="102" />
-<hkern u1="&#x2e;" u2="&#xdd;" k="123" />
-<hkern u1="&#x2e;" u2="&#xdc;" k="41" />
-<hkern u1="&#x2e;" u2="&#xdb;" k="41" />
-<hkern u1="&#x2e;" u2="&#xda;" k="41" />
-<hkern u1="&#x2e;" u2="&#xd9;" k="41" />
-<hkern u1="&#x2e;" u2="&#xd8;" k="102" />
-<hkern u1="&#x2e;" u2="&#xd6;" k="102" />
-<hkern u1="&#x2e;" u2="&#xd5;" k="102" />
-<hkern u1="&#x2e;" u2="&#xd4;" k="102" />
-<hkern u1="&#x2e;" u2="&#xd3;" k="102" />
-<hkern u1="&#x2e;" u2="&#xd2;" k="102" />
-<hkern u1="&#x2e;" u2="&#xc7;" k="102" />
-<hkern u1="&#x2e;" u2="Y" k="123" />
-<hkern u1="&#x2e;" u2="W" k="123" />
-<hkern u1="&#x2e;" u2="V" k="123" />
-<hkern u1="&#x2e;" u2="U" k="41" />
-<hkern u1="&#x2e;" u2="T" k="143" />
-<hkern u1="&#x2e;" u2="Q" k="102" />
-<hkern u1="&#x2e;" u2="O" k="102" />
-<hkern u1="&#x2e;" u2="G" k="102" />
-<hkern u1="&#x2e;" u2="C" k="102" />
-<hkern u1="A" u2="&#x201d;" k="143" />
-<hkern u1="A" u2="&#x2019;" k="143" />
-<hkern u1="A" u2="&#x178;" k="123" />
-<hkern u1="A" u2="&#x152;" k="41" />
-<hkern u1="A" u2="&#xdd;" k="123" />
-<hkern u1="A" u2="&#xd8;" k="41" />
-<hkern u1="A" u2="&#xd6;" k="41" />
-<hkern u1="A" u2="&#xd5;" k="41" />
-<hkern u1="A" u2="&#xd4;" k="41" />
-<hkern u1="A" u2="&#xd3;" k="41" />
-<hkern u1="A" u2="&#xd2;" k="41" />
-<hkern u1="A" u2="&#xc7;" k="41" />
-<hkern u1="A" u2="Y" k="123" />
-<hkern u1="A" u2="W" k="82" />
-<hkern u1="A" u2="V" k="82" />
-<hkern u1="A" u2="T" k="143" />
-<hkern u1="A" u2="Q" k="41" />
-<hkern u1="A" u2="O" k="41" />
-<hkern u1="A" u2="J" k="-266" />
-<hkern u1="A" u2="G" k="41" />
-<hkern u1="A" u2="C" k="41" />
-<hkern u1="A" u2="&#x27;" k="143" />
-<hkern u1="A" u2="&#x22;" k="143" />
-<hkern u1="B" u2="&#x201e;" k="82" />
-<hkern u1="B" u2="&#x201a;" k="82" />
-<hkern u1="B" u2="&#x178;" k="20" />
-<hkern u1="B" u2="&#xdd;" k="20" />
-<hkern u1="B" u2="&#xc5;" k="41" />
-<hkern u1="B" u2="&#xc4;" k="41" />
-<hkern u1="B" u2="&#xc3;" k="41" />
-<hkern u1="B" u2="&#xc2;" k="41" />
-<hkern u1="B" u2="&#xc1;" k="41" />
-<hkern u1="B" u2="&#xc0;" k="41" />
-<hkern u1="B" u2="Z" k="20" />
-<hkern u1="B" u2="Y" k="20" />
-<hkern u1="B" u2="X" k="41" />
-<hkern u1="B" u2="W" k="20" />
-<hkern u1="B" u2="V" k="20" />
-<hkern u1="B" u2="T" k="61" />
-<hkern u1="B" u2="A" k="41" />
-<hkern u1="B" u2="&#x2e;" k="82" />
-<hkern u1="B" u2="&#x2c;" k="82" />
-<hkern u1="C" u2="&#x152;" k="41" />
-<hkern u1="C" u2="&#xd8;" k="41" />
-<hkern u1="C" u2="&#xd6;" k="41" />
-<hkern u1="C" u2="&#xd5;" k="41" />
-<hkern u1="C" u2="&#xd4;" k="41" />
-<hkern u1="C" u2="&#xd3;" k="41" />
-<hkern u1="C" u2="&#xd2;" k="41" />
-<hkern u1="C" u2="&#xc7;" k="41" />
-<hkern u1="C" u2="Q" k="41" />
-<hkern u1="C" u2="O" k="41" />
-<hkern u1="C" u2="G" k="41" />
-<hkern u1="C" u2="C" k="41" />
-<hkern u1="D" u2="&#x201e;" k="82" />
-<hkern u1="D" u2="&#x201a;" k="82" />
-<hkern u1="D" u2="&#x178;" k="20" />
-<hkern u1="D" u2="&#xdd;" k="20" />
-<hkern u1="D" u2="&#xc5;" k="41" />
-<hkern u1="D" u2="&#xc4;" k="41" />
-<hkern u1="D" u2="&#xc3;" k="41" />
-<hkern u1="D" u2="&#xc2;" k="41" />
-<hkern u1="D" u2="&#xc1;" k="41" />
-<hkern u1="D" u2="&#xc0;" k="41" />
-<hkern u1="D" u2="Z" k="20" />
-<hkern u1="D" u2="Y" k="20" />
-<hkern u1="D" u2="X" k="41" />
-<hkern u1="D" u2="W" k="20" />
-<hkern u1="D" u2="V" k="20" />
-<hkern u1="D" u2="T" k="61" />
-<hkern u1="D" u2="A" k="41" />
-<hkern u1="D" u2="&#x2e;" k="82" />
-<hkern u1="D" u2="&#x2c;" k="82" />
-<hkern u1="E" u2="J" k="-123" />
-<hkern u1="F" u2="&#x201e;" k="123" />
-<hkern u1="F" u2="&#x201a;" k="123" />
-<hkern u1="F" u2="&#xc5;" k="41" />
-<hkern u1="F" u2="&#xc4;" k="41" />
-<hkern u1="F" u2="&#xc3;" k="41" />
-<hkern u1="F" u2="&#xc2;" k="41" />
-<hkern u1="F" u2="&#xc1;" k="41" />
-<hkern u1="F" u2="&#xc0;" k="41" />
-<hkern u1="F" u2="A" k="41" />
-<hkern u1="F" u2="&#x3f;" k="-41" />
-<hkern u1="F" u2="&#x2e;" k="123" />
-<hkern u1="F" u2="&#x2c;" k="123" />
-<hkern u1="K" u2="&#x152;" k="41" />
-<hkern u1="K" u2="&#xd8;" k="41" />
-<hkern u1="K" u2="&#xd6;" k="41" />
-<hkern u1="K" u2="&#xd5;" k="41" />
-<hkern u1="K" u2="&#xd4;" k="41" />
-<hkern u1="K" u2="&#xd3;" k="41" />
-<hkern u1="K" u2="&#xd2;" k="41" />
-<hkern u1="K" u2="&#xc7;" k="41" />
-<hkern u1="K" u2="Q" k="41" />
-<hkern u1="K" u2="O" k="41" />
-<hkern u1="K" u2="G" k="41" />
-<hkern u1="K" u2="C" k="41" />
-<hkern u1="L" u2="&#x201d;" k="164" />
-<hkern u1="L" u2="&#x2019;" k="164" />
-<hkern u1="L" u2="&#x178;" k="61" />
-<hkern u1="L" u2="&#x152;" k="41" />
-<hkern u1="L" u2="&#xdd;" k="61" />
-<hkern u1="L" u2="&#xdc;" k="20" />
-<hkern u1="L" u2="&#xdb;" k="20" />
-<hkern u1="L" u2="&#xda;" k="20" />
-<hkern u1="L" u2="&#xd9;" k="20" />
-<hkern u1="L" u2="&#xd8;" k="41" />
-<hkern u1="L" u2="&#xd6;" k="41" />
-<hkern u1="L" u2="&#xd5;" k="41" />
-<hkern u1="L" u2="&#xd4;" k="41" />
-<hkern u1="L" u2="&#xd3;" k="41" />
-<hkern u1="L" u2="&#xd2;" k="41" />
-<hkern u1="L" u2="&#xc7;" k="41" />
-<hkern u1="L" u2="Y" k="61" />
-<hkern u1="L" u2="W" k="41" />
-<hkern u1="L" u2="V" k="41" />
-<hkern u1="L" u2="U" k="20" />
-<hkern u1="L" u2="T" k="41" />
-<hkern u1="L" u2="Q" k="41" />
-<hkern u1="L" u2="O" k="41" />
-<hkern u1="L" u2="G" k="41" />
-<hkern u1="L" u2="C" k="41" />
-<hkern u1="L" u2="&#x27;" k="164" />
-<hkern u1="L" u2="&#x22;" k="164" />
-<hkern u1="O" u2="&#x201e;" k="82" />
-<hkern u1="O" u2="&#x201a;" k="82" />
-<hkern u1="O" u2="&#x178;" k="20" />
-<hkern u1="O" u2="&#xdd;" k="20" />
-<hkern u1="O" u2="&#xc5;" k="41" />
-<hkern u1="O" u2="&#xc4;" k="41" />
-<hkern u1="O" u2="&#xc3;" k="41" />
-<hkern u1="O" u2="&#xc2;" k="41" />
-<hkern u1="O" u2="&#xc1;" k="41" />
-<hkern u1="O" u2="&#xc0;" k="41" />
-<hkern u1="O" u2="Z" k="20" />
-<hkern u1="O" u2="Y" k="20" />
-<hkern u1="O" u2="X" k="41" />
-<hkern u1="O" u2="W" k="20" />
-<hkern u1="O" u2="V" k="20" />
-<hkern u1="O" u2="T" k="61" />
-<hkern u1="O" u2="A" k="41" />
-<hkern u1="O" u2="&#x2e;" k="82" />
-<hkern u1="O" u2="&#x2c;" k="82" />
-<hkern u1="P" u2="&#x201e;" k="266" />
-<hkern u1="P" u2="&#x201a;" k="266" />
-<hkern u1="P" u2="&#xc5;" k="102" />
-<hkern u1="P" u2="&#xc4;" k="102" />
-<hkern u1="P" u2="&#xc3;" k="102" />
-<hkern u1="P" u2="&#xc2;" k="102" />
-<hkern u1="P" u2="&#xc1;" k="102" />
-<hkern u1="P" u2="&#xc0;" k="102" />
-<hkern u1="P" u2="Z" k="20" />
-<hkern u1="P" u2="X" k="41" />
-<hkern u1="P" u2="A" k="102" />
-<hkern u1="P" u2="&#x2e;" k="266" />
-<hkern u1="P" u2="&#x2c;" k="266" />
-<hkern u1="Q" u2="&#x201e;" k="82" />
-<hkern u1="Q" u2="&#x201a;" k="82" />
-<hkern u1="Q" u2="&#x178;" k="20" />
-<hkern u1="Q" u2="&#xdd;" k="20" />
-<hkern u1="Q" u2="&#xc5;" k="41" />
-<hkern u1="Q" u2="&#xc4;" k="41" />
-<hkern u1="Q" u2="&#xc3;" k="41" />
-<hkern u1="Q" u2="&#xc2;" k="41" />
-<hkern u1="Q" u2="&#xc1;" k="41" />
-<hkern u1="Q" u2="&#xc0;" k="41" />
-<hkern u1="Q" u2="Z" k="20" />
-<hkern u1="Q" u2="Y" k="20" />
-<hkern u1="Q" u2="X" k="41" />
-<hkern u1="Q" u2="W" k="20" />
-<hkern u1="Q" u2="V" k="20" />
-<hkern u1="Q" u2="T" k="61" />
-<hkern u1="Q" u2="A" k="41" />
-<hkern u1="Q" u2="&#x2e;" k="82" />
-<hkern u1="Q" u2="&#x2c;" k="82" />
-<hkern u1="T" u2="&#x201e;" k="123" />
-<hkern u1="T" u2="&#x201a;" k="123" />
-<hkern u1="T" u2="&#x2014;" k="82" />
-<hkern u1="T" u2="&#x2013;" k="82" />
-<hkern u1="T" u2="&#x153;" k="143" />
-<hkern u1="T" u2="&#x152;" k="41" />
-<hkern u1="T" u2="&#xfd;" k="41" />
-<hkern u1="T" u2="&#xfc;" k="102" />
-<hkern u1="T" u2="&#xfb;" k="102" />
-<hkern u1="T" u2="&#xfa;" k="102" />
-<hkern u1="T" u2="&#xf9;" k="102" />
-<hkern u1="T" u2="&#xf8;" k="143" />
-<hkern u1="T" u2="&#xf6;" k="143" />
-<hkern u1="T" u2="&#xf5;" k="143" />
-<hkern u1="T" u2="&#xf4;" k="143" />
-<hkern u1="T" u2="&#xf3;" k="143" />
-<hkern u1="T" u2="&#xf2;" k="143" />
-<hkern u1="T" u2="&#xeb;" k="143" />
-<hkern u1="T" u2="&#xea;" k="143" />
-<hkern u1="T" u2="&#xe9;" k="143" />
-<hkern u1="T" u2="&#xe8;" k="143" />
-<hkern u1="T" u2="&#xe7;" k="143" />
-<hkern u1="T" u2="&#xe6;" k="164" />
-<hkern u1="T" u2="&#xe5;" k="164" />
-<hkern u1="T" u2="&#xe4;" k="164" />
-<hkern u1="T" u2="&#xe3;" k="164" />
-<hkern u1="T" u2="&#xe2;" k="164" />
-<hkern u1="T" u2="&#xe1;" k="164" />
-<hkern u1="T" u2="&#xe0;" k="143" />
-<hkern u1="T" u2="&#xd8;" k="41" />
-<hkern u1="T" u2="&#xd6;" k="41" />
-<hkern u1="T" u2="&#xd5;" k="41" />
-<hkern u1="T" u2="&#xd4;" k="41" />
-<hkern u1="T" u2="&#xd3;" k="41" />
-<hkern u1="T" u2="&#xd2;" k="41" />
-<hkern u1="T" u2="&#xc7;" k="41" />
-<hkern u1="T" u2="&#xc5;" k="143" />
-<hkern u1="T" u2="&#xc4;" k="143" />
-<hkern u1="T" u2="&#xc3;" k="143" />
-<hkern u1="T" u2="&#xc2;" k="143" />
-<hkern u1="T" u2="&#xc1;" k="143" />
-<hkern u1="T" u2="&#xc0;" k="143" />
-<hkern u1="T" u2="z" k="82" />
-<hkern u1="T" u2="y" k="41" />
-<hkern u1="T" u2="x" k="41" />
-<hkern u1="T" u2="w" k="41" />
-<hkern u1="T" u2="v" k="41" />
-<hkern u1="T" u2="u" k="102" />
-<hkern u1="T" u2="s" k="123" />
-<hkern u1="T" u2="r" k="102" />
-<hkern u1="T" u2="q" k="143" />
-<hkern u1="T" u2="p" k="102" />
-<hkern u1="T" u2="o" k="143" />
-<hkern u1="T" u2="n" k="102" />
-<hkern u1="T" u2="m" k="102" />
-<hkern u1="T" u2="g" k="143" />
-<hkern u1="T" u2="e" k="143" />
-<hkern u1="T" u2="d" k="143" />
-<hkern u1="T" u2="c" k="143" />
-<hkern u1="T" u2="a" k="164" />
-<hkern u1="T" u2="T" k="-41" />
-<hkern u1="T" u2="Q" k="41" />
-<hkern u1="T" u2="O" k="41" />
-<hkern u1="T" u2="G" k="41" />
-<hkern u1="T" u2="C" k="41" />
-<hkern u1="T" u2="A" k="143" />
-<hkern u1="T" u2="&#x3f;" k="-41" />
-<hkern u1="T" u2="&#x2e;" k="123" />
-<hkern u1="T" u2="&#x2d;" k="82" />
-<hkern u1="T" u2="&#x2c;" k="123" />
-<hkern u1="U" u2="&#x201e;" k="41" />
-<hkern u1="U" u2="&#x201a;" k="41" />
-<hkern u1="U" u2="&#xc5;" k="20" />
-<hkern u1="U" u2="&#xc4;" k="20" />
-<hkern u1="U" u2="&#xc3;" k="20" />
-<hkern u1="U" u2="&#xc2;" k="20" />
-<hkern u1="U" u2="&#xc1;" k="20" />
-<hkern u1="U" u2="&#xc0;" k="20" />
-<hkern u1="U" u2="A" k="20" />
-<hkern u1="U" u2="&#x2e;" k="41" />
-<hkern u1="U" u2="&#x2c;" k="41" />
-<hkern u1="V" u2="&#x201e;" k="102" />
-<hkern u1="V" u2="&#x201a;" k="102" />
-<hkern u1="V" u2="&#x153;" k="41" />
-<hkern u1="V" u2="&#x152;" k="20" />
-<hkern u1="V" u2="&#xfc;" k="20" />
-<hkern u1="V" u2="&#xfb;" k="20" />
-<hkern u1="V" u2="&#xfa;" k="20" />
-<hkern u1="V" u2="&#xf9;" k="20" />
-<hkern u1="V" u2="&#xf8;" k="41" />
-<hkern u1="V" u2="&#xf6;" k="41" />
-<hkern u1="V" u2="&#xf5;" k="41" />
-<hkern u1="V" u2="&#xf4;" k="41" />
-<hkern u1="V" u2="&#xf3;" k="41" />
-<hkern u1="V" u2="&#xf2;" k="41" />
-<hkern u1="V" u2="&#xeb;" k="41" />
-<hkern u1="V" u2="&#xea;" k="41" />
-<hkern u1="V" u2="&#xe9;" k="41" />
-<hkern u1="V" u2="&#xe8;" k="41" />
-<hkern u1="V" u2="&#xe7;" k="41" />
-<hkern u1="V" u2="&#xe6;" k="41" />
-<hkern u1="V" u2="&#xe5;" k="41" />
-<hkern u1="V" u2="&#xe4;" k="41" />
-<hkern u1="V" u2="&#xe3;" k="41" />
-<hkern u1="V" u2="&#xe2;" k="41" />
-<hkern u1="V" u2="&#xe1;" k="41" />
-<hkern u1="V" u2="&#xe0;" k="41" />
-<hkern u1="V" u2="&#xd8;" k="20" />
-<hkern u1="V" u2="&#xd6;" k="20" />
-<hkern u1="V" u2="&#xd5;" k="20" />
-<hkern u1="V" u2="&#xd4;" k="20" />
-<hkern u1="V" u2="&#xd3;" k="20" />
-<hkern u1="V" u2="&#xd2;" k="20" />
-<hkern u1="V" u2="&#xc7;" k="20" />
-<hkern u1="V" u2="&#xc5;" k="82" />
-<hkern u1="V" u2="&#xc4;" k="82" />
-<hkern u1="V" u2="&#xc3;" k="82" />
-<hkern u1="V" u2="&#xc2;" k="82" />
-<hkern u1="V" u2="&#xc1;" k="82" />
-<hkern u1="V" u2="&#xc0;" k="82" />
-<hkern u1="V" u2="u" k="20" />
-<hkern u1="V" u2="s" k="20" />
-<hkern u1="V" u2="r" k="20" />
-<hkern u1="V" u2="q" k="41" />
-<hkern u1="V" u2="p" k="20" />
-<hkern u1="V" u2="o" k="41" />
-<hkern u1="V" u2="n" k="20" />
-<hkern u1="V" u2="m" k="20" />
-<hkern u1="V" u2="g" k="20" />
-<hkern u1="V" u2="e" k="41" />
-<hkern u1="V" u2="d" k="41" />
-<hkern u1="V" u2="c" k="41" />
-<hkern u1="V" u2="a" k="41" />
-<hkern u1="V" u2="Q" k="20" />
-<hkern u1="V" u2="O" k="20" />
-<hkern u1="V" u2="G" k="20" />
-<hkern u1="V" u2="C" k="20" />
-<hkern u1="V" u2="A" k="82" />
-<hkern u1="V" u2="&#x3f;" k="-41" />
-<hkern u1="V" u2="&#x2e;" k="102" />
-<hkern u1="V" u2="&#x2c;" k="102" />
-<hkern u1="W" u2="&#x201e;" k="102" />
-<hkern u1="W" u2="&#x201a;" k="102" />
-<hkern u1="W" u2="&#x153;" k="41" />
-<hkern u1="W" u2="&#x152;" k="20" />
-<hkern u1="W" u2="&#xfc;" k="20" />
-<hkern u1="W" u2="&#xfb;" k="20" />
-<hkern u1="W" u2="&#xfa;" k="20" />
-<hkern u1="W" u2="&#xf9;" k="20" />
-<hkern u1="W" u2="&#xf8;" k="41" />
-<hkern u1="W" u2="&#xf6;" k="41" />
-<hkern u1="W" u2="&#xf5;" k="41" />
-<hkern u1="W" u2="&#xf4;" k="41" />
-<hkern u1="W" u2="&#xf3;" k="41" />
-<hkern u1="W" u2="&#xf2;" k="41" />
-<hkern u1="W" u2="&#xeb;" k="41" />
-<hkern u1="W" u2="&#xea;" k="41" />
-<hkern u1="W" u2="&#xe9;" k="41" />
-<hkern u1="W" u2="&#xe8;" k="41" />
-<hkern u1="W" u2="&#xe7;" k="41" />
-<hkern u1="W" u2="&#xe6;" k="41" />
-<hkern u1="W" u2="&#xe5;" k="41" />
-<hkern u1="W" u2="&#xe4;" k="41" />
-<hkern u1="W" u2="&#xe3;" k="41" />
-<hkern u1="W" u2="&#xe2;" k="41" />
-<hkern u1="W" u2="&#xe1;" k="41" />
-<hkern u1="W" u2="&#xe0;" k="41" />
-<hkern u1="W" u2="&#xd8;" k="20" />
-<hkern u1="W" u2="&#xd6;" k="20" />
-<hkern u1="W" u2="&#xd5;" k="20" />
-<hkern u1="W" u2="&#xd4;" k="20" />
-<hkern u1="W" u2="&#xd3;" k="20" />
-<hkern u1="W" u2="&#xd2;" k="20" />
-<hkern u1="W" u2="&#xc7;" k="20" />
-<hkern u1="W" u2="&#xc5;" k="82" />
-<hkern u1="W" u2="&#xc4;" k="82" />
-<hkern u1="W" u2="&#xc3;" k="82" />
-<hkern u1="W" u2="&#xc2;" k="82" />
-<hkern u1="W" u2="&#xc1;" k="82" />
-<hkern u1="W" u2="&#xc0;" k="82" />
-<hkern u1="W" u2="u" k="20" />
-<hkern u1="W" u2="s" k="20" />
-<hkern u1="W" u2="r" k="20" />
-<hkern u1="W" u2="q" k="41" />
-<hkern u1="W" u2="p" k="20" />
-<hkern u1="W" u2="o" k="41" />
-<hkern u1="W" u2="n" k="20" />
-<hkern u1="W" u2="m" k="20" />
-<hkern u1="W" u2="g" k="20" />
-<hkern u1="W" u2="e" k="41" />
-<hkern u1="W" u2="d" k="41" />
-<hkern u1="W" u2="c" k="41" />
-<hkern u1="W" u2="a" k="41" />
-<hkern u1="W" u2="Q" k="20" />
-<hkern u1="W" u2="O" k="20" />
-<hkern u1="W" u2="G" k="20" />
-<hkern u1="W" u2="C" k="20" />
-<hkern u1="W" u2="A" k="82" />
-<hkern u1="W" u2="&#x3f;" k="-41" />
-<hkern u1="W" u2="&#x2e;" k="102" />
-<hkern u1="W" u2="&#x2c;" k="102" />
-<hkern u1="X" u2="&#x152;" k="41" />
-<hkern u1="X" u2="&#xd8;" k="41" />
-<hkern u1="X" u2="&#xd6;" k="41" />
-<hkern u1="X" u2="&#xd5;" k="41" />
-<hkern u1="X" u2="&#xd4;" k="41" />
-<hkern u1="X" u2="&#xd3;" k="41" />
-<hkern u1="X" u2="&#xd2;" k="41" />
-<hkern u1="X" u2="&#xc7;" k="41" />
-<hkern u1="X" u2="Q" k="41" />
-<hkern u1="X" u2="O" k="41" />
-<hkern u1="X" u2="G" k="41" />
-<hkern u1="X" u2="C" k="41" />
-<hkern u1="Y" u2="&#x201e;" k="123" />
-<hkern u1="Y" u2="&#x201a;" k="123" />
-<hkern u1="Y" u2="&#x153;" k="102" />
-<hkern u1="Y" u2="&#x152;" k="41" />
-<hkern u1="Y" u2="&#xfc;" k="61" />
-<hkern u1="Y" u2="&#xfb;" k="61" />
-<hkern u1="Y" u2="&#xfa;" k="61" />
-<hkern u1="Y" u2="&#xf9;" k="61" />
-<hkern u1="Y" u2="&#xf8;" k="102" />
-<hkern u1="Y" u2="&#xf6;" k="102" />
-<hkern u1="Y" u2="&#xf5;" k="102" />
-<hkern u1="Y" u2="&#xf4;" k="102" />
-<hkern u1="Y" u2="&#xf3;" k="102" />
-<hkern u1="Y" u2="&#xf2;" k="102" />
-<hkern u1="Y" u2="&#xeb;" k="102" />
-<hkern u1="Y" u2="&#xea;" k="102" />
-<hkern u1="Y" u2="&#xe9;" k="102" />
-<hkern u1="Y" u2="&#xe8;" k="102" />
-<hkern u1="Y" u2="&#xe7;" k="102" />
-<hkern u1="Y" u2="&#xe6;" k="102" />
-<hkern u1="Y" u2="&#xe5;" k="102" />
-<hkern u1="Y" u2="&#xe4;" k="102" />
-<hkern u1="Y" u2="&#xe3;" k="102" />
-<hkern u1="Y" u2="&#xe2;" k="102" />
-<hkern u1="Y" u2="&#xe1;" k="102" />
-<hkern u1="Y" u2="&#xe0;" k="102" />
-<hkern u1="Y" u2="&#xd8;" k="41" />
-<hkern u1="Y" u2="&#xd6;" k="41" />
-<hkern u1="Y" u2="&#xd5;" k="41" />
-<hkern u1="Y" u2="&#xd4;" k="41" />
-<hkern u1="Y" u2="&#xd3;" k="41" />
-<hkern u1="Y" u2="&#xd2;" k="41" />
-<hkern u1="Y" u2="&#xc7;" k="41" />
-<hkern u1="Y" u2="&#xc5;" k="123" />
-<hkern u1="Y" u2="&#xc4;" k="123" />
-<hkern u1="Y" u2="&#xc3;" k="123" />
-<hkern u1="Y" u2="&#xc2;" k="123" />
-<hkern u1="Y" u2="&#xc1;" k="123" />
-<hkern u1="Y" u2="&#xc0;" k="123" />
-<hkern u1="Y" u2="z" k="41" />
-<hkern u1="Y" u2="u" k="61" />
-<hkern u1="Y" u2="s" k="82" />
-<hkern u1="Y" u2="r" k="61" />
-<hkern u1="Y" u2="q" k="102" />
-<hkern u1="Y" u2="p" k="61" />
-<hkern u1="Y" u2="o" k="102" />
-<hkern u1="Y" u2="n" k="61" />
-<hkern u1="Y" u2="m" k="61" />
-<hkern u1="Y" u2="g" k="41" />
-<hkern u1="Y" u2="e" k="102" />
-<hkern u1="Y" u2="d" k="102" />
-<hkern u1="Y" u2="c" k="102" />
-<hkern u1="Y" u2="a" k="102" />
-<hkern u1="Y" u2="Q" k="41" />
-<hkern u1="Y" u2="O" k="41" />
-<hkern u1="Y" u2="G" k="41" />
-<hkern u1="Y" u2="C" k="41" />
-<hkern u1="Y" u2="A" k="123" />
-<hkern u1="Y" u2="&#x3f;" k="-41" />
-<hkern u1="Y" u2="&#x2e;" k="123" />
-<hkern u1="Y" u2="&#x2c;" k="123" />
-<hkern u1="Z" u2="&#x152;" k="20" />
-<hkern u1="Z" u2="&#xd8;" k="20" />
-<hkern u1="Z" u2="&#xd6;" k="20" />
-<hkern u1="Z" u2="&#xd5;" k="20" />
-<hkern u1="Z" u2="&#xd4;" k="20" />
-<hkern u1="Z" u2="&#xd3;" k="20" />
-<hkern u1="Z" u2="&#xd2;" k="20" />
-<hkern u1="Z" u2="&#xc7;" k="20" />
-<hkern u1="Z" u2="Q" k="20" />
-<hkern u1="Z" u2="O" k="20" />
-<hkern u1="Z" u2="G" k="20" />
-<hkern u1="Z" u2="C" k="20" />
-<hkern u1="[" u2="J" k="-184" />
-<hkern u1="a" u2="&#x201d;" k="20" />
-<hkern u1="a" u2="&#x2019;" k="20" />
-<hkern u1="a" u2="&#x27;" k="20" />
-<hkern u1="a" u2="&#x22;" k="20" />
-<hkern u1="b" u2="&#x201d;" k="20" />
-<hkern u1="b" u2="&#x2019;" k="20" />
-<hkern u1="b" u2="&#xfd;" k="41" />
-<hkern u1="b" u2="z" k="20" />
-<hkern u1="b" u2="y" k="41" />
-<hkern u1="b" u2="x" k="41" />
-<hkern u1="b" u2="w" k="41" />
-<hkern u1="b" u2="v" k="41" />
-<hkern u1="b" u2="&#x27;" k="20" />
-<hkern u1="b" u2="&#x22;" k="20" />
-<hkern u1="c" u2="&#x201d;" k="-41" />
-<hkern u1="c" u2="&#x2019;" k="-41" />
-<hkern u1="c" u2="&#x27;" k="-41" />
-<hkern u1="c" u2="&#x22;" k="-41" />
-<hkern u1="e" u2="&#x201d;" k="20" />
-<hkern u1="e" u2="&#x2019;" k="20" />
-<hkern u1="e" u2="&#xfd;" k="41" />
-<hkern u1="e" u2="z" k="20" />
-<hkern u1="e" u2="y" k="41" />
-<hkern u1="e" u2="x" k="41" />
-<hkern u1="e" u2="w" k="41" />
-<hkern u1="e" u2="v" k="41" />
-<hkern u1="e" u2="&#x27;" k="20" />
-<hkern u1="e" u2="&#x22;" k="20" />
-<hkern u1="f" u2="&#x201d;" k="-123" />
-<hkern u1="f" u2="&#x2019;" k="-123" />
-<hkern u1="f" u2="&#x27;" k="-123" />
-<hkern u1="f" u2="&#x22;" k="-123" />
-<hkern u1="h" u2="&#x201d;" k="20" />
-<hkern u1="h" u2="&#x2019;" k="20" />
-<hkern u1="h" u2="&#x27;" k="20" />
-<hkern u1="h" u2="&#x22;" k="20" />
-<hkern u1="k" u2="&#x153;" k="41" />
-<hkern u1="k" u2="&#xf8;" k="41" />
-<hkern u1="k" u2="&#xf6;" k="41" />
-<hkern u1="k" u2="&#xf5;" k="41" />
-<hkern u1="k" u2="&#xf4;" k="41" />
-<hkern u1="k" u2="&#xf3;" k="41" />
-<hkern u1="k" u2="&#xf2;" k="41" />
-<hkern u1="k" u2="&#xeb;" k="41" />
-<hkern u1="k" u2="&#xea;" k="41" />
-<hkern u1="k" u2="&#xe9;" k="41" />
-<hkern u1="k" u2="&#xe8;" k="41" />
-<hkern u1="k" u2="&#xe7;" k="41" />
-<hkern u1="k" u2="&#xe0;" k="41" />
-<hkern u1="k" u2="q" k="41" />
-<hkern u1="k" u2="o" k="41" />
-<hkern u1="k" u2="e" k="41" />
-<hkern u1="k" u2="d" k="41" />
-<hkern u1="k" u2="c" k="41" />
-<hkern u1="m" u2="&#x201d;" k="20" />
-<hkern u1="m" u2="&#x2019;" k="20" />
-<hkern u1="m" u2="&#x27;" k="20" />
-<hkern u1="m" u2="&#x22;" k="20" />
-<hkern u1="n" u2="&#x201d;" k="20" />
-<hkern u1="n" u2="&#x2019;" k="20" />
-<hkern u1="n" u2="&#x27;" k="20" />
-<hkern u1="n" u2="&#x22;" k="20" />
-<hkern u1="o" u2="&#x201d;" k="20" />
-<hkern u1="o" u2="&#x2019;" k="20" />
-<hkern u1="o" u2="&#xfd;" k="41" />
-<hkern u1="o" u2="z" k="20" />
-<hkern u1="o" u2="y" k="41" />
-<hkern u1="o" u2="x" k="41" />
-<hkern u1="o" u2="w" k="41" />
-<hkern u1="o" u2="v" k="41" />
-<hkern u1="o" u2="&#x27;" k="20" />
-<hkern u1="o" u2="&#x22;" k="20" />
-<hkern u1="p" u2="&#x201d;" k="20" />
-<hkern u1="p" u2="&#x2019;" k="20" />
-<hkern u1="p" u2="&#xfd;" k="41" />
-<hkern u1="p" u2="z" k="20" />
-<hkern u1="p" u2="y" k="41" />
-<hkern u1="p" u2="x" k="41" />
-<hkern u1="p" u2="w" k="41" />
-<hkern u1="p" u2="v" k="41" />
-<hkern u1="p" u2="&#x27;" k="20" />
-<hkern u1="p" u2="&#x22;" k="20" />
-<hkern u1="r" u2="&#x201d;" k="-82" />
-<hkern u1="r" u2="&#x2019;" k="-82" />
-<hkern u1="r" u2="&#x153;" k="41" />
-<hkern u1="r" u2="&#xf8;" k="41" />
-<hkern u1="r" u2="&#xf6;" k="41" />
-<hkern u1="r" u2="&#xf5;" k="41" />
-<hkern u1="r" u2="&#xf4;" k="41" />
-<hkern u1="r" u2="&#xf3;" k="41" />
-<hkern u1="r" u2="&#xf2;" k="41" />
-<hkern u1="r" u2="&#xeb;" k="41" />
-<hkern u1="r" u2="&#xea;" k="41" />
-<hkern u1="r" u2="&#xe9;" k="41" />
-<hkern u1="r" u2="&#xe8;" k="41" />
-<hkern u1="r" u2="&#xe7;" k="41" />
-<hkern u1="r" u2="&#xe6;" k="41" />
-<hkern u1="r" u2="&#xe5;" k="41" />
-<hkern u1="r" u2="&#xe4;" k="41" />
-<hkern u1="r" u2="&#xe3;" k="41" />
-<hkern u1="r" u2="&#xe2;" k="41" />
-<hkern u1="r" u2="&#xe1;" k="41" />
-<hkern u1="r" u2="&#xe0;" k="41" />
-<hkern u1="r" u2="q" k="41" />
-<hkern u1="r" u2="o" k="41" />
-<hkern u1="r" u2="g" k="20" />
-<hkern u1="r" u2="e" k="41" />
-<hkern u1="r" u2="d" k="41" />
-<hkern u1="r" u2="c" k="41" />
-<hkern u1="r" u2="a" k="41" />
-<hkern u1="r" u2="&#x27;" k="-82" />
-<hkern u1="r" u2="&#x22;" k="-82" />
-<hkern u1="t" u2="&#x201d;" k="-41" />
-<hkern u1="t" u2="&#x2019;" k="-41" />
-<hkern u1="t" u2="&#x27;" k="-41" />
-<hkern u1="t" u2="&#x22;" k="-41" />
-<hkern u1="v" u2="&#x201e;" k="82" />
-<hkern u1="v" u2="&#x201d;" k="-82" />
-<hkern u1="v" u2="&#x201a;" k="82" />
-<hkern u1="v" u2="&#x2019;" k="-82" />
-<hkern u1="v" u2="&#x3f;" k="-41" />
-<hkern u1="v" u2="&#x2e;" k="82" />
-<hkern u1="v" u2="&#x2c;" k="82" />
-<hkern u1="v" u2="&#x27;" k="-82" />
-<hkern u1="v" u2="&#x22;" k="-82" />
-<hkern u1="w" u2="&#x201e;" k="82" />
-<hkern u1="w" u2="&#x201d;" k="-82" />
-<hkern u1="w" u2="&#x201a;" k="82" />
-<hkern u1="w" u2="&#x2019;" k="-82" />
-<hkern u1="w" u2="&#x3f;" k="-41" />
-<hkern u1="w" u2="&#x2e;" k="82" />
-<hkern u1="w" u2="&#x2c;" k="82" />
-<hkern u1="w" u2="&#x27;" k="-82" />
-<hkern u1="w" u2="&#x22;" k="-82" />
-<hkern u1="x" u2="&#x153;" k="41" />
-<hkern u1="x" u2="&#xf8;" k="41" />
-<hkern u1="x" u2="&#xf6;" k="41" />
-<hkern u1="x" u2="&#xf5;" k="41" />
-<hkern u1="x" u2="&#xf4;" k="41" />
-<hkern u1="x" u2="&#xf3;" k="41" />
-<hkern u1="x" u2="&#xf2;" k="41" />
-<hkern u1="x" u2="&#xeb;" k="41" />
-<hkern u1="x" u2="&#xea;" k="41" />
-<hkern u1="x" u2="&#xe9;" k="41" />
-<hkern u1="x" u2="&#xe8;" k="41" />
-<hkern u1="x" u2="&#xe7;" k="41" />
-<hkern u1="x" u2="&#xe0;" k="41" />
-<hkern u1="x" u2="q" k="41" />
-<hkern u1="x" u2="o" k="41" />
-<hkern u1="x" u2="e" k="41" />
-<hkern u1="x" u2="d" k="41" />
-<hkern u1="x" u2="c" k="41" />
-<hkern u1="y" u2="&#x201e;" k="82" />
-<hkern u1="y" u2="&#x201d;" k="-82" />
-<hkern u1="y" u2="&#x201a;" k="82" />
-<hkern u1="y" u2="&#x2019;" k="-82" />
-<hkern u1="y" u2="&#x3f;" k="-41" />
-<hkern u1="y" u2="&#x2e;" k="82" />
-<hkern u1="y" u2="&#x2c;" k="82" />
-<hkern u1="y" u2="&#x27;" k="-82" />
-<hkern u1="y" u2="&#x22;" k="-82" />
-<hkern u1="&#x7b;" u2="J" k="-184" />
-<hkern u1="&#xc0;" u2="&#x201d;" k="143" />
-<hkern u1="&#xc0;" u2="&#x2019;" k="143" />
-<hkern u1="&#xc0;" u2="&#x178;" k="123" />
-<hkern u1="&#xc0;" u2="&#x152;" k="41" />
-<hkern u1="&#xc0;" u2="&#xdd;" k="123" />
-<hkern u1="&#xc0;" u2="&#xd8;" k="41" />
-<hkern u1="&#xc0;" u2="&#xd6;" k="41" />
-<hkern u1="&#xc0;" u2="&#xd5;" k="41" />
-<hkern u1="&#xc0;" u2="&#xd4;" k="41" />
-<hkern u1="&#xc0;" u2="&#xd3;" k="41" />
-<hkern u1="&#xc0;" u2="&#xd2;" k="41" />
-<hkern u1="&#xc0;" u2="&#xc7;" k="41" />
-<hkern u1="&#xc0;" u2="Y" k="123" />
-<hkern u1="&#xc0;" u2="W" k="82" />
-<hkern u1="&#xc0;" u2="V" k="82" />
-<hkern u1="&#xc0;" u2="T" k="143" />
-<hkern u1="&#xc0;" u2="Q" k="41" />
-<hkern u1="&#xc0;" u2="O" k="41" />
-<hkern u1="&#xc0;" u2="J" k="-266" />
-<hkern u1="&#xc0;" u2="G" k="41" />
-<hkern u1="&#xc0;" u2="C" k="41" />
-<hkern u1="&#xc0;" u2="&#x27;" k="143" />
-<hkern u1="&#xc0;" u2="&#x22;" k="143" />
-<hkern u1="&#xc1;" u2="&#x201d;" k="143" />
-<hkern u1="&#xc1;" u2="&#x2019;" k="143" />
-<hkern u1="&#xc1;" u2="&#x178;" k="123" />
-<hkern u1="&#xc1;" u2="&#x152;" k="41" />
-<hkern u1="&#xc1;" u2="&#xdd;" k="123" />
-<hkern u1="&#xc1;" u2="&#xd8;" k="41" />
-<hkern u1="&#xc1;" u2="&#xd6;" k="41" />
-<hkern u1="&#xc1;" u2="&#xd5;" k="41" />
-<hkern u1="&#xc1;" u2="&#xd4;" k="41" />
-<hkern u1="&#xc1;" u2="&#xd3;" k="41" />
-<hkern u1="&#xc1;" u2="&#xd2;" k="41" />
-<hkern u1="&#xc1;" u2="&#xc7;" k="41" />
-<hkern u1="&#xc1;" u2="Y" k="123" />
-<hkern u1="&#xc1;" u2="W" k="82" />
-<hkern u1="&#xc1;" u2="V" k="82" />
-<hkern u1="&#xc1;" u2="T" k="143" />
-<hkern u1="&#xc1;" u2="Q" k="41" />
-<hkern u1="&#xc1;" u2="O" k="41" />
-<hkern u1="&#xc1;" u2="J" k="-266" />
-<hkern u1="&#xc1;" u2="G" k="41" />
-<hkern u1="&#xc1;" u2="C" k="41" />
-<hkern u1="&#xc1;" u2="&#x27;" k="143" />
-<hkern u1="&#xc1;" u2="&#x22;" k="143" />
-<hkern u1="&#xc2;" u2="&#x201d;" k="143" />
-<hkern u1="&#xc2;" u2="&#x2019;" k="143" />
-<hkern u1="&#xc2;" u2="&#x178;" k="123" />
-<hkern u1="&#xc2;" u2="&#x152;" k="41" />
-<hkern u1="&#xc2;" u2="&#xdd;" k="123" />
-<hkern u1="&#xc2;" u2="&#xd8;" k="41" />
-<hkern u1="&#xc2;" u2="&#xd6;" k="41" />
-<hkern u1="&#xc2;" u2="&#xd5;" k="41" />
-<hkern u1="&#xc2;" u2="&#xd4;" k="41" />
-<hkern u1="&#xc2;" u2="&#xd3;" k="41" />
-<hkern u1="&#xc2;" u2="&#xd2;" k="41" />
-<hkern u1="&#xc2;" u2="&#xc7;" k="41" />
-<hkern u1="&#xc2;" u2="Y" k="123" />
-<hkern u1="&#xc2;" u2="W" k="82" />
-<hkern u1="&#xc2;" u2="V" k="82" />
-<hkern u1="&#xc2;" u2="T" k="143" />
-<hkern u1="&#xc2;" u2="Q" k="41" />
-<hkern u1="&#xc2;" u2="O" k="41" />
-<hkern u1="&#xc2;" u2="J" k="-266" />
-<hkern u1="&#xc2;" u2="G" k="41" />
-<hkern u1="&#xc2;" u2="C" k="41" />
-<hkern u1="&#xc2;" u2="&#x27;" k="143" />
-<hkern u1="&#xc2;" u2="&#x22;" k="143" />
-<hkern u1="&#xc3;" u2="&#x201d;" k="143" />
-<hkern u1="&#xc3;" u2="&#x2019;" k="143" />
-<hkern u1="&#xc3;" u2="&#x178;" k="123" />
-<hkern u1="&#

<TRUNCATED>
http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/assets/fonts/opensans-regular-webfont.ttf
----------------------------------------------------------------------
diff --git a/website/assets/fonts/opensans-regular-webfont.ttf b/website/assets/fonts/opensans-regular-webfont.ttf
deleted file mode 100644
index 4621cf1..0000000
Binary files a/website/assets/fonts/opensans-regular-webfont.ttf and /dev/null differ


[34/52] [abbrv] kylin git commit: KYLIN-1416 remove website from coding branch

Posted by li...@apache.org.
http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/assets/fonts/fontawesome-webfont.woff
----------------------------------------------------------------------
diff --git a/website/assets/fonts/fontawesome-webfont.woff b/website/assets/fonts/fontawesome-webfont.woff
deleted file mode 100644
index 628b6a5..0000000
Binary files a/website/assets/fonts/fontawesome-webfont.woff and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/assets/fonts/helveticaneue-webfont.css
----------------------------------------------------------------------
diff --git a/website/assets/fonts/helveticaneue-webfont.css b/website/assets/fonts/helveticaneue-webfont.css
deleted file mode 100644
index 6a42ad4..0000000
--- a/website/assets/fonts/helveticaneue-webfont.css
+++ /dev/null
@@ -1,8 +0,0 @@
-@font-face {
-    font-family: 'HelveticaNeueRegular';
-    src: url('helveticaneue-webfont.eot');
-    src: url('helveticaneue-webfont.eot') format('embedded-opentype'),
-         url('helveticaneue-webfont.woff') format('woff'),
-         url('helveticaneue-webfont.ttf') format('truetype'),
-         url('helveticaneue-webfont.svg#HelveticaNeueRegular') format('svg');
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/assets/fonts/helveticaneue-webfont.eot
----------------------------------------------------------------------
diff --git a/website/assets/fonts/helveticaneue-webfont.eot b/website/assets/fonts/helveticaneue-webfont.eot
deleted file mode 100644
index 9eb6c64..0000000
Binary files a/website/assets/fonts/helveticaneue-webfont.eot and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/assets/fonts/helveticaneue-webfont.svg
----------------------------------------------------------------------
diff --git a/website/assets/fonts/helveticaneue-webfont.svg b/website/assets/fonts/helveticaneue-webfont.svg
deleted file mode 100644
index d6476d6..0000000
--- a/website/assets/fonts/helveticaneue-webfont.svg
+++ /dev/null
@@ -1,693 +0,0 @@
-<?xml version="1.0" standalone="no"?>
-<!DOCTYPE svg PUBLIC "-//W3C//DTD SVG 1.1//EN" "http://www.w3.org/Graphics/SVG/1.1/DTD/svg11.dtd" >
-<svg>
-<metadata>
-Created by FontForge 20110222 at Thu Mar  6 12:16:42 2014
- By Orthosie Webhosting
-Copyright (c) 1981, 1982, 1983, 1989 and 1993, Linotype Library GmbH or its affiliated Linotype-Hell companies. All rights reserved.
-
-The digitally encoded machine readable software for producing the Typefaces licensed to you is now the property of Heidelberger Druckmaschinen AG and its licensors, and may not be reproduced, used, displayed, modified, disclosed or transferred without the express written approval of Heidelberger Druckmaschinen AG.
-
-Copyright (c) 1988, 1990, 1993 Adobe Systems Incorporated. All Rights Reserved.
-</metadata>
-<defs>
-<font id="HelveticaNeueRegular" horiz-adv-x="556" >
-  <font-face 
-    font-family="HelveticaNeueRegular"
-    font-weight="400"
-    font-stretch="normal"
-    units-per-em="1000"
-    panose-1="2 0 5 3 0 0 0 2 0 4"
-    ascent="800"
-    descent="-200"
-    x-height="517"
-    cap-height="714"
-    bbox="-58 -213 1034 952"
-    underline-thickness="50"
-    underline-position="-75"
-    unicode-range="U+0020-2122"
-  />
-<missing-glyph horiz-adv-x="500" 
-d="M442 8h-384v496h384v-496zM50 512v-512h400v512h-400z" />
-    <glyph glyph-name=".notdef" horiz-adv-x="500" 
-d="M442 8h-384v496h384v-496zM50 512v-512h400v512h-400z" />
-    <glyph glyph-name=".null" horiz-adv-x="0" 
- />
-    <glyph glyph-name="nonmarkingreturn" horiz-adv-x="333" 
- />
-    <glyph glyph-name="space" unicode=" " horiz-adv-x="278" 
- />
-    <glyph glyph-name="exclam" unicode="!" horiz-adv-x="259" 
-d="M74 111v-111h111v111h-111zM177 494v220h-95v-220l25 -311h45z" />
-    <glyph glyph-name="quotedbl" unicode="&#x22;" horiz-adv-x="426" 
-d="M249 714v-258h68v258h-68zM109 714v-258h68v258h-68z" />
-    <glyph glyph-name="numbersign" unicode="#" 
-d="M231 424h124l-21 -150h-124zM131 206l-29 -206h69l29 206h124l-29 -206h69l29 206h101v68h-91l21 150h98v68h-89l29 205h-69l-29 -205h-124l29 205h-69l-29 -205h-111v-68h102l-21 -150h-107v-68h97z" />
-    <glyph glyph-name="dollar" unicode="$" 
-d="M110 218h-85q8 -105 67 -164.5t159 -69.5v-82h55v82q48 4 88 20t69 43.5t45.5 65.5t16.5 87q0 46 -17.5 79t-42.5 55t-52 34.5t-47 17.5l-60 16v246q92 -17 112 -116h85q-20 88 -68 138t-129 60v67h-55v-66q-43 -3 -80.5 -19t-65 -43.5t-43.5 -65t-16 -82.5q0 -44 12 -75
-t37.5 -52.5t64 -37t91.5 -28.5v-265q-60 10 -99.5 49t-41.5 106zM306 63v251q27 -8 51 -17.5t41 -23.5t27 -33.5t10 -48.5q0 -32 -9.5 -55.5t-27 -39t-41 -23.5t-51.5 -10zM251 651v-234q-25 7 -46 16t-36.5 21.5t-24 31t-8.5 44.5q0 57 33.5 86.5t81.5 34.5z" />
-    <glyph glyph-name="percent" unicode="%" horiz-adv-x="1000" 
-d="M167 508q0 23 3 48.5t12.5 47t26.5 36t45 14.5q29 0 45.5 -14.5t25.5 -36t11.5 -47t2.5 -47.5q0 -23 -3 -49t-12.5 -47.5t-26.5 -35.5t-46 -14q-28 0 -44.5 14.5t-25.5 36t-11.5 47t-2.5 47.5zM92 506q0 -41 8.5 -77t27.5 -63t50 -42.5t76 -15.5q44 0 74 17t49.5 44.5
-t28 64t8.5 75.5q0 41 -7.5 77.5t-26 63.5t-49 43t-77.5 16q-46 0 -77 -16.5t-50 -44.5t-27 -64.5t-8 -77.5zM661 188q0 23 3 48.5t12.5 47t26.5 36t45 14.5q29 0 45.5 -14.5t25.5 -36t11.5 -47t2.5 -47.5q0 -23 -3 -49t-12.5 -47.5t-26.5 -35.5t-46 -14q-28 0 -44.5 14.5
-t-25.5 36t-11.5 47t-2.5 47.5zM586 186q0 -41 8.5 -77t27.5 -62.5t50 -42t76 -15.5q44 0 74 16.5t49.5 44t28 64t8.5 75.5q0 41 -7.5 77.5t-26 63.5t-49 43t-77.5 16q-46 0 -77 -16.5t-50 -44.5t-27 -64.5t-8 -77.5zM726 722l-538 -743h86l537 743h-85z" />
-    <glyph glyph-name="ampersand" unicode="&#x26;" horiz-adv-x="630" 
-d="M275 439l-24 30q-12 15 -22 30.5t-16.5 31.5t-6.5 32q0 39 26 57.5t56 18.5q38 0 60 -22.5t22 -53.5q0 -23 -9 -41.5t-23 -33.5t-31 -27zM458 89l76 -89h111l-139 161l19 43q8 18 13.5 37t9 41.5t6.5 54.5h-81q-3 -55 -23 -111l-129 157q26 15 50 33.5t43 41.5t30 50.5
-t11 59.5q0 36 -14.5 63.5t-38.5 46t-55.5 27.5t-64.5 9q-39 0 -69 -12t-50.5 -32.5t-31 -47t-10.5 -55.5q0 -27 6.5 -48.5t18 -41t25.5 -38.5l29 -40q-34 -18 -64.5 -38.5t-53 -46t-36 -57.5t-13.5 -74q0 -22 7.5 -55t31.5 -64t68 -53t116 -22q59 0 114.5 25t87.5 75z
-M247 345l164 -197q-26 -39 -64.5 -62t-87.5 -23q-26 0 -50.5 9t-43.5 24.5t-30.5 38t-11.5 49.5q0 32 10 55.5t27 42.5t39.5 34t47.5 29z" />
-    <glyph glyph-name="quotesingle" unicode="'" horiz-adv-x="278" 
-d="M105 714v-258h68v258h-68z" />
-    <glyph glyph-name="parenleft" unicode="(" horiz-adv-x="259" 
-d="M204 -197h65q-71 115 -101.5 228t-30.5 239q0 123 30 235t102 226h-65q-78 -103 -117.5 -223t-39.5 -238q0 -66 11 -126t31.5 -117t49.5 -112.5t65 -111.5z" />
-    <glyph glyph-name="parenright" unicode=")" horiz-adv-x="259" 
-d="M-9 -197h64q78 103 117.5 222.5t39.5 237.5q0 66 -11 126t-31.5 117.5t-49.5 113t-65 111.5h-64q70 -115 100.5 -228.5t30.5 -239.5q0 -122 -30 -234.5t-101 -225.5z" />
-    <glyph glyph-name="asterisk" unicode="*" horiz-adv-x="352" 
-d="M201 601v113h-53v-113l-106 40l-18 -48l106 -38l-65 -91l45 -31l65 96l69 -96l41 31l-66 91l109 38l-20 48z" />
-    <glyph glyph-name="plus" unicode="+" horiz-adv-x="600" 
-d="M334 287v219h-68v-219h-218v-68h218v-219h68v219h218v68h-218z" />
-    <glyph glyph-name="comma" unicode="," horiz-adv-x="278" 
-d="M83 111v-111h59q1 -12 -1.5 -26t-9.5 -27.5t-18.5 -25t-28.5 -17.5v-50q58 17 84 58t26 98v101h-111z" />
-    <glyph glyph-name="hyphen" unicode="-" horiz-adv-x="389" 
-d="M50 318v-80h289v80h-289z" />
-    <glyph glyph-name="period" unicode="." horiz-adv-x="278" 
-d="M83 111v-111h111v111h-111z" />
-    <glyph glyph-name="slash" unicode="/" horiz-adv-x="333" 
-d="M278 731l-295 -747h72l296 747h-73z" />
-    <glyph glyph-name="zero" unicode="0" 
-d="M132 349q0 26 0.5 57.5t5 63t13 61.5t25 52.5t41.5 36.5t61 14t61 -14t41.5 -36.5t25 -52.5t13 -61.5t5 -63t0.5 -57.5q0 -40 -2.5 -89.5t-16 -93t-43.5 -73.5t-84 -30t-84 30t-43.5 73.5t-16 93t-2.5 89.5zM42 348q0 -39 2 -81t10 -82t23 -75.5t41 -62.5t65.5 -42.5
-t94.5 -15.5q56 0 95 15.5t65 42.5t41 62.5t23 75.5t10 82t2 81t-2 81t-10 82t-23 76t-41 63t-65.5 43t-94.5 16t-94.5 -16t-65.5 -43t-41 -63t-23 -76t-10 -82t-2 -81z" />
-    <glyph glyph-name="one" unicode="1" 
-d="M356 0v709h-65q-7 -40 -26 -66t-46.5 -41t-61.5 -20.5t-70 -5.5v-68h184v-508h85z" />
-    <glyph glyph-name="two" unicode="2" 
-d="M44 458h85v8q0 28 7 56q7 31 24 56t43.5 40.5t63.5 15.5q28 0 53 -9t43.5 -26t29.5 -40.5t11 -52.5q0 -37 -11.5 -65t-34 -52t-56.5 -47.5t-79 -51.5q-37 -22 -71 -47t-61 -58t-44.5 -77.5t-22.5 -107.5h463v75h-364q6 33 25.5 58.5t47 47.5t60.5 41l66 40q33 21 64 44
-t55 51.5t38.5 64.5t14.5 83q0 50 -17.5 88t-47.5 63.5t-70.5 39t-86.5 13.5q-56 0 -100 -19t-73.5 -52.5t-43.5 -79.5q-12 -38 -12 -81q0 -9 1 -19z" />
-    <glyph glyph-name="three" unicode="3" 
-d="M218 405v-72q24 3 51 3q32 0 59.5 -8.5t47.5 -26t32 -43t12 -58.5q0 -32 -12.5 -57.5t-33.5 -43t-49 -27t-59 -9.5q-73 0 -111 43.5t-40 112.5h-85v-6q0 -51 16 -92q16 -43 47 -72.5t75 -44.5t98 -15q50 0 94.5 13.5t77.5 40.5t52.5 67.5t19.5 93.5q0 64 -31.5 111
-t-96.5 61v2q42 19 70 56t28 85q0 49 -16.5 85t-45.5 59t-68.5 34.5t-85.5 11.5q-53 0 -93.5 -17t-67.5 -47t-41.5 -72t-16.5 -93h85q0 31 8 59t24.5 49t42 33.5t59.5 12.5q54 0 90 -28.5t36 -85.5q0 -28 -11 -50t-29.5 -36.5t-43 -22t-51.5 -7.5h-18h-10q-4 0 -9 1z" />
-    <glyph glyph-name="four" unicode="4" 
-d="M339 241h-238l236 347h2v-347zM419 241v468h-68l-323 -461v-82h311v-166h80v166h96v75h-96z" />
-    <glyph glyph-name="five" unicode="5" 
-d="M470 622v75h-350l-66 -368l73 -4q25 30 57.5 48.5t74.5 18.5q36 0 65.5 -12t50.5 -33.5t32.5 -51t11.5 -64.5q0 -42 -12 -73.5t-32.5 -52.5t-48 -31.5t-57.5 -10.5q-32 0 -58.5 9.5t-46 27t-31 41t-13.5 50.5h-85q1 -48 19 -86t49 -63.5t71.5 -39t86.5 -13.5
-q62 0 108.5 19.5t77.5 52.5t46.5 75.5t15.5 87.5q0 61 -18 106.5t-49 76t-73 45.5t-89 15q-36 0 -72.5 -12.5t-59.5 -38.5l-2 2l38 204h286z" />
-    <glyph glyph-name="six" unicode="6" 
-d="M414 528h85q-8 88 -60 134.5t-140 46.5q-76 0 -126 -32t-80 -85t-42.5 -121.5t-12.5 -141.5q0 -56 8.5 -116t34.5 -110t74 -82t127 -32q67 0 112.5 22.5t73 57t39.5 76.5t12 81q0 49 -15 91t-42.5 73t-68 48.5t-91.5 17.5q-58 0 -102.5 -22t-74.5 -71l-2 2q1 41 8 88
-t25.5 87.5t51.5 67t87 26.5q51 0 82 -29t37 -77zM286 381q36 0 63 -13t45 -34.5t26.5 -51t8.5 -62.5q0 -31 -9.5 -59.5t-27.5 -50.5t-44.5 -34.5t-61.5 -12.5t-62.5 12.5t-46 33.5t-28.5 50t-10 62t9.5 62.5t28.5 51t46 34t63 12.5z" />
-    <glyph glyph-name="seven" unicode="7" 
-d="M509 624v73h-459v-80h372q-56 -59 -104.5 -129t-85.5 -148.5t-60.5 -164t-29.5 -175.5h95q5 83 28 171t59.5 171t83.5 156t101 126z" />
-    <glyph glyph-name="eight" unicode="8" 
-d="M155 525q0 27 10 47t26.5 34t39.5 21t48 7q52 0 85.5 -27.5t33.5 -81.5t-33 -84t-84 -30q-26 0 -49 7t-40 21t-27 35.5t-10 50.5zM65 528q0 -48 27 -88.5t72 -58.5q-60 -21 -92 -67.5t-32 -109.5q0 -54 18.5 -94.5t51 -67.5t76.5 -40t95 -13q49 0 92 14t74.5 41.5
-t50 67.5t18.5 92q0 66 -31 111.5t-95 65.5q45 20 71.5 59.5t26.5 87.5q0 34 -12 66.5t-37.5 58t-66.5 41t-99 15.5q-41 0 -78.5 -12t-66.5 -35t-46 -56.5t-17 -77.5zM130 200q0 31 11.5 56t32 43t48 27.5t58.5 9.5q30 0 56.5 -10.5t46.5 -28.5t31.5 -42.5t11.5 -53.5
-q0 -30 -10.5 -55t-30 -43.5t-46 -29t-58.5 -10.5q-66 0 -108.5 36.5t-42.5 100.5z" />
-    <glyph glyph-name="nine" unicode="9" 
-d="M134 171h-85q8 -92 67 -137t148 -45q129 0 187.5 98.5t58.5 282.5q0 101 -19.5 166.5t-52.5 103.5t-77 53.5t-94 15.5q-51 0 -94 -17t-74 -47.5t-48 -73t-17 -93.5q0 -52 14.5 -96t43 -75t70.5 -48.5t96 -17.5q52 0 96 26.5t68 71.5l2 -2q-4 -139 -43 -206.5t-117 -67.5
-q-51 0 -88 28t-42 80zM409 472q0 -31 -10 -59.5t-29 -50t-46 -34t-60 -12.5q-31 0 -56.5 12.5t-44 33.5t-29 48t-10.5 56q0 33 7.5 63t24 53.5t43 37.5t64.5 14q36 0 63 -13t45.5 -35.5t28 -51.5t9.5 -62z" />
-    <glyph glyph-name="colon" unicode=":" horiz-adv-x="278" 
-d="M194 406v111h-111v-111h111zM83 111v-111h111v111h-111z" />
-    <glyph glyph-name="semicolon" unicode=";" horiz-adv-x="278" 
-d="M194 406v111h-111v-111h111zM83 111v-111h59q1 -12 -1.5 -26t-9.5 -27.5t-18.5 -25t-28.5 -17.5v-50q58 17 84.5 58t25.5 98v101h-111z" />
-    <glyph glyph-name="less" unicode="&#x3c;" horiz-adv-x="600" 
-d="M554 446v68l-508 -230v-62l508 -229v67l-425 193z" />
-    <glyph glyph-name="equal" unicode="=" horiz-adv-x="600" 
-d="M48 186v-68h504v68h-504zM552 320v68h-504v-68h504z" />
-    <glyph glyph-name="greater" unicode="&#x3e;" horiz-adv-x="600" 
-d="M46 60v-67l508 229v62l-508 230v-68l425 -193z" />
-    <glyph glyph-name="question" unicode="?" 
-d="M218 111v-111h111v111h-111zM55 491h85q0 35 8 65t25.5 52t43.5 35t62 13q54 0 85.5 -31t35.5 -86v-11q0 -30 -9 -52q-11 -26 -29.5 -47.5t-40 -41t-41 -42.5t-33 -52.5t-15.5 -71.5v-47h85v39q0 26 7.5 46.5t20 38t27.5 32.5l32 31l32 33q15 17 27 37t19.5 44.5
-t7.5 55.5q0 48 -15.5 85t-43.5 62.5t-66.5 39t-85.5 13.5q-52 0 -94 -18t-71 -50.5t-44 -76.5q-15 -41 -15 -89v-6z" />
-    <glyph glyph-name="at" unicode="@" horiz-adv-x="800" 
-d="M534 270l84 284h-63l-23 -63q-17 45 -47 63.5t-65 18.5q-53 0 -96.5 -25t-74 -65t-47.5 -91t-17 -103q0 -33 10.5 -62t29 -50t45 -33.5t58.5 -12.5q37 0 69 20.5t55 46.5h3q2 -33 20 -49t44 -16q35 0 76 21t76 59t58.5 91.5t23.5 118.5q0 68 -27 124.5t-73.5 97
-t-108.5 63.5t-131 23q-76 0 -143.5 -29t-118 -79.5t-80 -118t-29.5 -144.5q0 -80 29 -148.5t79.5 -119t119 -79.5t147.5 -29q48 0 94 11.5t86.5 34t74.5 55t58 74.5h-63q-43 -55 -109.5 -88t-138.5 -33q-63 0 -119 24.5t-98.5 67t-67 100.5t-24.5 125q0 66 23.5 124
-t64.5 102t96.5 69.5t119.5 25.5q53 0 101.5 -18t86.5 -51.5t60.5 -81.5t22.5 -107q0 -40 -13 -81t-33 -74t-44.5 -54t-47.5 -21q-24 0 -24 31q0 11 3.5 24.5t7.5 26.5zM420 505q16 0 30.5 -8t26 -20.5t18 -29.5t6.5 -35q0 -31 -12 -68t-33 -69.5t-49 -54t-61 -21.5
-q-38 0 -62 27.5t-24 69.5q0 33 12 69.5t33 67.5t50.5 51.5t64.5 20.5z" />
-    <glyph glyph-name="A" unicode="A" horiz-adv-x="648" 
-d="M201 295l121 333h2l119 -333h-242zM272 714l-278 -714h97l80 215h302l78 -215h105l-279 714h-105z" />
-    <glyph glyph-name="B" unicode="B" horiz-adv-x="685" 
-d="M173 407v227h190q31 0 58.5 -3.5t48.5 -15.5t33 -34t12 -60q0 -57 -35.5 -85.5t-116.5 -28.5h-190zM78 714v-714h345q53 0 93.5 17.5t68 46t41.5 65.5t14 76q0 66 -35 112t-101 60v2q51 20 78.5 61.5t27.5 98.5q0 54 -24.5 91t-59.5 58q-16 10 -38.5 15t-48.5 7.5
-t-52.5 3t-49.5 0.5h-259zM173 80v247h217q30 0 58 -5.5t49.5 -19t34.5 -36.5t13 -58q0 -60 -35.5 -94t-98.5 -34h-238z" />
-    <glyph glyph-name="C" unicode="C" horiz-adv-x="722" 
-d="M580 499h95q-8 57 -34 100.5t-65 72.5t-89 44t-106 15q-82 0 -145.5 -29.5t-106 -80.5t-64.5 -119.5t-22 -146.5t20.5 -146t61.5 -118t103 -78.5t145 -28.5q137 0 216 75t93 210h-95q-3 -44 -18 -82t-41.5 -65.5t-63 -43t-83.5 -15.5q-64 0 -110 24t-75.5 64.5t-43.5 95
-t-14 115.5q0 56 14 108t43.5 92.5t75 64.5t109.5 24q75 0 129.5 -38t70.5 -114z" />
-    <glyph glyph-name="D" unicode="D" horiz-adv-x="704" 
-d="M173 80v554h160q66 0 111 -18.5t73.5 -54t41 -86t12.5 -114.5q0 -66 -13.5 -112.5t-34.5 -77.5t-47.5 -49t-53.5 -27.5t-51 -12t-40 -2.5h-158zM78 714v-714h245q89 0 154 25t107 72.5t62 117t20 159.5q0 172 -89 256t-254 84h-245z" />
-    <glyph glyph-name="E" unicode="E" horiz-adv-x="611" 
-d="M78 714v-714h496v80h-401v247h371v80h-371v227h398v80h-493z" />
-    <glyph glyph-name="F" unicode="F" horiz-adv-x="574" 
-d="M78 714v-714h95v327h328v80h-328v227h374v80h-469z" />
-    <glyph glyph-name="G" unicode="G" horiz-adv-x="759" 
-d="M612 89l25 -89h60v376h-313v-80h228v-11q0 -43 -13 -81q-15 -42 -44.5 -73.5t-72.5 -49.5t-99 -18q-60 0 -105.5 23.5t-76.5 63t-47 91t-16 107.5q0 57 14 111.5t43.5 97t76 68.5t111.5 26q40 0 75.5 -9.5t63.5 -28.5t47 -48.5t26 -70.5h95q-10 63 -37.5 107.5t-68 73.5
-t-92 42.5t-109.5 13.5q-85 0 -148.5 -31.5t-106 -84.5t-64 -123.5t-21.5 -148.5q0 -70 23 -135t67 -115t107 -79.5t143 -29.5q63 0 125 24.5t104 80.5z" />
-    <glyph glyph-name="H" unicode="H" horiz-adv-x="722" 
-d="M78 714v-714h95v327h376v-327h95v714h-95v-307h-376v307h-95z" />
-    <glyph glyph-name="I" unicode="I" horiz-adv-x="259" 
-d="M82 714v-714h95v714h-95z" />
-    <glyph glyph-name="J" unicode="J" horiz-adv-x="519" 
-d="M441 178v536h-95v-522q0 -59 -24.5 -94t-89.5 -35q-35 0 -57.5 10.5t-35 29.5t-17.5 45t-5 57v30h-95v-44q0 -102 54 -154.5t154 -52.5q60 0 100.5 16.5t65 44t35 62.5t10.5 71z" />
-    <glyph glyph-name="K" unicode="K" horiz-adv-x="667" 
-d="M78 714v-714h95v250l119 111l258 -361h120l-313 426l301 288h-123l-362 -356v356h-95z" />
-    <glyph glyph-name="L" unicode="L" 
-d="M78 714v-714h473v80h-378v634h-95z" />
-    <glyph glyph-name="M" unicode="M" horiz-adv-x="871" 
-d="M80 714v-714h90v594h2l223 -594h81l223 594h2v-594h90v714h-130l-226 -600l-225 600h-130z" />
-    <glyph glyph-name="N" unicode="N" horiz-adv-x="722" 
-d="M77 714v-714h90v573h2l373 -573h104v714h-90v-579h-2l-376 579h-101z" />
-    <glyph glyph-name="O" unicode="O" horiz-adv-x="760" 
-d="M133 357q0 54 14 106.5t44 94.5t77 67.5t112 25.5t112 -25.5t77 -67.5t44 -94.5t14 -106.5t-14 -106.5t-44 -94.5t-77 -67.5t-112 -25.5t-112 25.5t-77 67.5t-44 94.5t-14 106.5zM38 357q0 -73 21.5 -140.5t64.5 -119.5t107 -82.5t149 -30.5t149 30.5t107 82.5
-t64.5 119.5t21.5 140.5t-21.5 140.5t-64.5 119.5t-107 83t-149 31t-149 -31t-107 -83t-64.5 -119.5t-21.5 -140.5z" />
-    <glyph glyph-name="P" unicode="P" horiz-adv-x="648" 
-d="M173 372v262h186q81 0 118.5 -33.5t37.5 -97.5t-37 -98q-37 -33 -114 -33h-5h-186zM78 714v-714h95v292h218h4q105 0 159 55q56 56 56 156t-55.5 155.5t-163.5 55.5h-313z" />
-    <glyph glyph-name="Q" unicode="Q" horiz-adv-x="760" 
-d="M450 204l-50 -58l79 -63q-48 -20 -99 -20q-65 0 -112 25.5t-77 67.5t-44 94.5t-14 106.5t14 106.5t44 94.5t77 67.5t112 25.5t112 -25.5t77 -67.5t44 -94.5t14 -106.5q0 -66 -20 -126.5t-62 -102.5zM660 -61l48 60l-94 74q54 54 81 128t27 156q0 73 -21.5 140.5
-t-64.5 119.5t-107 83t-149 31t-149 -31t-107 -83t-64.5 -119.5t-21.5 -140.5t21.5 -140.5t64.5 -119.5t107 -82.5t149 -30.5q99 0 171 42z" />
-    <glyph glyph-name="R" unicode="R" horiz-adv-x="685" 
-d="M78 714v-714h95v305h232q35 0 56 -10.5t34 -28t19 -41t10 -49.5q5 -26 6 -53t2 -50.5t4.5 -42.5t13.5 -30h106q-15 18 -22.5 41.5t-11.5 49.5t-5 53t-3 53q-3 26 -8.5 50t-17 43.5t-31 33.5t-50.5 20v2q65 18 94.5 67t29.5 114q0 87 -57.5 137t-159.5 50h-336zM371 385
-h-198v249h236q67 0 97 -34t30 -88q0 -39 -13.5 -63.5t-36 -39t-52.5 -19.5t-63 -5z" />
-    <glyph glyph-name="S" unicode="S" horiz-adv-x="648" 
-d="M491 505h90q-2 59 -22.5 101.5t-56 70.5t-82.5 41t-102 13q-49 0 -95.5 -12.5t-83 -38t-58.5 -65t-22 -93.5q0 -49 19.5 -81.5t52 -53t73.5 -33t84 -22.5l83 -18q41 -9 73.5 -23.5t52 -37.5t19.5 -60q0 -39 -16 -64t-42 -39.5t-58.5 -20.5t-64.5 -6q-40 0 -78 10
-t-66.5 31t-46 53.5t-17.5 77.5h-90q0 -65 23.5 -112.5t64 -78t94 -45.5t113.5 -15q49 0 98.5 11.5t89.5 37t65.5 66t25.5 97.5q0 53 -19.5 88t-52 58t-73.5 36.5t-83.5 23.5t-83.5 18.5t-73.5 21.5t-52 33.5t-19.5 53.5q0 35 13.5 58.5t36 37.5t51.5 20t59 6
-q74 0 121.5 -34.5t55.5 -111.5z" />
-    <glyph glyph-name="T" unicode="T" horiz-adv-x="574" 
-d="M240 634v-634h95v634h238v80h-571v-80h238z" />
-    <glyph glyph-name="U" unicode="U" horiz-adv-x="722" 
-d="M648 258v456h-95v-456q0 -96 -47.5 -145.5t-138.5 -49.5q-96 0 -147 49.5t-51 145.5v456h-95v-456q0 -143 77.5 -208.5t215.5 -65.5q135 0 208 68.5t73 205.5z" />
-    <glyph glyph-name="V" unicode="V" horiz-adv-x="611" 
-d="M359 0l254 714h-99l-206 -619h-2l-204 619h-102l251 -714h108z" />
-    <glyph glyph-name="W" unicode="W" horiz-adv-x="926" 
-d="M723 0l191 714h-95l-145 -594h-2l-158 594h-103l-160 -594h-2l-140 594h-97l183 -714h99l165 600h2l163 -600h99z" />
-    <glyph glyph-name="X" unicode="X" horiz-adv-x="611" 
-d="M248 367l-248 -367h106l198 295l192 -295h116l-251 367l236 347h-106l-186 -279l-178 279h-114z" />
-    <glyph glyph-name="Y" unicode="Y" horiz-adv-x="648" 
-d="M373 292l273 422h-108l-209 -340l-214 340h-113l276 -422v-292h95v292z" />
-    <glyph glyph-name="Z" unicode="Z" horiz-adv-x="611" 
-d="M22 80v-80h568v80h-455l443 554v80h-522v-80h410z" />
-    <glyph glyph-name="bracketleft" unicode="[" horiz-adv-x="259" 
-d="M259 663v68h-187v-928h187v68h-107v792h107z" />
-    <glyph glyph-name="backslash" unicode="\" horiz-adv-x="333" 
-d="M351 -16l-296 747h-72l295 -747h73z" />
-    <glyph glyph-name="bracketright" unicode="]" horiz-adv-x="259" 
-d="M0 -129v-68h187v928h-187v-68h107v-792h-107z" />
-    <glyph glyph-name="asciicircum" unicode="^" horiz-adv-x="600" 
-d="M268 697l-218 -433h71l178 360l181 -360h70l-218 433h-64z" />
-    <glyph glyph-name="underscore" unicode="_" horiz-adv-x="500" 
-d="M0 -74v-50h500v50h-500z" />
-    <glyph glyph-name="grave" unicode="`" horiz-adv-x="222" 
-d="M-22 731l130 -143h64l-86 143h-108z" />
-    <glyph glyph-name="a" unicode="a" horiz-adv-x="537" 
-d="M522 2v66q-15 -5 -26 -5q-20 0 -23.5 14t-3.5 44v266q0 45 -17 73t-45 43.5t-62.5 20.5t-69.5 5q-43 0 -82 -9t-69 -29.5t-48 -53.5t-20 -81h85q3 57 39 77.5t90 20.5q20 0 40.5 -3t37 -12t27 -26t10.5 -45q0 -24 -14 -36.5t-38 -19t-55.5 -10t-66.5 -10.5
-q-33 -6 -64.5 -15t-56 -26.5t-39.5 -45.5t-15 -72q0 -39 13.5 -66.5t37 -45t54.5 -25t66 -7.5q54 0 100.5 18.5t81.5 60.5q0 -42 19.5 -60.5t52.5 -18.5q39 0 61 13zM384 267v-87q0 -18 -9 -38.5t-28 -38t-49 -29t-72 -11.5q-19 0 -37.5 4t-32 13t-22 23.5t-8.5 35.5
-q0 32 14.5 50.5t37.5 28.5t52 14.5t57.5 8t54.5 9t42 17.5z" />
-    <glyph glyph-name="b" unicode="b" horiz-adv-x="593" 
-d="M67 714v-714h85v69h2q14 -23 34.5 -38.5t43.5 -24.5t46.5 -13t42.5 -4q59 0 103.5 21t74 57.5t44 85.5t14.5 104t-15 104t-44.5 86.5t-74 59.5t-104.5 22q-54 0 -99 -19.5t-66 -62.5h-2v267h-85zM467 263q0 -38 -8 -74t-26 -64t-47.5 -45t-73.5 -17t-75 16.5t-50.5 43.5
-t-28.5 62.5t-9 73.5q0 36 8.5 71t27.5 62.5t49 44.5t73 17q41 0 71.5 -16t50 -43t29 -61.5t9.5 -70.5z" />
-    <glyph glyph-name="c" unicode="c" horiz-adv-x="537" 
-d="M415 351h88q-5 46 -24 79.5t-48.5 55.5t-68 32.5t-82.5 10.5q-61 0 -107 -21.5t-76.5 -59t-45.5 -88t-15 -108.5t15.5 -106.5t46 -83.5t76 -54t104.5 -19q99 0 156.5 52t71.5 148h-87q-8 -60 -43.5 -93t-98.5 -33q-40 0 -69 16t-47 42.5t-26.5 60.5t-8.5 70q0 39 8 75.5
-t26.5 64.5t49.5 45t77 17q54 0 86 -27t42 -76z" />
-    <glyph glyph-name="d" unicode="d" horiz-adv-x="593" 
-d="M126 254q0 38 8 74t26 64t48 45t73 17q44 0 75 -16.5t50.5 -43.5t28.5 -62.5t9 -73.5q0 -36 -8.5 -71t-27.5 -62.5t-49 -44.5t-73 -17q-41 0 -71.5 16t-50 43t-29 61.5t-9.5 70.5zM526 0v714h-85v-266h-2q-14 23 -34.5 38.5t-43.5 25t-46 13.5t-43 4q-59 0 -103.5 -21.5
-t-74 -58t-44 -85.5t-14.5 -104t15 -104t44.5 -86t74 -59t104.5 -22q54 0 99 19t66 62h2v-70h85z" />
-    <glyph glyph-name="e" unicode="e" horiz-adv-x="537" 
-d="M423 308h-297q2 30 13 56.5t30 46.5t45.5 31.5t59.5 11.5q32 0 58.5 -11.5t46 -31t31 -46.5t13.5 -57zM505 164h-84q-11 -51 -45.5 -76t-88.5 -25q-42 0 -73 14t-51 37.5t-29 53.5q-8 28 -8 58v7h387v15q0 40 -9 84q-10 52 -37.5 96t-73 72.5t-114.5 28.5
-q-53 0 -97.5 -20t-77 -56t-50.5 -85t-18 -108q2 -59 17.5 -109t46.5 -86t76.5 -56t107.5 -20q88 0 146 44t75 131z" />
-    <glyph glyph-name="f" unicode="f" horiz-adv-x="296" 
-d="M99 442v-442h85v442h100v75h-100v72q0 34 17 46t48 12q11 0 24 -1.5t24 -5.5v74q-12 4 -27.5 6t-27.5 2q-70 0 -106.5 -32.5t-36.5 -95.5v-77h-87v-75h87z" />
-    <glyph glyph-name="g" unicode="g" horiz-adv-x="574" 
-d="M510 44v473h-80v-74h-1q-23 43 -64 64.5t-90 21.5q-67 0 -113 -25.5t-74 -65t-40 -88t-12 -94.5q0 -53 14.5 -100.5t43.5 -83t72 -56.5t101 -21q50 0 94.5 22.5t66.5 70.5h2v-34q0 -43 -8.5 -79t-27.5 -61.5t-48 -40t-71 -14.5q-21 0 -44 4.5t-42.5 14.5t-33 26t-14.5 39
-h-85q2 -42 22 -71t50.5 -47t68 -26t73.5 -8q124 0 182 63t58 190zM272 69q-42 0 -70 17.5t-45 46t-24 63.5t-7 70q0 37 8.5 71t27 60t47.5 41.5t70 15.5q40 0 68 -16t45.5 -42.5t25.5 -59.5t8 -67q0 -36 -8.5 -72t-26.5 -64.5t-47.5 -46t-71.5 -17.5z" />
-    <glyph glyph-name="h" unicode="h" 
-d="M64 714v-714h85v292q0 35 9.5 64.5t28.5 51.5t47.5 34t67.5 12q49 0 77 -28t28 -76v-350h85v340q0 42 -8.5 76.5t-29.5 59.5t-55 39t-85 14q-23 0 -47.5 -5t-47 -15.5t-40.5 -27t-28 -40.5h-2v273h-85z" />
-    <glyph glyph-name="i" unicode="i" horiz-adv-x="222" 
-d="M154 610v104h-85v-104h85zM69 517v-517h85v517h-85z" />
-    <glyph glyph-name="j" unicode="j" horiz-adv-x="222" 
-d="M154 610v104h-85v-104h85zM154 -65v582h-85v-568q0 -40 -10 -55.5t-38 -15.5h-18q-4 0 -7.5 0.5t-8.5 1.5v-73q20 -4 43 -4q57 0 90.5 31.5t33.5 100.5z" />
-    <glyph glyph-name="k" unicode="k" horiz-adv-x="519" 
-d="M69 714v-714h85v196l80 74l177 -270h108l-220 329l205 188h-114l-236 -226v423h-85z" />
-    <glyph glyph-name="l" unicode="l" horiz-adv-x="222" 
-d="M69 714v-714h85v714h-85z" />
-    <glyph glyph-name="m" unicode="m" horiz-adv-x="853" 
-d="M64 517v-517h85v322q0 15 7.5 37.5t24 43.5t43 36t63.5 15q29 0 47.5 -8.5t29.5 -24t15.5 -36.5t4.5 -46v-339h85v322q0 60 36 96t99 36q31 0 50.5 -9t30.5 -24.5t15 -36.5t4 -45v-339h85v379q0 40 -12.5 68.5t-35 46.5t-54 26.5t-70.5 8.5q-51 0 -93.5 -23t-68.5 -65
-q-16 48 -55 68t-87 20q-109 0 -167 -88h-2v76h-80z" />
-    <glyph glyph-name="n" unicode="n" 
-d="M64 517v-517h85v292q0 35 9.5 64.5t28.5 51.5t47.5 34t67.5 12q49 0 77 -28t28 -76v-350h85v340q0 42 -8.5 76.5t-29.5 59.5t-55 39t-85 14q-115 0 -168 -94h-2v82h-80z" />
-    <glyph glyph-name="o" unicode="o" horiz-adv-x="574" 
-d="M126 258q0 47 12.5 83.5t34.5 61.5t51.5 38t62.5 13t62.5 -13t51.5 -38t34.5 -61.5t12.5 -83.5t-12.5 -83.5t-34.5 -61t-51.5 -37.5t-62.5 -13t-62.5 13t-51.5 37.5t-34.5 61t-12.5 83.5zM36 258q0 -57 16 -106t48 -85.5t79 -57t108 -20.5q62 0 108.5 20.5t78.5 57
-t48 85.5t16 106t-16 106.5t-48 86t-78.5 57.5t-108.5 21q-61 0 -108 -21t-79 -57.5t-48 -86t-16 -106.5z" />
-    <glyph glyph-name="p" unicode="p" horiz-adv-x="593" 
-d="M67 517v-714h85v266h2q14 -23 34.5 -38.5t43.5 -24.5t46.5 -13t42.5 -4q59 0 103.5 21t74 57.5t44 85.5t14.5 104t-15 104t-44.5 86.5t-74 59.5t-104.5 22q-54 0 -99 -19.5t-66 -62.5h-2v70h-85zM467 263q0 -38 -8 -74t-26 -64t-47.5 -45t-73.5 -17t-75 16.5t-50.5 43.5
-t-28.5 62.5t-9 73.5q0 36 8.5 71t27.5 62.5t49 44.5t73 17q41 0 71.5 -16t50 -43t29 -61.5t9.5 -70.5z" />
-    <glyph glyph-name="q" unicode="q" horiz-adv-x="593" 
-d="M126 254q0 38 8 74t26 64t48 45t73 17q44 0 75 -16.5t50.5 -43.5t28.5 -62.5t9 -73.5q0 -36 -8.5 -71t-27.5 -62.5t-49 -44.5t-73 -17q-41 0 -71.5 16t-50 43t-29 61.5t-9.5 70.5zM526 -197v714h-85v-69h-2q-14 23 -34.5 38.5t-43.5 25t-46 13.5t-43 4
-q-59 0 -103.5 -21.5t-74 -58t-44 -85.5t-14.5 -104t15 -104t44.5 -86t74 -59t104.5 -22q54 0 99 19t66 62h2v-267h85z" />
-    <glyph glyph-name="r" unicode="r" horiz-adv-x="333" 
-d="M61 517v-517h85v230q0 50 10 88.5t32 65.5t58 41t87 14v90h-9q-63 0 -105 -28q-45 -30 -76 -93h-2v109h-80z" />
-    <glyph glyph-name="s" unicode="s" horiz-adv-x="500" 
-d="M116 163h-85q2 -48 20 -81.5t48 -54t69 -29.5t82 -9q39 0 78.5 7.5t71 26.5t51 50t19.5 78q0 37 -14.5 62t-38 41.5t-54 26.5t-62.5 17l-60 14q-30 6 -54 15.5t-39.5 24t-15.5 36.5q0 20 10 32.5t26 20t35.5 10.5t38.5 3q21 0 41.5 -4.5t37.5 -14.5t28 -26.5t13 -41.5h85
-q-3 47 -20 78.5t-45.5 50t-65.5 26t-81 7.5q-34 0 -68.5 -8.5t-62 -26t-45 -45.5t-17.5 -67q0 -50 25 -78t62.5 -43.5t81.5 -24t81.5 -19t62.5 -27.5t25 -50q0 -24 -12 -39.5t-30.5 -23.5t-40.5 -11t-42 -3q-26 0 -50.5 5t-44 16.5t-31.5 31t-13 47.5z" />
-    <glyph glyph-name="t" unicode="t" horiz-adv-x="315" 
-d="M182 517v155h-85v-155h-88v-75h88v-329q0 -36 7 -58t21.5 -34t38 -16.5t56.5 -4.5h65v75h-39q-20 0 -32.5 1.5t-19.5 6.5t-9.5 14t-2.5 24v321h103v75h-103z" />
-    <glyph glyph-name="u" unicode="u" 
-d="M492 0v517h-85v-292q0 -35 -9.5 -64.5t-28.5 -51.5t-47.5 -34t-67.5 -12q-49 0 -77 28t-28 76v350h-85v-340q0 -42 8.5 -76.5t29.5 -59.5t55 -38.5t85 -13.5q57 0 99 22.5t69 70.5h2v-82h80z" />
-    <glyph glyph-name="v" unicode="v" horiz-adv-x="500" 
-d="M297 0l189 517h-89l-141 -431h-2l-145 431h-95l192 -517h91z" />
-    <glyph glyph-name="w" unicode="w" horiz-adv-x="758" 
-d="M576 0l165 517h-88l-116 -423h-2l-108 423h-93l-104 -423h-2l-117 423h-94l166 -517h92l104 411h2l105 -411h90z" />
-    <glyph glyph-name="x" unicode="x" horiz-adv-x="518" 
-d="M203 272l-194 -272h103l144 214l144 -214h109l-200 279l178 238h-102l-129 -181l-124 181h-109z" />
-    <glyph glyph-name="y" unicode="y" horiz-adv-x="500" 
-d="M267 -68l225 585h-90l-145 -423h-2l-151 423h-96l205 -514l-35 -87q-12 -23 -28 -35t-42 -12q-13 0 -26 4l-24 8v-78q14 -5 29 -7t30 -2q29 0 50 7.5t38.5 24t32 42.5t29.5 64z" />
-    <glyph glyph-name="z" unicode="z" horiz-adv-x="480" 
-d="M22 65v-65h436v75h-330l316 384v58h-402v-75h292z" />
-    <glyph glyph-name="braceleft" unicode="{" horiz-adv-x="333" 
-d="M309 671v60h-70q-19 0 -38 -8.5t-34.5 -25t-25.5 -41t-10 -56.5v-182q0 -26 -7 -48t-18.5 -37.5t-26.5 -24t-31 -8.5v-68q16 0 31 -8t26.5 -23t18.5 -37t7 -49v-181q0 -32 10 -56.5t25.5 -41t34.5 -25t38 -8.5h70v60h-50q-15 0 -24 7.5t-14.5 20t-7.5 28t-2 30.5v182
-q0 38 -10.5 63t-24.5 40.5t-28.5 22.5t-23.5 8v3q9 2 23.5 10t28.5 24t24.5 40.5t10.5 59.5v183q0 16 2 31.5t7.5 27.5t14.5 19.5t24 7.5h50z" />
-    <glyph glyph-name="bar" unicode="|" horiz-adv-x="222" 
-d="M77 786v-999h68v999h-68z" />
-    <glyph glyph-name="braceright" unicode="}" horiz-adv-x="333" 
-d="M24 -137v-60h70q19 0 38 8.5t34.5 25t25.5 41t10 56.5v181q0 26 7 47.5t18.5 37t26.5 24t31 8.5v68q-16 0 -31 8t-26.5 23.5t-18.5 37.5t-7 49v182q0 32 -10 56.5t-25.5 41t-34.5 25t-38 8.5h-70v-60h50q15 0 24 -7.5t14.5 -20t7.5 -28t2 -32.5v-181q0 -38 10.5 -63
-t24.5 -40.5t28.5 -22.5t23.5 -8v-3q-9 -2 -23.5 -10t-28.5 -24t-24.5 -40.5t-10.5 -59.5v-182q0 -16 -2 -31.5t-7.5 -27.5t-14.5 -19.5t-24 -7.5h-50z" />
-    <glyph glyph-name="asciitilde" unicode="~" horiz-adv-x="600" 
-d="M531 269l-36 50l-18 -27q-9 -13 -19.5 -24t-23 -18t-27.5 -7q-20 0 -47 14t-55 26q-29 13 -57 22.5t-54 9.5q-25 0 -44.5 -7.5t-35 -19.5t-26.5 -27.5t-19 -31.5l36 -50q6 13 13.5 26.5t18 24.5t23.5 18t29 7q35 0 63 -12.5t52 -26.5q25 -14 49.5 -23.5t52.5 -9.5
-q23 0 41.5 7.5t33.5 19.5t27 27.5t23 31.5z" />
-    <glyph glyph-name="uni00A0" unicode="&#xa0;" horiz-adv-x="278" 
- />
-    <glyph glyph-name="exclamdown" unicode="&#xa1;" horiz-adv-x="259" 
-d="M185 406v111h-111v-111h111zM82 22v-219h95v219l-25 311h-45z" />
-    <glyph glyph-name="cent" unicode="&#xa2;" 
-d="M272 -11v-104h39v104q47 0 83.5 17t62.5 45.5t41 64.5t19 73h-85q-8 -51 -37.5 -86t-83.5 -40v391q48 -2 77 -29.5t39 -73.5h87q-6 45 -23 78.5t-43.5 55t-61 32.5t-75.5 12v95h-39v-95q-56 -4 -98.5 -27.5t-71 -61t-42.5 -86t-14 -102.5q0 -55 15 -102.5t43.5 -82.5
-t70.5 -55.5t97 -22.5zM272 454v-391q-36 3 -62 20t-42.5 43t-24 59t-7.5 67q0 37 7 71.5t22.5 62t41.5 45.5t65 23z" />
-    <glyph glyph-name="sterling" unicode="&#xa3;" 
-d="M47 372v-45h85q9 -20 17 -45t8 -48q0 -29 -9 -55t-24.5 -49t-35.5 -42t-41 -33l47 -65q23 15 47 24.5t53 9.5t54.5 -6.5t50.5 -14.5l50 -13q24 -6 49 -6q39 0 77.5 9.5t68.5 35.5l-43 66q-18 -14 -41.5 -25.5t-47.5 -11.5q-23 0 -44 6l-40 12q-20 5 -43 10t-52 5
-q-30 0 -54.5 -9t-50.5 -20l-2 2q22 18 43 37t37 41.5t26 49t10 60.5q0 21 -6 40t-14 35h137v45h-159l-18 35q-9 17 -16.5 35t-12.5 38.5t-5 44.5q0 28 11 52t30.5 41.5t45.5 27.5t56 10q72 0 108 -47.5t38 -135.5h85v8q0 123 -62 185q-63 65 -174 65q-46 0 -87 -14
-t-72.5 -40t-50 -63.5t-18.5 -85.5t13.5 -85t33.5 -71h-58z" />
-    <glyph glyph-name="currency" unicode="&#xa4;" 
-d="M281 524q35 0 64.5 -14t51 -37.5t34 -55t12.5 -66.5q0 -36 -12.5 -68t-34 -55.5t-51.5 -37.5t-65 -14t-65.5 13.5t-53 36.5t-35.5 55t-13 68q0 37 13 69t35.5 55.5t53 37t66.5 13.5zM74 604l-49 -49l57 -58q-28 -28 -41 -67t-13 -81t13.5 -79t38.5 -67l-55 -58l48 -48
-l55 58q30 -27 68.5 -40.5t83.5 -13.5q42 0 80.5 14t67.5 41l56 -58l47 47l-55 56q26 31 39 69.5t13 80.5t-13 81t-39 67l55 55l-50 51l-57 -56q-27 23 -65 36.5t-78 13.5q-42 0 -82 -13.5t-69 -37.5z" />
-    <glyph glyph-name="yen" unicode="&#xa5;" 
-d="M232 174v-174h95v174h144v60h-144v42l18 33h126v60h-93l178 345h-104l-169 -353l-174 353h-109l183 -345h-92v-60h123l18 -33v-42h-141v-60h141z" />
-    <glyph glyph-name="brokenbar" unicode="&#xa6;" horiz-adv-x="222" 
-d="M145 411v250h-68v-250h68zM77 161v-249h68v249h-68z" />
-    <glyph glyph-name="section" unicode="&#xa7;" 
-d="M380 546h85q-2 94 -52 139.5t-132 45.5q-34 0 -67.5 -10.5t-59.5 -31.5t-42 -51t-16 -69q0 -32 12 -57.5t34 -48.5q-22 -6 -41.5 -19.5t-34 -32t-23 -41t-8.5 -45.5q0 -41 15 -72t39.5 -56t56 -44.5t63.5 -36.5l64 -35q31 -17 55.5 -35t39.5 -39t15 -46q0 -21 -9 -36.5
-t-23 -26t-32.5 -15.5t-36.5 -5q-51 0 -82 31.5t-32 82.5h-85q6 -88 58.5 -138.5t142.5 -50.5q41 0 75 11.5t59 32.5t39 50.5t14 65.5q0 75 -59 119q45 18 74.5 57.5t29.5 89.5q0 37 -14.5 65.5t-37.5 51.5t-53 41.5t-61 35.5l-61 33q-30 16 -53.5 33.5t-37.5 37t-14 43.5
-q0 45 27.5 65.5t67.5 20.5q48 0 74 -32t27 -78zM190 420q34 -23 68 -44l70 -41l34 -21q19 -12 35 -26.5t26.5 -32t10.5 -38.5q0 -34 -21.5 -57.5t-50.5 -35.5q-34 23 -69 43.5t-71 39.5l-37 23q-19 11 -34 25t-24.5 32t-9.5 42q0 32 22 56.5t51 34.5z" />
-    <glyph glyph-name="dieresis" unicode="&#xa8;" horiz-adv-x="222" 
-d="M-27 696v-104h89v104h-89zM160 696v-104h90v104h-90z" />
-    <glyph glyph-name="copyright" unicode="&#xa9;" horiz-adv-x="800" 
-d="M599 285h-64q-7 -45 -39 -73.5t-83 -28.5q-36 0 -64.5 14.5t-48.5 38.5t-30.5 56t-10.5 67q0 36 10 67t29 54.5t47.5 36.5t65.5 13q51 0 83.5 -27.5t40.5 -69.5h63q-12 75 -64 113.5t-124 38.5q-51 0 -92 -17t-69 -47t-43 -72t-15 -92q0 -49 15.5 -91t44.5 -72.5t70 -48
-t92 -17.5q69 0 119.5 42t66.5 115zM92 359q0 67 23.5 125t65 100.5t98 67t122.5 24.5q65 0 121 -24.5t97.5 -67t65 -100.5t23.5 -125q0 -69 -23.5 -127.5t-65 -101.5t-97.5 -67.5t-121 -24.5q-66 0 -122.5 24.5t-98 67.5t-65 101.5t-23.5 127.5zM24 359q0 -81 30 -149.5
-t81.5 -119t120 -78.5t145.5 -28t145 28t119 78.5t81 119t30 149.5q0 79 -30 147t-81 118t-119 78.5t-145 28.5t-145.5 -28.5t-120 -78.5t-81.5 -118t-30 -147z" />
-    <glyph glyph-name="ordfeminine" unicode="&#xaa;" horiz-adv-x="378" 
-d="M155 430q-25 0 -43.5 7.5t-18.5 33.5t18.5 36.5t45.5 15.5q28 5 56 9t44 14v-47q0 -11 -5.5 -23.5t-17.5 -22.5t-31.5 -16.5t-47.5 -6.5zM317 462v160q0 27 -12 44t-30.5 26.5t-41.5 13t-46 3.5q-28 0 -54.5 -5.5t-46.5 -18.5t-32.5 -33t-13.5 -49h60q2 32 24.5 44
-t58.5 12q14 0 28 -1.5t25 -6.5t17.5 -14.5t6.5 -26.5q0 -10 -5 -16.5t-17.5 -11t-35 -7.5t-57.5 -7q-22 -2 -43.5 -8t-38.5 -17.5t-27.5 -29.5t-10.5 -44q0 -23 9.5 -40t25.5 -28t37 -16t45 -5q74 0 117 46q3 -46 51 -46q13 0 23 2t20 6v44q-6 -2 -11 -2h-11q-8 0 -11 7
-t-3 25z" />
-    <glyph glyph-name="guillemotleft" unicode="&#xab;" horiz-adv-x="463" 
-d="M50 234l152 -120v86l-100 78l100 78v86l-152 -120v-88zM246 234l152 -120v86l-100 78l100 78v86l-152 -120v-88z" />
-    <glyph glyph-name="logicalnot" unicode="&#xac;" horiz-adv-x="600" 
-d="M552 109v279h-504v-68h436v-211h68z" />
-    <glyph glyph-name="uni00AD" unicode="&#xad;" horiz-adv-x="389" 
-d="M50 318v-80h289v80h-289z" />
-    <glyph glyph-name="registered" unicode="&#xae;" horiz-adv-x="800" 
-d="M323 382v139h90q17 0 34.5 -2.5t31 -9.5t22 -20t8.5 -35q0 -26 -9.5 -40.5t-25.5 -21.5t-36.5 -8.5t-43.5 -1.5h-71zM255 576v-437h68v188h72l119 -188h73l-125 193q49 5 82 32.5t33 86.5q0 63 -37.5 94t-115.5 31h-169zM92 359q0 67 23.5 125t65 100.5t98 67t122.5 24.5
-q65 0 121 -24.5t97.5 -67t65 -100.5t23.5 -125q0 -69 -23.5 -127.5t-65 -101.5t-97.5 -67.5t-121 -24.5q-66 0 -122.5 24.5t-98 67.5t-65 101.5t-23.5 127.5zM24 359q0 -81 30 -149.5t81.5 -119t120 -78.5t145.5 -28t145 28t119 78.5t81 119t30 149.5q0 79 -30 147t-81 118
-t-119 78.5t-145 28.5t-145.5 -28.5t-120 -78.5t-81.5 -118t-30 -147z" />
-    <glyph glyph-name="macron" unicode="&#xaf;" horiz-adv-x="222" 
-d="M-58 675v-55h340v55h-340z" />
-    <glyph glyph-name="degree" unicode="&#xb0;" horiz-adv-x="400" 
-d="M106 565q0 18 6.5 34.5t18.5 30t29.5 21.5t39.5 8t39.5 -8t29.5 -21.5t18.5 -30t6.5 -34.5t-6.5 -34.5t-18.5 -30t-29.5 -21.5t-39.5 -8t-39.5 8t-29.5 21.5t-18.5 30t-6.5 34.5zM56 565q0 -30 11 -56.5t30.5 -46t46 -30.5t56.5 -11t56 11t45.5 30.5t31 46t11.5 56.5
-t-11.5 56t-31 45.5t-45.5 31t-56 11.5t-56.5 -11.5t-46 -31t-30.5 -45.5t-11 -56z" />
-    <glyph glyph-name="plusminus" unicode="&#xb1;" horiz-adv-x="600" 
-d="M48 68v-68h504v68h-504zM48 339v-68h218v-167h68v167h218v68h-218v166h-68v-166h-218z" />
-    <glyph glyph-name="uni00B2" unicode="&#xb2;" horiz-adv-x="333" 
-d="M22 555h68q0 39 18.5 69t63.5 30q35 0 59 -19t24 -51q0 -22 -7 -39t-21.5 -31.5t-36.5 -28.5l-51 -31q-24 -14 -46 -29t-39 -34.5t-27.5 -46t-12.5 -64.5h303v55h-235q4 16 16.5 29t30 25t39 24t42.5 25q21 12 41.5 26t36 31t25.5 38.5t10 49.5q0 30 -11.5 53.5t-32 39.5
-t-47.5 24.5t-57 8.5q-70 0 -111.5 -42.5t-41.5 -111.5z" />
-    <glyph glyph-name="uni00B3" unicode="&#xb3;" horiz-adv-x="333" 
-d="M132 527v-50h33q18 0 34.5 -4t29.5 -13.5t21 -24t8 -35.5q0 -34 -28 -53t-66 -19q-90 0 -92 86h-68q0 -35 12.5 -61.5t34 -44t51 -26.5t62.5 -9q32 0 61 8t51.5 24t36 40.5t13.5 57.5q0 38 -21 66t-66 36v2q28 11 48 33.5t20 51.5t-11 51t-30.5 36.5t-46 22t-57.5 7.5
-q-67 0 -105.5 -37.5t-42.5 -103.5h68q0 37 19 61.5t61 24.5q33 0 55 -16t22 -46q0 -35 -25 -50t-58 -15h-24z" />
-    <glyph glyph-name="acute" unicode="&#xb4;" horiz-adv-x="222" 
-d="M136 731l-86 -143h64l131 143h-109z" />
-    <glyph glyph-name="uni00B5" unicode="&#xb5;" 
-d="M492 0v517h-85v-292q0 -35 -9.5 -64.5t-28.5 -51.5t-47.5 -34t-67.5 -12q-49 0 -77 28t-28 76v350h-85v-714h85v223h2q13 -20 36 -28.5t55 -8.5q57 0 99 22.5t69 70.5h2v-82h80z" />
-    <glyph glyph-name="paragraph" unicode="&#xb6;" horiz-adv-x="600" 
-d="M515 -154v868h-239q-106 0 -167 -53.5t-61 -152.5q0 -39 15.5 -73t43 -59t66 -39.5t84.5 -14.5v-476h68v813h122v-813h68z" />
-    <glyph glyph-name="periodcentered" unicode="&#xb7;" horiz-adv-x="278" 
-d="M75 311q0 -26 19.5 -45t45.5 -19q25 0 44 20t19 44q0 26 -19 45t-45 19t-45 -19t-19 -45z" />
-    <glyph glyph-name="cedilla" unicode="&#xb8;" horiz-adv-x="222" 
-d="M23 -146l-18 -44q23 -8 45.5 -13.5t46.5 -5.5q21 0 43 4.5t40 15t29.5 27.5t11.5 42q0 37 -26 55t-58 18q-23 0 -33 -7l-2 2l42 52h-42l-61 -77l17 -28q11 4 20 6t21 2q20 0 33.5 -8.5t13.5 -22.5q0 -9 -6 -14.5t-15 -9.5t-20 -5.5t-21 -1.5q-16 0 -32.5 4t-28.5 9z" />
-    <glyph glyph-name="uni00B9" unicode="&#xb9;" horiz-adv-x="333" 
-d="M249 280v429h-50q-10 -41 -47 -56t-86 -15v-50h115v-308h68z" />
-    <glyph glyph-name="ordmasculine" unicode="&#xba;" horiz-adv-x="384" 
-d="M192 659q16 0 33.5 -4.5t32 -17.5t24 -36t9.5 -59q0 -22 -5.5 -42.5t-17 -36t-30 -24.5t-45.5 -9t-46 9t-31 25t-17.5 36.5t-5.5 42.5q0 60 28.5 88t70.5 28zM192 709q-40 0 -71.5 -13t-52.5 -35.5t-32 -52.5t-11 -65q0 -36 11.5 -65.5t33 -51.5t52.5 -34t71 -12
-q82 0 124 45.5t42 117.5q0 35 -11 65.5t-32 53t-52 35t-72 12.5z" />
-    <glyph glyph-name="guillemotright" unicode="&#xbb;" horiz-adv-x="463" 
-d="M413 322l-152 120v-86l100 -78l-100 -78v-86l152 120v88zM217 322l-152 120v-86l100 -78l-100 -78v-86l152 120v88z" />
-    <glyph glyph-name="onequarter" unicode="&#xbc;" horiz-adv-x="834" 
-d="M219 280v429h-50q-10 -42 -47 -56.5t-86 -14.5v-50h115v-308h68zM585 725l-435 -752h64l435 752h-64zM511 151l148 193h2v-193h-150zM723 151v278h-51l-210 -275v-58h199v-96h62v96h60v55h-60z" />
-    <glyph glyph-name="onehalf" unicode="&#xbd;" horiz-adv-x="834" 
-d="M502 275h68q0 39 18.5 69t63.5 30q35 0 59 -19t24 -51q0 -22 -7 -39t-21.5 -31.5t-36.5 -28.5l-51 -31q-24 -14 -46 -29t-39 -34.5t-27.5 -46t-12.5 -64.5h303v55h-235q4 16 16.5 29t30 25t39 24t42.5 25q21 12 41.5 26t36 31t25.5 38.5t10 49.5q0 30 -11.5 53.5
-t-32 39.5t-47.5 24.5t-57 8.5q-70 0 -111.5 -42.5t-41.5 -111.5zM219 280v429h-50q-10 -42 -47 -56.5t-86 -14.5v-50h115v-308h68zM570 725l-435 -752h64l435 752h-64z" />
-    <glyph glyph-name="threequarters" unicode="&#xbe;" horiz-adv-x="834" 
-d="M177 527v-50h33q18 0 34.5 -4t29.5 -13.5t21 -24t8 -35.5q0 -34 -28 -53t-66 -19q-90 0 -92 86h-68q0 -35 12.5 -61.5t34 -44t51 -26.5t62.5 -9q32 0 61 8t51.5 24t36 40.5t13.5 57.5q0 38 -21 66t-66 36v2q28 11 48 33.5t20 51.5t-11 51t-30.5 36.5t-46 22t-57.5 7.5
-q-67 0 -105.5 -37.5t-42.5 -103.5h68q0 37 19 61.5t61 24.5q33 0 55 -16t22 -46q0 -35 -25 -50t-58 -15h-24zM650 725l-435 -752h64l435 752h-64zM529 147l148 193h2v-193h-150zM740 151v278h-51l-210 -275v-58h199v-96h62v96h60v55h-60z" />
-    <glyph glyph-name="questiondown" unicode="&#xbf;" 
-d="M337 406v111h-111v-111h111zM503 30h-85q0 -34 -8 -64t-25.5 -52t-43.5 -35t-62 -13q-54 0 -85.5 31t-35.5 86v11q0 29 9 51q11 26 29.5 47.5t40 41t41 42.5t33 53t15.5 71v39h-85v-31q0 -25 -7.5 -46t-20 -38.5t-27.5 -33.5l-32 -30q-17 -16 -32.5 -33t-27.5 -37
-t-19 -44.5t-7 -54.5q0 -48 15.5 -85t43.5 -62.5t67 -39t85 -13.5q52 0 94 18.5t71.5 50.5t44.5 76q14 40 14 88v6z" />
-    <glyph glyph-name="Agrave" unicode="&#xc0;" horiz-adv-x="648" 
-d="M190 908l131 -143h64l-86 143h-109zM201 295l121 333h2l119 -333h-242zM272 714l-278 -714h97l80 215h302l78 -215h105l-279 714h-105z" />
-    <glyph glyph-name="Aacute" unicode="&#xc1;" horiz-adv-x="648" 
-d="M349 908l-86 -143h64l131 143h-109zM201 295l121 333h2l119 -333h-242zM272 714l-278 -714h97l80 215h302l78 -215h105l-279 714h-105z" />
-    <glyph glyph-name="Acircumflex" unicode="&#xc2;" horiz-adv-x="648" 
-d="M277 908l-103 -143h77l70 96l69 -96h84l-104 143h-93zM201 295l121 333h2l119 -333h-242zM272 714l-278 -714h97l80 215h302l78 -215h105l-279 714h-105z" />
-    <glyph glyph-name="Atilde" unicode="&#xc3;" horiz-adv-x="648" 
-d="M498 888h-52q-5 -15 -15.5 -29.5t-30.5 -14.5q-16 0 -32.5 6.5t-33.5 14t-35.5 13.5t-38.5 6q-24 0 -41 -9t-29 -24t-20.5 -35t-14.5 -42h52q-2 19 15 30.5t33 11.5q14 0 31.5 -6t35.5 -13q18 -8 35.5 -14t30.5 -6q29 0 47.5 8.5t31.5 22.5q25 35 31 80zM201 295l121 333
-h2l119 -333h-242zM272 714l-278 -714h97l80 215h302l78 -215h105l-279 714h-105z" />
-    <glyph glyph-name="Adieresis" unicode="&#xc4;" horiz-adv-x="648" 
-d="M185 873v-104h90v104h-90zM373 873v-104h90v104h-90zM201 295l121 333h2l119 -333h-242zM272 714l-278 -714h97l80 215h302l78 -215h105l-279 714h-105z" />
-    <glyph glyph-name="Aring" unicode="&#xc5;" horiz-adv-x="648" 
-d="M211 839q0 -23 9 -43.5t24.5 -36t36 -24.5t43.5 -9t43.5 9t36 24.5t24.5 36t9 43.5t-9 43.5t-24.5 36t-36 24.5t-43.5 9t-43.5 -9t-36 -24.5t-24.5 -36t-9 -43.5zM256 839q0 29 19.5 48.5t48.5 19.5t48.5 -19.5t19.5 -48.5t-19.5 -48.5t-48.5 -19.5t-48.5 19.5
-t-19.5 48.5zM201 295l121 333h2l119 -333h-242zM272 714l-278 -714h97l80 215h302l78 -215h105l-279 714h-105z" />
-    <glyph glyph-name="AE" unicode="&#xc6;" horiz-adv-x="926" 
-d="M407 634h41v-346h-209zM354 714l-360 -714h106l102 208h246v-208h447v80h-357v247h330v80h-330v227h351v80h-535z" />
-    <glyph glyph-name="Ccedilla" unicode="&#xc7;" horiz-adv-x="722" 
-d="M580 499h95q-8 57 -34 100.5t-65 72.5t-89 44t-106 15q-82 0 -145.5 -29.5t-106 -80.5t-64.5 -119.5t-22 -146.5q0 -75 18.5 -139.5t55 -113.5t92.5 -79.5t131 -36.5l-49 -63l17 -28q11 4 20 6t21 2q20 0 33.5 -8.5t13.5 -22.5q0 -9 -6 -14.5t-15 -9.5t-20 -5.5t-21 -1.5
-q-16 0 -32.5 4t-28.5 9l-18 -44q23 -8 45.5 -13.5t46.5 -5.5q21 0 43 4.5t40 15t29.5 27.5t11.5 42q0 37 -26 55t-58 18q-23 0 -33 -7l-2 2l29 37q132 2 209.5 77t91.5 207h-95q-3 -44 -18 -82t-41.5 -65.5t-63 -43t-83.5 -15.5q-64 0 -110 24t-75.5 64.5t-43.5 95
-t-14 115.5q0 56 14 108t43.5 92.5t75 64.5t109.5 24q75 0 129.5 -38t70.5 -114z" />
-    <glyph glyph-name="Egrave" unicode="&#xc8;" horiz-adv-x="611" 
-d="M172 908l131 -143h64l-86 143h-109zM78 714v-714h496v80h-401v247h371v80h-371v227h398v80h-493z" />
-    <glyph glyph-name="Eacute" unicode="&#xc9;" horiz-adv-x="611" 
-d="M331 908l-86 -143h64l131 143h-109zM78 714v-714h496v80h-401v247h371v80h-371v227h398v80h-493z" />
-    <glyph glyph-name="Ecircumflex" unicode="&#xca;" horiz-adv-x="611" 
-d="M259 908l-103 -143h77l70 96l69 -96h84l-104 143h-93zM78 714v-714h496v80h-401v247h371v80h-371v227h398v80h-493z" />
-    <glyph glyph-name="Edieresis" unicode="&#xcb;" horiz-adv-x="611" 
-d="M167 873v-104h90v104h-90zM355 873v-104h90v104h-90zM78 714v-714h496v80h-401v247h371v80h-371v227h398v80h-493z" />
-    <glyph glyph-name="Igrave" unicode="&#xcc;" horiz-adv-x="259" 
-d="M-3 908l130 -143h64l-86 143h-108zM82 714v-714h95v714h-95z" />
-    <glyph glyph-name="Iacute" unicode="&#xcd;" horiz-adv-x="259" 
-d="M155 908l-86 -143h64l131 143h-109zM82 714v-714h95v714h-95z" />
-    <glyph glyph-name="Icircumflex" unicode="&#xce;" horiz-adv-x="259" 
-d="M83 908l-102 -143h76l70 96l69 -96h84l-104 143h-93zM82 714v-714h95v714h-95z" />
-    <glyph glyph-name="Idieresis" unicode="&#xcf;" horiz-adv-x="259" 
-d="M-8 873v-104h89v104h-89zM179 873v-104h90v104h-90zM82 714v-714h95v714h-95z" />
-    <glyph glyph-name="Eth" unicode="&#xd0;" horiz-adv-x="704" 
-d="M173 407v227h160q66 0 111 -18.5t73.5 -54t41 -86t12.5 -114.5q0 -66 -13.5 -112.5t-34.5 -77.5t-47.5 -49t-53.5 -27.5t-51 -12t-40 -2.5h-158v259h214v68h-214zM78 339v-339h245q89 0 154 25t107 72.5t62 117t20 159.5q0 172 -89 256t-254 84h-245v-307h-70v-68h70z
-" />
-    <glyph glyph-name="Ntilde" unicode="&#xd1;" horiz-adv-x="722" 
-d="M535 888h-52q-5 -15 -15.5 -29.5t-30.5 -14.5q-16 0 -32.5 6.5t-33.5 14t-35.5 13.5t-38.5 6q-24 0 -41 -9t-29 -24t-20.5 -35t-14.5 -42h52q-2 19 15 30.5t33 11.5q14 0 31.5 -6t35.5 -13q18 -8 35.5 -14t30.5 -6q29 0 47.5 8.5t31.5 22.5q25 35 31 80zM77 714v-714h90
-v573h2l373 -573h104v714h-90v-579h-2l-376 579h-101z" />
-    <glyph glyph-name="Ograve" unicode="&#xd2;" horiz-adv-x="760" 
-d="M246 908l131 -143h64l-86 143h-109zM133 357q0 54 14 106.5t44 94.5t77 67.5t112 25.5t112 -25.5t77 -67.5t44 -94.5t14 -106.5t-14 -106.5t-44 -94.5t-77 -67.5t-112 -25.5t-112 25.5t-77 67.5t-44 94.5t-14 106.5zM38 357q0 -73 21.5 -140.5t64.5 -119.5t107 -82.5
-t149 -30.5t149 30.5t107 82.5t64.5 119.5t21.5 140.5t-21.5 140.5t-64.5 119.5t-107 83t-149 31t-149 -31t-107 -83t-64.5 -119.5t-21.5 -140.5z" />
-    <glyph glyph-name="Oacute" unicode="&#xd3;" horiz-adv-x="760" 
-d="M405 908l-86 -143h64l131 143h-109zM133 357q0 54 14 106.5t44 94.5t77 67.5t112 25.5t112 -25.5t77 -67.5t44 -94.5t14 -106.5t-14 -106.5t-44 -94.5t-77 -67.5t-112 -25.5t-112 25.5t-77 67.5t-44 94.5t-14 106.5zM38 357q0 -73 21.5 -140.5t64.5 -119.5t107 -82.5
-t149 -30.5t149 30.5t107 82.5t64.5 119.5t21.5 140.5t-21.5 140.5t-64.5 119.5t-107 83t-149 31t-149 -31t-107 -83t-64.5 -119.5t-21.5 -140.5z" />
-    <glyph glyph-name="Ocircumflex" unicode="&#xd4;" horiz-adv-x="760" 
-d="M333 908l-103 -143h77l70 96l69 -96h84l-104 143h-93zM133 357q0 54 14 106.5t44 94.5t77 67.5t112 25.5t112 -25.5t77 -67.5t44 -94.5t14 -106.5t-14 -106.5t-44 -94.5t-77 -67.5t-112 -25.5t-112 25.5t-77 67.5t-44 94.5t-14 106.5zM38 357q0 -73 21.5 -140.5
-t64.5 -119.5t107 -82.5t149 -30.5t149 30.5t107 82.5t64.5 119.5t21.5 140.5t-21.5 140.5t-64.5 119.5t-107 83t-149 31t-149 -31t-107 -83t-64.5 -119.5t-21.5 -140.5z" />
-    <glyph glyph-name="Otilde" unicode="&#xd5;" horiz-adv-x="760" 
-d="M554 888h-52q-5 -15 -15.5 -29.5t-30.5 -14.5q-16 0 -32.5 6.5t-33.5 14t-35.5 13.5t-38.5 6q-24 0 -41 -9t-29 -24t-20.5 -35t-14.5 -42h52q-2 19 15 30.5t33 11.5q14 0 31.5 -6t35.5 -13q18 -8 35.5 -14t30.5 -6q29 0 47.5 8.5t31.5 22.5q25 35 31 80zM133 357
-q0 54 14 106.5t44 94.5t77 67.5t112 25.5t112 -25.5t77 -67.5t44 -94.5t14 -106.5t-14 -106.5t-44 -94.5t-77 -67.5t-112 -25.5t-112 25.5t-77 67.5t-44 94.5t-14 106.5zM38 357q0 -73 21.5 -140.5t64.5 -119.5t107 -82.5t149 -30.5t149 30.5t107 82.5t64.5 119.5
-t21.5 140.5t-21.5 140.5t-64.5 119.5t-107 83t-149 31t-149 -31t-107 -83t-64.5 -119.5t-21.5 -140.5z" />
-    <glyph glyph-name="Odieresis" unicode="&#xd6;" horiz-adv-x="760" 
-d="M241 873v-104h90v104h-90zM429 873v-104h90v104h-90zM133 357q0 54 14 106.5t44 94.5t77 67.5t112 25.5t112 -25.5t77 -67.5t44 -94.5t14 -106.5t-14 -106.5t-44 -94.5t-77 -67.5t-112 -25.5t-112 25.5t-77 67.5t-44 94.5t-14 106.5zM38 357q0 -73 21.5 -140.5
-t64.5 -119.5t107 -82.5t149 -30.5t149 30.5t107 82.5t64.5 119.5t21.5 140.5t-21.5 140.5t-64.5 119.5t-107 83t-149 31t-149 -31t-107 -83t-64.5 -119.5t-21.5 -140.5z" />
-    <glyph glyph-name="multiply" unicode="&#xd7;" horiz-adv-x="600" 
-d="M71 72l48 -48l182 182l181 -182l48 48l-181 182l180 180l-48 48l-180 -180l-181 180l-48 -48l181 -180z" />
-    <glyph glyph-name="Oslash" unicode="&#xd8;" horiz-adv-x="760" 
-d="M723 700l-32 30l-79 -87q-42 42 -99.5 65t-132.5 23q-85 0 -149 -31t-107 -83t-64.5 -119.5t-21.5 -140.5q0 -72 20.5 -138t62.5 -117l-84 -92l33 -29l81 88q42 -40 99 -62.5t130 -22.5q85 0 149 30.5t107 82.5t64.5 119.5t21.5 140.5q0 72 -19.5 136.5t-61.5 116.5z
-M210 134l369 407q25 -39 36.5 -87t11.5 -97q0 -54 -14 -106.5t-44 -94.5t-77 -67.5t-112 -25.5q-57 0 -99 19t-71 52zM552 577l-369 -407q-26 40 -38 88t-12 99q0 54 14 106.5t44 94.5t77 67.5t112 25.5q57 0 100 -20t72 -54z" />
-    <glyph glyph-name="Ugrave" unicode="&#xd9;" horiz-adv-x="722" 
-d="M227 908l131 -143h64l-86 143h-109zM648 258v456h-95v-456q0 -96 -47.5 -145.5t-138.5 -49.5q-96 0 -147 49.5t-51 145.5v456h-95v-456q0 -143 77.5 -208.5t215.5 -65.5q135 0 208 68.5t73 205.5z" />
-    <glyph glyph-name="Uacute" unicode="&#xda;" horiz-adv-x="722" 
-d="M386 908l-86 -143h64l131 143h-109zM648 258v456h-95v-456q0 -96 -47.5 -145.5t-138.5 -49.5q-96 0 -147 49.5t-51 145.5v456h-95v-456q0 -143 77.5 -208.5t215.5 -65.5q135 0 208 68.5t73 205.5z" />
-    <glyph glyph-name="Ucircumflex" unicode="&#xdb;" horiz-adv-x="722" 
-d="M314 908l-103 -143h77l70 96l69 -96h84l-104 143h-93zM648 258v456h-95v-456q0 -96 -47.5 -145.5t-138.5 -49.5q-96 0 -147 49.5t-51 145.5v456h-95v-456q0 -143 77.5 -208.5t215.5 -65.5q135 0 208 68.5t73 205.5z" />
-    <glyph glyph-name="Udieresis" unicode="&#xdc;" horiz-adv-x="722" 
-d="M222 873v-104h90v104h-90zM410 873v-104h90v104h-90zM648 258v456h-95v-456q0 -96 -47.5 -145.5t-138.5 -49.5q-96 0 -147 49.5t-51 145.5v456h-95v-456q0 -143 77.5 -208.5t215.5 -65.5q135 0 208 68.5t73 205.5z" />
-    <glyph glyph-name="Yacute" unicode="&#xdd;" horiz-adv-x="648" 
-d="M349 908l-86 -143h64l131 143h-109zM373 292l273 422h-108l-209 -340l-214 340h-113l276 -422v-292h95v292z" />
-    <glyph glyph-name="Thorn" unicode="&#xde;" horiz-adv-x="648" 
-d="M173 258v262h186q81 0 118.5 -33t37.5 -99q0 -64 -37 -98q-37 -32 -114 -32h-5h-186zM78 0h95v178h218h4q105 0 159 54q56 56 56 156t-55.5 156t-163.5 56h-218v114h-95v-714z" />
-    <glyph glyph-name="germandbls" unicode="&#xdf;" 
-d="M64 531v-531h85v516q0 42 9 69t24.5 42.5t37.5 22t48 6.5q24 0 44 -9t35 -24t23.5 -35.5t8.5 -44.5q0 -34 -12.5 -57.5t-33.5 -37t-49 -18.5q-19 -4 -40 -4q-8 0 -17 1v-68q42 0 76 -5t58 -21t37 -45.5t13 -78.5q0 -66 -40.5 -106t-109.5 -40q-8 0 -17 0.5t-17 2.5v-73
-q20 -4 43 -4q51 0 93.5 16.5t73 47t47.5 73t17 94.5q0 33 -8.5 63.5t-25.5 54.5t-42.5 40.5t-59.5 20.5v2q57 18 80.5 58t23.5 97q0 44 -18.5 76.5t-48 54.5t-66.5 33t-74 11q-45 0 -81.5 -14.5t-62.5 -41t-40 -63t-14 -81.5z" />
-    <glyph glyph-name="agrave" unicode="&#xe0;" horiz-adv-x="537" 
-d="M135 731l131 -143h64l-86 143h-109zM522 2v66q-15 -5 -26 -5q-20 0 -23.5 14t-3.5 44v266q0 45 -17 73t-45 43.5t-62.5 20.5t-69.5 5q-43 0 -82 -9t-69 -29.5t-48 -53.5t-20 -81h85q3 57 39 77.5t90 20.5q20 0 40.5 -3t37 -12t27 -26t10.5 -45q0 -24 -14 -36.5t-38 -19
-t-55.5 -10t-66.5 -10.5q-33 -6 -64.5 -15t-56 -26.5t-39.5 -45.5t-15 -72q0 -39 13.5 -66.5t37 -45t54.5 -25t66 -7.5q54 0 100.5 18.5t81.5 60.5q0 -42 19.5 -60.5t52.5 -18.5q39 0 61 13zM384 267v-87q0 -18 -9 -38.5t-28 -38t-49 -29t-72 -11.5q-19 0 -37.5 4t-32 13
-t-22 23.5t-8.5 35.5q0 32 14.5 50.5t37.5 28.5t52 14.5t57.5 8t54.5 9t42 17.5z" />
-    <glyph glyph-name="aacute" unicode="&#xe1;" horiz-adv-x="537" 
-d="M294 731l-86 -143h64l131 143h-109zM522 2v66q-15 -5 -26 -5q-20 0 -23.5 14t-3.5 44v266q0 45 -17 73t-45 43.5t-62.5 20.5t-69.5 5q-43 0 -82 -9t-69 -29.5t-48 -53.5t-20 -81h85q3 57 39 77.5t90 20.5q20 0 40.5 -3t37 -12t27 -26t10.5 -45q0 -24 -14 -36.5t-38 -19
-t-55.5 -10t-66.5 -10.5q-33 -6 -64.5 -15t-56 -26.5t-39.5 -45.5t-15 -72q0 -39 13.5 -66.5t37 -45t54.5 -25t66 -7.5q54 0 100.5 18.5t81.5 60.5q0 -42 19.5 -60.5t52.5 -18.5q39 0 61 13zM384 267v-87q0 -18 -9 -38.5t-28 -38t-49 -29t-72 -11.5q-19 0 -37.5 4t-32 13
-t-22 23.5t-8.5 35.5q0 32 14.5 50.5t37.5 28.5t52 14.5t57.5 8t54.5 9t42 17.5z" />
-    <glyph glyph-name="acircumflex" unicode="&#xe2;" horiz-adv-x="537" 
-d="M222 731l-103 -143h77l70 96l69 -96h84l-104 143h-93zM522 2v66q-15 -5 -26 -5q-20 0 -23.5 14t-3.5 44v266q0 45 -17 73t-45 43.5t-62.5 20.5t-69.5 5q-43 0 -82 -9t-69 -29.5t-48 -53.5t-20 -81h85q3 57 39 77.5t90 20.5q20 0 40.5 -3t37 -12t27 -26t10.5 -45
-q0 -24 -14 -36.5t-38 -19t-55.5 -10t-66.5 -10.5q-33 -6 -64.5 -15t-56 -26.5t-39.5 -45.5t-15 -72q0 -39 13.5 -66.5t37 -45t54.5 -25t66 -7.5q54 0 100.5 18.5t81.5 60.5q0 -42 19.5 -60.5t52.5 -18.5q39 0 61 13zM384 267v-87q0 -18 -9 -38.5t-28 -38t-49 -29t-72 -11.5
-q-19 0 -37.5 4t-32 13t-22 23.5t-8.5 35.5q0 32 14.5 50.5t37.5 28.5t52 14.5t57.5 8t54.5 9t42 17.5z" />
-    <glyph glyph-name="atilde" unicode="&#xe3;" horiz-adv-x="537" 
-d="M443 711h-52q-5 -15 -15.5 -29.5t-30.5 -14.5q-16 0 -32.5 6.5t-33.5 14t-35.5 13.5t-38.5 6q-24 0 -41 -9t-29 -24t-20.5 -35t-14.5 -42h52q-2 19 15 30.5t33 11.5q14 0 31.5 -6t35.5 -13q18 -8 35.5 -14t30.5 -6q29 0 47.5 8.5t31.5 22.5q25 35 31 80zM522 2v66
-q-15 -5 -26 -5q-20 0 -23.5 14t-3.5 44v266q0 45 -17 73t-45 43.5t-62.5 20.5t-69.5 5q-43 0 -82 -9t-69 -29.5t-48 -53.5t-20 -81h85q3 57 39 77.5t90 20.5q20 0 40.5 -3t37 -12t27 -26t10.5 -45q0 -24 -14 -36.5t-38 -19t-55.5 -10t-66.5 -10.5q-33 -6 -64.5 -15
-t-56 -26.5t-39.5 -45.5t-15 -72q0 -39 13.5 -66.5t37 -45t54.5 -25t66 -7.5q54 0 100.5 18.5t81.5 60.5q0 -42 19.5 -60.5t52.5 -18.5q39 0 61 13zM384 267v-87q0 -18 -9 -38.5t-28 -38t-49 -29t-72 -11.5q-19 0 -37.5 4t-32 13t-22 23.5t-8.5 35.5q0 32 14.5 50.5
-t37.5 28.5t52 14.5t57.5 8t54.5 9t42 17.5z" />
-    <glyph glyph-name="adieresis" unicode="&#xe4;" horiz-adv-x="537" 
-d="M130 696v-104h90v104h-90zM318 696v-104h90v104h-90zM522 2v66q-15 -5 -26 -5q-20 0 -23.5 14t-3.5 44v266q0 45 -17 73t-45 43.5t-62.5 20.5t-69.5 5q-43 0 -82 -9t-69 -29.5t-48 -53.5t-20 -81h85q3 57 39 77.5t90 20.5q20 0 40.5 -3t37 -12t27 -26t10.5 -45
-q0 -24 -14 -36.5t-38 -19t-55.5 -10t-66.5 -10.5q-33 -6 -64.5 -15t-56 -26.5t-39.5 -45.5t-15 -72q0 -39 13.5 -66.5t37 -45t54.5 -25t66 -7.5q54 0 100.5 18.5t81.5 60.5q0 -42 19.5 -60.5t52.5 -18.5q39 0 61 13zM384 267v-87q0 -18 -9 -38.5t-28 -38t-49 -29t-72 -11.5
-q-19 0 -37.5 4t-32 13t-22 23.5t-8.5 35.5q0 32 14.5 50.5t37.5 28.5t52 14.5t57.5 8t54.5 9t42 17.5z" />
-    <glyph glyph-name="aring" unicode="&#xe5;" horiz-adv-x="537" 
-d="M156 662q0 -23 9 -43.5t24.5 -36t36 -24.5t43.5 -9t43.5 9t36 24.5t24.5 36t9 43.5t-9 43.5t-24.5 36t-36 24.5t-43.5 9t-43.5 -9t-36 -24.5t-24.5 -36t-9 -43.5zM201 662q0 29 19.5 48.5t48.5 19.5t48.5 -19.5t19.5 -48.5t-19.5 -48.5t-48.5 -19.5t-48.5 19.5
-t-19.5 48.5zM522 2v66q-15 -5 -26 -5q-20 0 -23.5 14t-3.5 44v266q0 45 -17 73t-45 43.5t-62.5 20.5t-69.5 5q-43 0 -82 -9t-69 -29.5t-48 -53.5t-20 -81h85q3 57 39 77.5t90 20.5q20 0 40.5 -3t37 -12t27 -26t10.5 -45q0 -24 -14 -36.5t-38 -19t-55.5 -10t-66.5 -10.5
-q-33 -6 -64.5 -15t-56 -26.5t-39.5 -45.5t-15 -72q0 -39 13.5 -66.5t37 -45t54.5 -25t66 -7.5q54 0 100.5 18.5t81.5 60.5q0 -42 19.5 -60.5t52.5 -18.5q39 0 61 13zM384 267v-87q0 -18 -9 -38.5t-28 -38t-49 -29t-72 -11.5q-19 0 -37.5 4t-32 13t-22 23.5t-8.5 35.5
-q0 32 14.5 50.5t37.5 28.5t52 14.5t57.5 8t54.5 9t42 17.5z" />
-    <glyph glyph-name="ae" unicode="&#xe6;" horiz-adv-x="870" 
-d="M467 233h379q0 60 -11 114t-38 94.5t-71.5 64t-112.5 23.5q-47 0 -93 -17.5t-72 -57.5q-12 24 -32 39t-44 23t-49 10.5t-48 2.5q-43 0 -82 -9t-69 -29.5t-48 -53.5t-20 -81h85q3 57 39 77.5t90 20.5q21 0 42 -2.5t37.5 -11t27 -24.5t10.5 -42q0 -23 -10 -35.5t-31 -20
-t-54.5 -12.5t-80.5 -14q-33 -6 -64.5 -15t-56 -26.5t-39.5 -45.5t-15 -72q0 -39 13.5 -66.5t37 -45t55.5 -25t69 -7.5q72 0 130.5 23.5t87.5 76.5q13 -29 32.5 -48t44 -30.5t52.5 -16.5t58 -5q88 0 146.5 44.5t77.5 129.5h-87q-12 -51 -47.5 -75.5t-83.5 -24.5
-q-39 0 -68.5 13.5t-48.5 37t-28.5 54.5t-9.5 65zM126 139q0 24 8.5 40t22.5 26.5t33 16.5t40 10q20 4 42.5 7t43.5 6.5t39.5 8.5t31.5 13v-87q0 -18 -9.5 -38.5t-29 -38t-50 -29t-72.5 -11.5q-19 0 -37.5 4t-32 13t-22 23.5t-8.5 35.5zM756 308h-289q0 32 10 58.5t29 46
-t46 30.5t61 11t60.5 -11.5t44.5 -31t27.5 -46.5t10.5 -57z" />
-    <glyph glyph-name="ccedilla" unicode="&#xe7;" horiz-adv-x="537" 
-d="M415 351h88q-5 46 -24 79.5t-48.5 55.5t-68 32.5t-82.5 10.5q-61 0 -107 -21.5t-76.5 -59t-45.5 -88t-15 -108.5q0 -55 14 -101t41 -80t67.5 -55t93.5 -25l-53 -68l17 -28q11 4 20 6t21 2q20 0 33.5 -8.5t13.5 -22.5q0 -9 -6 -14.5t-15 -9.5t-20 -5.5t-21 -1.5
-q-16 0 -32.5 4t-28.5 9l-18 -44q23 -8 45.5 -13.5t46.5 -5.5q21 0 43 4.5t40 15t29.5 27.5t11.5 42q0 37 -26 55t-58 18q-23 0 -33 -7l-2 2l33 42q91 4 145 56t68 143h-87q-8 -60 -43.5 -93t-98.5 -33q-40 0 -69 16t-47 42.5t-26.5 60.5t-8.5 70q0 39 8 75.5t26.5 64.5
-t49.5 45t77 17q54 0 86 -27t42 -76z" />
-    <glyph glyph-name="egrave" unicode="&#xe8;" horiz-adv-x="537" 
-d="M135 731l131 -143h64l-86 143h-109zM423 308h-297q2 30 13 56.5t30 46.5t45.5 31.5t59.5 11.5q32 0 58.5 -11.5t46 -31t31 -46.5t13.5 -57zM505 164h-84q-11 -51 -45.5 -76t-88.5 -25q-42 0 -73 14t-51 37.5t-29 53.5q-8 28 -8 58v7h387v15q0 40 -9 84q-10 52 -37.5 96
-t-73 72.5t-114.5 28.5q-53 0 -97.5 -20t-77 -56t-50.5 -85t-18 -108q2 -59 17.5 -109t46.5 -86t76.5 -56t107.5 -20q88 0 146 44t75 131z" />
-    <glyph glyph-name="eacute" unicode="&#xe9;" horiz-adv-x="537" 
-d="M294 731l-86 -143h64l131 143h-109zM423 308h-297q2 30 13 56.5t30 46.5t45.5 31.5t59.5 11.5q32 0 58.5 -11.5t46 -31t31 -46.5t13.5 -57zM505 164h-84q-11 -51 -45.5 -76t-88.5 -25q-42 0 -73 14t-51 37.5t-29 53.5q-8 28 -8 58v7h387v15q0 40 -9 84q-10 52 -37.5 96
-t-73 72.5t-114.5 28.5q-53 0 -97.5 -20t-77 -56t-50.5 -85t-18 -108q2 -59 17.5 -109t46.5 -86t76.5 -56t107.5 -20q88 0 146 44t75 131z" />
-    <glyph glyph-name="ecircumflex" unicode="&#xea;" horiz-adv-x="537" 
-d="M222 731l-103 -143h77l70 96l69 -96h84l-104 143h-93zM423 308h-297q2 30 13 56.5t30 46.5t45.5 31.5t59.5 11.5q32 0 58.5 -11.5t46 -31t31 -46.5t13.5 -57zM505 164h-84q-11 -51 -45.5 -76t-88.5 -25q-42 0 -73 14t-51 37.5t-29 53.5q-8 28 -8 58v7h387v15q0 40 -9 84
-q-10 52 -37.5 96t-73 72.5t-114.5 28.5q-53 0 -97.5 -20t-77 -56t-50.5 -85t-18 -108q2 -59 17.5 -109t46.5 -86t76.5 -56t107.5 -20q88 0 146 44t75 131z" />
-    <glyph glyph-name="edieresis" unicode="&#xeb;" horiz-adv-x="537" 
-d="M130 696v-104h90v104h-90zM318 696v-104h90v104h-90zM423 308h-297q2 30 13 56.5t30 46.5t45.5 31.5t59.5 11.5q32 0 58.5 -11.5t46 -31t31 -46.5t13.5 -57zM505 164h-84q-11 -51 -45.5 -76t-88.5 -25q-42 0 -73 14t-51 37.5t-29 53.5q-8 28 -8 58v7h387v15q0 40 -9 84
-q-10 52 -37.5 96t-73 72.5t-114.5 28.5q-53 0 -97.5 -20t-77 -56t-50.5 -85t-18 -108q2 -59 17.5 -109t46.5 -86t76.5 -56t107.5 -20q88 0 146 44t75 131z" />
-    <glyph glyph-name="igrave" unicode="&#xec;" horiz-adv-x="222" 
-d="M-22 731l130 -143h64l-86 143h-108zM69 0h85v517h-85v-517z" />
-    <glyph glyph-name="iacute" unicode="&#xed;" horiz-adv-x="222" 
-d="M136 731l-86 -143h64l131 143h-109zM69 0h85v517h-85v-517z" />
-    <glyph glyph-name="icircumflex" unicode="&#xee;" horiz-adv-x="222" 
-d="M64 731l-102 -143h76l70 96l69 -96h84l-104 143h-93zM69 0h85v517h-85v-517z" />
-    <glyph glyph-name="idieresis" unicode="&#xef;" horiz-adv-x="222" 
-d="M-27 696v-104h89v104h-89zM160 696v-104h90v104h-90zM69 0h85v517h-85v-517z" />
-    <glyph glyph-name="eth" unicode="&#xf0;" horiz-adv-x="574" 
-d="M282 431q44 0 75.5 -14t51.5 -38t29.5 -55t9.5 -66q0 -47 -12.5 -83.5t-34.5 -61t-51.5 -37.5t-62.5 -13t-62.5 13t-51.5 37.5t-34.5 61t-12.5 83.5q0 77 39.5 125t116.5 48zM272 506q-57 0 -101 -20.5t-74 -55t-45.5 -79t-15.5 -93.5q0 -57 16 -106t48 -85.5t79 -57
-t108 -20.5q62 0 108.5 20.5t78.5 57t48 85.5t16 106q0 115 -41.5 211.5t-132.5 171.5l115 53l-35 37l-120 -56q-20 16 -42 31l-35 25l-70 -45q14 -8 34 -21t36 -27l-109 -51l38 -40l115 54q12 -8 28 -23t33 -35t33 -43.5t28 -49.5h-2q-27 31 -63 43.5t-76 12.5z" />
-    <glyph glyph-name="ntilde" unicode="&#xf1;" 
-d="M452 711h-52q-5 -15 -15.5 -29.5t-30.5 -14.5q-16 0 -32.5 6.5t-33.5 14t-35.5 13.5t-38.5 6q-24 0 -41 -9t-29 -24t-20.5 -35t-14.5 -42h52q-2 19 15 30.5t33 11.5q14 0 31.5 -6t35.5 -13q18 -8 35.5 -14t30.5 -6q29 0 47.5 8.5t31.5 22.5q25 35 31 80zM64 517v-517h85
-v292q0 35 9.5 64.5t28.5 51.5t47.5 34t67.5 12q49 0 77 -28t28 -76v-350h85v340q0 42 -8.5 76.5t-29.5 59.5t-55 39t-85 14q-115 0 -168 -94h-2v82h-80z" />
-    <glyph glyph-name="ograve" unicode="&#xf2;" horiz-adv-x="574" 
-d="M153 731l131 -143h64l-86 143h-109zM126 258q0 47 12.5 83.5t34.5 61.5t51.5 38t62.5 13t62.5 -13t51.5 -38t34.5 -61.5t12.5 -83.5t-12.5 -83.5t-34.5 -61t-51.5 -37.5t-62.5 -13t-62.5 13t-51.5 37.5t-34.5 61t-12.5 83.5zM36 258q0 -57 16 -106t48 -85.5t79 -57
-t108 -20.5q62 0 108.5 20.5t78.5 57t48 85.5t16 106t-16 106.5t-48 86t-78.5 57.5t-108.5 21q-61 0 -108 -21t-79 -57.5t-48 -86t-16 -106.5z" />
-    <glyph glyph-name="oacute" unicode="&#xf3;" horiz-adv-x="574" 
-d="M312 731l-86 -143h64l131 143h-109zM126 258q0 47 12.5 83.5t34.5 61.5t51.5 38t62.5 13t62.5 -13t51.5 -38t34.5 -61.5t12.5 -83.5t-12.5 -83.5t-34.5 -61t-51.5 -37.5t-62.5 -13t-62.5 13t-51.5 37.5t-34.5 61t-12.5 83.5zM36 258q0 -57 16 -106t48 -85.5t79 -57
-t108 -20.5q62 0 108.5 20.5t78.5 57t48 85.5t16 106t-16 106.5t-48 86t-78.5 57.5t-108.5 21q-61 0 -108 -21t-79 -57.5t-48 -86t-16 -106.5z" />
-    <glyph glyph-name="ocircumflex" unicode="&#xf4;" horiz-adv-x="574" 
-d="M240 731l-103 -143h77l70 96l69 -96h84l-104 143h-93zM126 258q0 47 12.5 83.5t34.5 61.5t51.5 38t62.5 13t62.5 -13t51.5 -38t34.5 -61.5t12.5 -83.5t-12.5 -83.5t-34.5 -61t-51.5 -37.5t-62.5 -13t-62.5 13t-51.5 37.5t-34.5 61t-12.5 83.5zM36 258q0 -57 16 -106
-t48 -85.5t79 -57t108 -20.5q62 0 108.5 20.5t78.5 57t48 85.5t16 106t-16 106.5t-48 86t-78.5 57.5t-108.5 21q-61 0 -108 -21t-79 -57.5t-48 -86t-16 -106.5z" />
-    <glyph glyph-name="otilde" unicode="&#xf5;" horiz-adv-x="574" 
-d="M461 711h-52q-5 -15 -15.5 -29.5t-30.5 -14.5q-16 0 -32.5 6.5t-33.5 14t-35.5 13.5t-38.5 6q-24 0 -41 -9t-29 -24t-20.5 -35t-14.5 -42h52q-2 19 15 30.5t33 11.5q14 0 31.5 -6t35.5 -13q18 -8 35.5 -14t30.5 -6q29 0 47.5 8.5t31.5 22.5q25 35 31 80zM126 258
-q0 47 12.5 83.5t34.5 61.5t51.5 38t62.5 13t62.5 -13t51.5 -38t34.5 -61.5t12.5 -83.5t-12.5 -83.5t-34.5 -61t-51.5 -37.5t-62.5 -13t-62.5 13t-51.5 37.5t-34.5 61t-12.5 83.5zM36 258q0 -57 16 -106t48 -85.5t79 -57t108 -20.5q62 0 108.5 20.5t78.5 57t48 85.5t16 106
-t-16 106.5t-48 86t-78.5 57.5t-108.5 21q-61 0 -108 -21t-79 -57.5t-48 -86t-16 -106.5z" />
-    <glyph glyph-name="odieresis" unicode="&#xf6;" horiz-adv-x="574" 
-d="M148 696v-104h90v104h-90zM336 696v-104h90v104h-90zM126 258q0 47 12.5 83.5t34.5 61.5t51.5 38t62.5 13t62.5 -13t51.5 -38t34.5 -61.5t12.5 -83.5t-12.5 -83.5t-34.5 -61t-51.5 -37.5t-62.5 -13t-62.5 13t-51.5 37.5t-34.5 61t-12.5 83.5zM36 258q0 -57 16 -106
-t48 -85.5t79 -57t108 -20.5q62 0 108.5 20.5t78.5 57t48 85.5t16 106t-16 106.5t-48 86t-78.5 57.5t-108.5 21q-61 0 -108 -21t-79 -57.5t-48 -86t-16 -106.5z" />
-    <glyph glyph-name="divide" unicode="&#xf7;" horiz-adv-x="600" 
-d="M236 47q0 -26 19.5 -44.5t45.5 -18.5q25 0 44 19.5t19 43.5q0 26 -19 45t-45 19t-45 -19t-19 -45zM236 459q0 -26 19.5 -45t45.5 -19q25 0 44 20t19 44q0 26 -19 45t-45 19t-45 -19t-19 -45zM48 287v-68h504v68h-504z" />
-    <glyph glyph-name="oslash" unicode="&#xf8;" horiz-adv-x="574" 
-d="M480 443l60 66l-27 26l-58 -65q-32 29 -73 44t-95 15q-61 0 -108 -21t-79 -57.5t-48 -86t-16 -106.5q0 -55 15 -102.5t45 -84.5l-62 -69l28 -23l60 66q31 -27 72 -41.5t93 -14.5q62 0 108.5 20.5t78.5 57t48 85.5t16 106q0 55 -14.5 102.5t-43.5 82.5zM178 108l242 269
-q13 -23 20.5 -52.5t7.5 -66.5q0 -47 -12.5 -83.5t-34.5 -61t-51.5 -37.5t-62.5 -13q-64 0 -109 45zM398 407l-242 -270q-14 25 -22 55t-8 66q0 47 12.5 83.5t34.5 61.5t51.5 38t62.5 13q67 0 111 -47z" />
-    <glyph glyph-name="ugrave" unicode="&#xf9;" 
-d="M144 731l131 -143h64l-86 143h-109zM492 0v517h-85v-292q0 -35 -9.5 -64.5t-28.5 -51.5t-47.5 -34t-67.5 -12q-49 0 -77 28t-28 76v350h-85v-340q0 -42 8.5 -76.5t29.5 -59.5t55 -38.5t85 -13.5q57 0 99 22.5t69 70.5h2v-82h80z" />
-    <glyph glyph-name="uacute" unicode="&#xfa;" 
-d="M303 731l-86 -143h64l131 143h-109zM492 0v517h-85v-292q0 -35 -9.5 -64.5t-28.5 -51.5t-47.5 -34t-67.5 -12q-49 0 -77 28t-28 76v350h-85v-340q0 -42 8.5 -76.5t29.5 -59.5t55 -38.5t85 -13.5q57 0 99 22.5t69 70.5h2v-82h80z" />
-    <glyph glyph-name="ucircumflex" unicode="&#xfb;" 
-d="M231 731l-103 -143h77l70 96l69 -96h84l-104 143h-93zM492 0v517h-85v-292q0 -35 -9.5 -64.5t-28.5 -51.5t-47.5 -34t-67.5 -12q-49 0 -77 28t-28 76v350h-85v-340q0 -42 8.5 -76.5t29.5 -59.5t55 -38.5t85 -13.5q57 0 99 22.5t69 70.5h2v-82h80z" />
-    <glyph glyph-name="udieresis" unicode="&#xfc;" 
-d="M139 696v-104h90v104h-90zM327 696v-104h90v104h-90zM492 0v517h-85v-292q0 -35 -9.5 -64.5t-28.5 -51.5t-47.5 -34t-67.5 -12q-49 0 -77 28t-28 76v350h-85v-340q0 -42 8.5 -76.5t29.5 -59.5t55 -38.5t85 -13.5q57 0 99 22.5t69 70.5h2v-82h80z" />
-    <glyph glyph-name="yacute" unicode="&#xfd;" horiz-adv-x="500" 
-d="M275 731l-86 -143h64l131 143h-109zM267 -68l225 585h-90l-145 -423h-2l-151 423h-96l205 -514l-35 -87q-12 -23 -28 -35t-42 -12q-13 0 -26 4l-24 8v-78q14 -5 29 -7t30 -2q29 0 50 7.5t38.5 24t32 42.5t29.5 64z" />
-    <glyph glyph-name="thorn" unicode="&#xfe;" horiz-adv-x="593" 
-d="M152 447v267h-85v-911h85v266h2q14 -23 34.5 -38.5t43.5 -24.5t46.5 -13t42.5 -4q59 0 103.5 21t74 57.5t44 85.5t14.5 104t-15 104t-44.5 86.5t-74 59.5t-104.5 22q-54 0 -99 -19.5t-66 -62.5h-2zM467 263q0 -38 -8 -74t-26 -64t-47.5 -45t-73.5 -17t-75 16.5
-t-50.5 43.5t-28.5 62.5t-9 73.5q0 36 8.5 71t27.5 62.5t49 44.5t73 17q41 0 71.5 -16t50 -43t29 -61.5t9.5 -70.5z" />
-    <glyph glyph-name="ydieresis" unicode="&#xff;" horiz-adv-x="500" 
-d="M111 696v-104h90v104h-90zM299 696v-104h90v104h-90zM267 -68l225 585h-90l-145 -423h-2l-151 423h-96l205 -514l-35 -87q-12 -23 -28 -35t-42 -12q-13 0 -26 4l-24 8v-78q14 -5 29 -7t30 -2q29 0 50 7.5t38.5 24t32 42.5t29.5 64z" />
-    <glyph glyph-name="OE" unicode="&#x152;" horiz-adv-x="1074" 
-d="M551 69v-69h483v80h-388v247h350v80h-350v227h382v80h-477v-59h-2q-11 20 -30.5 34t-43.5 23.5t-50.5 14t-51.5 4.5q-79 0 -141.5 -28.5t-105.5 -79t-65.5 -119t-22.5 -149.5q0 -79 21.5 -146.5t63 -117.5t102.5 -78.5t140 -28.5q27 0 54.5 5.5t52 16t44.5 26.5t33 37h2z
-M372 63q-58 0 -103 22.5t-75 61.5t-45.5 91.5t-15.5 112.5t15 114.5t45 96t75 65.5t106 24q57 0 99.5 -23t62.5 -54q11 -17 13 -35t2 -34v-301q0 -31 -14 -57t-38 -44.5t-56.5 -29t-70.5 -10.5z" />
-    <glyph glyph-name="oe" unicode="&#x153;" horiz-adv-x="891" 
-d="M415 260q0 -35 -6.5 -70t-23 -63.5t-44 -46t-70.5 -17.5q-39 0 -67 16.5t-45 43t-25 59.5t-8 66q0 38 7.5 75t24.5 66t45 47t68 18q44 0 72 -17t44 -44.5t22 -62t6 -70.5zM774 308h-285q-1 64 38 105t106 41t104 -40t37 -106zM489 233h375q1 13 1 25q0 47 -9 90
-q-12 54 -40 94t-73 63.5t-110 23.5q-55 0 -105.5 -22.5t-77.5 -68.5q-26 54 -72.5 72.5t-106.5 18.5t-104 -21t-73.5 -57.5t-43.5 -85.5t-14 -106q0 -56 14 -105t42.5 -86t73 -58t105.5 -21q58 0 106.5 22.5t72.5 78.5q24 -56 73.5 -78.5t114.5 -22.5q170 0 221 175h-81
-q-15 -48 -52 -74.5t-88 -26.5q-38 0 -66.5 14t-47 37.5t-27.5 53.5q-8 28 -8 58v7z" />
-    <glyph glyph-name="Ydieresis" unicode="&#x178;" horiz-adv-x="648" 
-d="M185 873v-104h90v104h-90zM373 873v-104h90v104h-90zM373 292l273 422h-108l-209 -340l-214 340h-113l276 -422v-292h95v292z" />
-    <glyph glyph-name="circumflex" unicode="&#x2c6;" horiz-adv-x="222" 
-d="M64 731l-102 -143h76l70 96l69 -96h84l-104 143h-93z" />
-    <glyph glyph-name="tilde" unicode="&#x2dc;" horiz-adv-x="222" 
-d="M285 711h-52q-5 -15 -15.5 -29.5t-30.5 -14.5q-16 0 -32.5 6.5t-33.5 14t-35.5 13.5t-38.5 6q-23 0 -40 -9t-29 -24t-20.5 -35t-14.5 -42h52q-2 19 14 30.5t33 11.5q14 0 31.5 -6t35.5 -13q18 -8 35.5 -14t30.5 -6q29 0 47.5 8.5t31.5 22.5q25 35 31 80z" />
-    <glyph glyph-name="uni2000" unicode="&#x2000;" horiz-adv-x="476" 
- />
-    <glyph glyph-name="uni2001" unicode="&#x2001;" horiz-adv-x="952" 
- />
-    <glyph glyph-name="uni2002" unicode="&#x2002;" horiz-adv-x="476" 
- />
-    <glyph glyph-name="uni2003" unicode="&#x2003;" horiz-adv-x="952" 
- />
-    <glyph glyph-name="uni2004" unicode="&#x2004;" horiz-adv-x="317" 
- />
-    <glyph glyph-name="uni2005" unicode="&#x2005;" horiz-adv-x="238" 
- />
-    <glyph glyph-name="uni2006" unicode="&#x2006;" horiz-adv-x="158" 
- />
-    <glyph glyph-name="uni2007" unicode="&#x2007;" horiz-adv-x="158" 
- />
-    <glyph glyph-name="uni2008" unicode="&#x2008;" horiz-adv-x="119" 
- />
-    <glyph glyph-name="uni2009" unicode="&#x2009;" horiz-adv-x="190" 
- />
-    <glyph glyph-name="uni200A" unicode="&#x200a;" horiz-adv-x="52" 
- />
-    <glyph glyph-name="uni2010" unicode="&#x2010;" horiz-adv-x="389" 
-d="M50 318v-80h289v80h-289z" />
-    <glyph glyph-name="uni2011" unicode="&#x2011;" horiz-adv-x="389" 
-d="M50 318v-80h289v80h-289z" />
-    <glyph glyph-name="endash" unicode="&#x2013;" horiz-adv-x="500" 
-d="M0 318v-80h500v80h-500z" />
-    <glyph glyph-name="emdash" unicode="&#x2014;" horiz-adv-x="1000" 
-d="M130 318v-80h740v80h-740z" />
-    <glyph glyph-name="quoteleft" unicode="&#x2018;" horiz-adv-x="278" 
-d="M195 456v111h-59q0 17 2.5 34.5t9.5 31.5t18 22.5t28 8.5v50q-28 0 -49 -12t-34.5 -32t-20 -46.5t-6.5 -56.5v-111h111z" />
-    <glyph glyph-name="quoteright" unicode="&#x2019;" horiz-adv-x="278" 
-d="M83 714v-111h59q0 -17 -2.5 -34.5t-9.5 -31.5t-18 -22.5t-28 -8.5v-50q29 0 49.5 12t34 32t20 46.5t6.5 56.5v111h-111z" />
-    <glyph glyph-name="quotesinglbase" unicode="&#x201a;" horiz-adv-x="278" 
-d="M83 111v-111h59q0 -17 -2.5 -34t-9.5 -31t-18 -22.5t-28 -8.5v-50q29 0 49.5 12t34 32t20 46.5t6.5 55.5v111h-111z" />
-    <glyph glyph-name="quotedblleft" unicode="&#x201c;" horiz-adv-x="426" 
-d="M177 456v111h-59q0 17 2.5 34.5t9.5 31.5t18 22.5t28 8.5v50q-28 0 -49 -12t-34.5 -32t-20 -46.5t-6.5 -56.5v-111h111zM360 456v111h-59q0 17 2.5 34.5t9.5 31.5t18 22.5t28 8.5v50q-28 0 -49 -12t-34.5 -32t-20 -46.5t-6.5 -56.5v-111h111z" />
-    <glyph glyph-name="quotedblright" unicode="&#x201d;" horiz-adv-x="426" 
-d="M66 714v-111h59q0 -17 -2.5 -34.5t-9.5 -31.5t-18 -22.5t-28 -8.5v-50q29 0 49.5 12t34 32t20 46.5t6.5 56.5v111h-111zM249 714v-111h59q0 -17 -2.5 -34.5t-9.5 -31.5t-18 -22.5t-28 -8.5v-50q29 0 49.5 12t34 32t20 46.5t6.5 56.5v111h-111z" />
-    <glyph glyph-name="quotedblbase" unicode="&#x201e;" horiz-adv-x="426" 
-d="M249 111v-111h59q0 -17 -2.5 -34t-9.5 -31t-18 -22.5t-28 -8.5v-50q29 0 49.5 12t34 32t20 46.5t6.5 55.5v111h-111zM66 111v-111h59q0 -17 -2.5 -34t-9.5 -31t-18 -22.5t-28 -8.5v-50q29 0 49.5 12t34 32t20 46.5t6.5 55.5v111h-111z" />
-    <glyph glyph-name="bullet" unicode="&#x2022;" horiz-adv-x="500" 
-d="M72 357q0 -37 14 -69.5t38 -57t56.5 -38.5t69.5 -14t69.5 14t57 38.5t38.5 57t14 69.5t-14 69.5t-38.5 56.5t-57 38t-69.5 14t-69.5 -14t-56.5 -38t-38 -56.5t-14 -69.5z" />
-    <glyph glyph-name="ellipsis" unicode="&#x2026;" horiz-adv-x="1000" 
-d="M777 111v-111h111v111h-111zM444 111v-111h111v111h-111zM111 111v-111h111v111h-111z" />
-    <glyph glyph-name="uni202F" unicode="&#x202f;" horiz-adv-x="190" 
- />
-    <glyph glyph-name="guilsinglleft" unicode="&#x2039;" horiz-adv-x="259" 
-d="M50 234l152 -120v86l-100 78l100 78v86l-152 -120v-88z" />
-    <glyph glyph-name="guilsinglright" unicode="&#x203a;" horiz-adv-x="259" 
-d="M209 322l-152 120v-86l100 -78l-100 -78v-86l152 120v88z" />
-    <glyph glyph-name="uni205F" unicode="&#x205f;" horiz-adv-x="238" 
- />
-    <glyph glyph-name="Euro" unicode="&#x20ac;" 
-d="M558 662q-90 63 -181 63q-118 0 -193 -81q-67 -71 -91 -194h-54l-26 -61h73q-1 -7 -1 -15v-17v-10q0 -5 1 -10h-46l-27 -61h77q42 -291 287 -291q87 0 154 43v105q-42 -39 -69 -53q-37 -19 -85 -19q-157 0 -190 215h218l26 61h-249q-1 5 -1 10v11v15q0 8 1 16h263l27 61
-h-283q17 83 56 134q49 64 124 64q73 0 143 -69z" />
-    <glyph glyph-name="trademark" unicode="&#x2122;" horiz-adv-x="990" 
-d="M927 302v412h-104l-121 -313l-123 313h-105v-412h68v344h2l135 -344h43l135 344h2v-344h68zM390 659v55h-326v-55h129v-357h68v357h129z" />
-    <hkern u1="A" u2="&#x2019;" k="74" />
-    <hkern u1="A" u2="y" k="18" />
-    <hkern u1="A" u2="w" k="18" />
-    <hkern u1="A" u2="v" k="18" />
-    <hkern u1="A" u2="Y" k="74" />
-    <hkern u1="A" u2="W" k="24" />
-    <hkern u1="A" u2="V" k="46" />
-    <hkern u1="A" u2="T" k="92" />
-    <hkern u1="F" u2="A" k="55" />
-    <hkern u1="F" u2="&#x2e;" k="157" />
-    <hkern u1="F" u2="&#x2c;" k="157" />
-    <hkern u1="L" u2="&#x2019;" k="74" />
-    <hkern u1="L" u2="y" k="37" />
-    <hkern u1="L" u2="Y" k="111" />
-    <hkern u1="L" u2="W" k="55" />
-    <hkern u1="L" u2="V" k="92" />
-    <hkern u1="L" u2="T" k="92" />
-    <hkern u1="P" u2="A" k="74" />
-    <hkern u1="P" u2="&#x2e;" k="179" />
-    <hkern u1="P" u2="&#x2c;" k="179" />
-    <hkern u1="R" u2="Y" k="37" />
-    <hkern u1="R" u2="W" k="18" />
-    <hkern u1="R" u2="V" k="18" />
-    <hkern u1="R" u2="T" k="18" />
-    <hkern u1="T" u2="y" k="111" />
-    <hkern u1="T" u2="w" k="111" />
-    <hkern u1="T" u2="u" k="92" />
-    <hkern u1="T" u2="s" k="111" />
-    <hkern u1="T" u2="r" k="92" />
-    <hkern u1="T" u2="o" k="111" />
-    <hkern u1="T" u2="e" k="111" />
-    <hkern u1="T" u2="c" k="111" />
-    <hkern u1="T" u2="a" k="111" />
-    <hkern u1="T" u2="A" k="92" />
-    <hkern u1="T" u2="&#x3b;" k="111" />
-    <hkern u1="T" u2="&#x3a;" k="111" />
-    <hkern u1="T" u2="&#x2e;" k="111" />
-    <hkern u1="T" u2="&#x2c;" k="111" />
-    <hkern u1="V" u2="y" k="18" />
-    <hkern u1="V" u2="u" k="37" />
-    <hkern u1="V" u2="r" k="37" />
-    <hkern u1="V" u2="o" k="55" />
-    <hkern u1="V" u2="i" k="18" />
-    <hkern u1="V" u2="e" k="55" />
-    <hkern u1="V" u2="a" k="55" />
-    <hkern u1="V" u2="A" k="46" />
-    <hkern u1="V" u2="&#x3b;" k="46" />
-    <hkern u1="V" u2="&#x3a;" k="46" />
-    <hkern u1="V" u2="&#x2e;" k="129" />
-    <hkern u1="V" u2="&#x2c;" k="129" />
-    <hkern u1="W" u2="u" k="18" />
-    <hkern u1="W" u2="r" k="18" />
-    <hkern u1="W" u2="o" k="57" />
-    <hkern u1="W" u2="e" k="57" />
-    <hkern u1="W" u2="a" k="37" />
-    <hkern u1="W" u2="A" k="24" />
-    <hkern u1="W" u2="&#x3b;" k="18" />
-    <hkern u1="W" u2="&#x3a;" k="18" />
-    <hkern u1="W" u2="&#x2e;" k="74" />
-    <hkern u1="W" u2="&#x2c;" k="74" />
-    <hkern u1="Y" u2="v" k="55" />
-    <hkern u1="Y" u2="u" k="74" />
-    <hkern u1="Y" u2="q" k="92" />
-    <hkern u1="Y" u2="p" k="74" />
-    <hkern u1="Y" u2="o" k="111" />
-    <hkern u1="Y" u2="i" k="37" />
-    <hkern u1="Y" u2="e" k="111" />
-    <hkern u1="Y" u2="a" k="92" />
-    <hkern u1="Y" u2="A" k="74" />
-    <hkern u1="Y" u2="&#x3b;" k="92" />
-    <hkern u1="Y" u2="&#x3a;" k="92" />
-    <hkern u1="Y" u2="&#x2e;" k="139" />
-    <hkern u1="Y" u2="&#x2c;" k="139" />
-    <hkern u1="f" u2="&#x2019;" k="-18" />
-    <hkern u1="f" u2="f" k="18" />
-    <hkern u1="r" u2="q" k="18" />
-    <hkern u1="r" u2="o" k="18" />
-    <hkern u1="r" u2="n" k="-18" />
-    <hkern u1="r" u2="g" k="28" />
-    <hkern u1="r" u2="e" k="18" />
-    <hkern u1="r" u2="d" k="18" />
-    <hkern u1="r" u2="c" k="18" />
-    <hkern u1="r" u2="&#x2e;" k="92" />
-    <hkern u1="r" u2="&#x2c;" k="92" />
-    <hkern u1="v" u2="&#x2e;" k="74" />
-    <hkern u1="v" u2="&#x2c;" k="74" />
-    <hkern u1="w" u2="&#x2e;" k="55" />
-    <hkern u1="w" u2="&#x2c;" k="55" />
-    <hkern u1="y" u2="&#x2e;" k="74" />
-    <hkern u1="y" u2="&#x2c;" k="74" />
-    <hkern u1="&#x2018;" u2="&#x2018;" k="95" />
-    <hkern u1="&#x2019;" u2="&#x2019;" k="95" />
-    <hkern u1="&#x2019;" u2="t" k="18" />
-    <hkern u1="&#x2019;" u2="s" k="74" />
-  </font>
-</defs></svg>

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/assets/fonts/helveticaneue-webfont.ttf
----------------------------------------------------------------------
diff --git a/website/assets/fonts/helveticaneue-webfont.ttf b/website/assets/fonts/helveticaneue-webfont.ttf
deleted file mode 100644
index 906f7bb..0000000
Binary files a/website/assets/fonts/helveticaneue-webfont.ttf and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/assets/fonts/helveticaneue-webfont.woff
----------------------------------------------------------------------
diff --git a/website/assets/fonts/helveticaneue-webfont.woff b/website/assets/fonts/helveticaneue-webfont.woff
deleted file mode 100644
index 839a019..0000000
Binary files a/website/assets/fonts/helveticaneue-webfont.woff and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/assets/fonts/helveticaneue_lt.css
----------------------------------------------------------------------
diff --git a/website/assets/fonts/helveticaneue_lt.css b/website/assets/fonts/helveticaneue_lt.css
deleted file mode 100644
index cb4cb73..0000000
--- a/website/assets/fonts/helveticaneue_lt.css
+++ /dev/null
@@ -1,8 +0,0 @@
-@font-face {
-    font-family: 'HelveticaNeueLight';
-    src: url('helveticaneue_lt.eot');
-    src: url('helveticaneue_lt.eot') format('embedded-opentype'),
-         url('helveticaneue_lt.woff') format('woff'),
-         url('helveticaneue_lt.ttf') format('truetype'),
-         url('helveticaneue_lt.svg#HelveticaNeueLight') format('svg');
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/assets/fonts/helveticaneue_lt.eot
----------------------------------------------------------------------
diff --git a/website/assets/fonts/helveticaneue_lt.eot b/website/assets/fonts/helveticaneue_lt.eot
deleted file mode 100644
index ecbe3aa..0000000
Binary files a/website/assets/fonts/helveticaneue_lt.eot and /dev/null differ


[17/52] [abbrv] kylin git commit: minor, update documents on jekyll installation

Posted by li...@apache.org.
minor, update documents on jekyll installation


Project: http://git-wip-us.apache.org/repos/asf/kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/b507b44f
Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/b507b44f
Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/b507b44f

Branch: refs/heads/1.x-HBase1.1.3
Commit: b507b44f25b3887489f31bfd81da26617fafbb51
Parents: 555a2e5
Author: dongli <do...@donglisMac.local>
Authored: Mon Feb 1 22:13:00 2016 +0800
Committer: dongli <do...@donglisMac.local>
Committed: Mon Feb 1 22:13:00 2016 +0800

----------------------------------------------------------------------
 website/_dev/howto_docs.md | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/b507b44f/website/_dev/howto_docs.md
----------------------------------------------------------------------
diff --git a/website/_dev/howto_docs.md b/website/_dev/howto_docs.md
index 3eca761..4850750 100644
--- a/website/_dev/howto_docs.md
+++ b/website/_dev/howto_docs.md
@@ -17,6 +17,7 @@ Install following tools before you add or edit documentation:
 2. Then, install [Jekyll](http://jekyllrb.com), and required plugins
 	* `gem install jekyll jekyll-multiple-languages kramdown rouge`  
 	* __Note__: Some specific version of jekyll and jekyll-multiple-languages does not work together (I got a "undefined method" error with jekyll 3.0.1 and jekyll-multiple-languages 2.0.3). In that case, `jekyll 2.5.3` and `jekyll-multiple-languages 1.0.8` is the known working version.
+        * eg. Use `gem install jekyll --version "=2.5.3"` to install a specific version.
 	
 3. And optionally any markdown editor you prefer
 


[39/52] [abbrv] kylin git commit: KYLIN-1416 remove website from coding branch

Posted by li...@apache.org.
http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_posts/blog/2015-01-25-introduce-data-model.md
----------------------------------------------------------------------
diff --git a/website/_posts/blog/2015-01-25-introduce-data-model.md b/website/_posts/blog/2015-01-25-introduce-data-model.md
deleted file mode 100644
index 6f23a10..0000000
--- a/website/_posts/blog/2015-01-25-introduce-data-model.md
+++ /dev/null
@@ -1,40 +0,0 @@
----
-layout: post-blog
-title:  "Introduce Data Model of Cube Designer"
-date:   2015-01-25 12:28:00
-author: Luke Han
-categories: blog
----
-
-### Background
-In previous version (before v0.6.4), Kylin introduced a GUI tool called Cube Designer for user (we called this role as __Cube Modeler__) to architect OLAP Cube with dimensions, measures and other settings. It works well for most of the features but still not user friendly yet: 
-
-1. A user has to add dimension one by one, considering there are 20+ even 50+ dimensions, the entire process is really boring. 
-2. Each dimension requires define join condition between fact table and lookup table which even already be defined in previous dimensions many times.
-3. Less validation check, especially for Hierarchy and Derived dimension, there are many exceptions in further steps which blocked many people to save the cube definition without any idea about the issue.
-4. Save/Next buttons are confusing user to click which one for real next step or just save current dimension settings
-
-### Data Model of Cube Designer
-With the feedback from our internal users and external community, we have came up one idea and would like to introduce a new concept (widely known in Data Warehouse and Business Intelligence domain): Data Model: a data model organises data elements and standardises how the data elements relate to one another.[Wikipedia](http://en.wikipedia.org/wiki/Data_model). In Kylin, it using [Star Schema](http://en.wikipedia.org/wiki/Star_schema) as Data Model, which is the simplest style of data warehouse schema. The star schema consists of a few "fact tables" (possibly only one, justifying the name) referencing any number of "dimension tables". It actually already there behind dimensions and measures and now just come to first step to define the relationship between different tables before create each dimension. 
-Now (after v0.6.4), to create a cube will follow below steps:
-
-1. Define data model first: pick up one fact table and then add other lookup tables (with their join conditions). The data mode must be presents as Star Schema.
-2. Then add dimensions, since all join conditions already presented in data model, each dimension could be more easy to create, just need to know what's kind of type: normal, hierarchy and derived (will have another blog to introduce them). There's also one helper called _Auto Generator_ to help generate many dimensions within simple clicks.
-3. Then define measures and others as previous cube designer did
-
-### Benefits
-1. A data model is very easy to communicate between different roles and teams. Most of cases it just mapping to real database table relationship, like from Hive tables
-2. More easy to create dimensions and measures based on the data model
-3. Friendly error message with enhanced validation check when save cube
-
-
-### What's Next
-After this refactor, Kylin is now be able to introduce more powerful features, the major idea is to using different storage to serve same data model:
-
-* __Logical Model__: A Data Model presents logical data structure like Star Schema beyond data tables with more business meaning
-* __Physical Model__: define how the underlying data to be stored in persistent system, like HBase. There are already two of them: MOLAP (current Kylin version) and InvertedIndex (coming with 0.7.x release). And it also easy to extend to support others without change the Logical Model.
-* A new GUI of Cube Designer to support above is on the way.
-
-
-
-

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_posts/blog/2015-06-10-release-v0.7.1-incubating.md
----------------------------------------------------------------------
diff --git a/website/_posts/blog/2015-06-10-release-v0.7.1-incubating.md b/website/_posts/blog/2015-06-10-release-v0.7.1-incubating.md
deleted file mode 100644
index f4c262a..0000000
--- a/website/_posts/blog/2015-06-10-release-v0.7.1-incubating.md
+++ /dev/null
@@ -1,149 +0,0 @@
----
-layout: post-blog
-title:  "Release v0.7.1-incubating"
-date:   2015-06-10 12:28:00
-author: Luke Han
-categories: blog
----
-
-Apache Kylin v0.7.1-incubating has rolled out on June 10, 2015. This is also the first Apache release after join incubating. 
-
-To download this release, please visit: [http://kylin.apache.org/download/](http://kylin.apache.org/download/), there are source code package, binary package, ODBC driver and installation guide avaliable.
-
-Any problem or issue, please send to Apache Kylin mailing list: [dev@kylin.apache.org](mailto:dev@kylin.apache.org)
-
-Or, report to Apache Kylin JIRA project: [https://issues.apache.org/jira/browse/KYLIN](https://issues.apache.org/jira/browse/KYLIN)
-
-__Main Changes:__
-
-* Package renamed from com.kylinolap to org.apache.kylin
-* Code cleaned up to apply Apache License policy
-* Easy install and setup with bunch of scripts and automation
-* Job engine refactor to be generic job manager for all jobs, and improved efficiency
-* Support Hive database other than 'default'
-* JDBC driver avaliable for client to interactive with Kylin server
-* Binary pacakge avaliable download 
-
-__New Feature__
-
-    * [KYLIN-327] - Binary distribution 
-    * [KYLIN-368] - Move MailService to Common module
-    * [KYLIN-540] - Data model upgrade for legacy cube descs
-    * [KYLIN-576] - Refactor expansion rate expression
-
-__Task__
-
-    * [KYLIN-361] - Rename package name with Apache Kylin
-    * [KYLIN-531] - Rename package name to org.apache.kylin
-    * [KYLIN-533] - Job Engine Refactoring
-    * [KYLIN-585] - Simplify deployment
-    * [KYLIN-586] - Add Apache License header in each source file
-    * [KYLIN-587] - Remove hard copy of javascript libraries
-    * [KYLIN-624] - Add dimension and metric info into DataModel
-    * [KYLIN-650] - Move all document from github wiki to code repository (using md file)
-    * [KYLIN-669] - Release v0.7.1 as first apache release
-
-__Sub-task__
-
-    * [KYLIN-670] - Update pom with "incubating" in version number
-    * [KYLIN-737] - Generate and sign release package for review and vote
-    * [KYLIN-795] - Release after success vote
-
-__Bug__
-
-    * [KYLIN-132] - Job framework
-    * [KYLIN-194] - Dict & ColumnValueContainer does not support number comparison, they do string comparison right now
-    * [KYLIN-220] - Enable swap column of Rowkeys in Cube Designer
-    * [KYLIN-230] - Error when create HTable
-    * [KYLIN-255] - Error when a aggregated function appear twice in select clause
-    * [KYLIN-383] - Sample Hive EDW database name should be replaced by "default" in the sample
-    * [KYLIN-399] - refreshed segment not correctly published to cube
-    * [KYLIN-412] - No exception or message when sync up table which can't access
-    * [KYLIN-421] - Hive table metadata issue
-    * [KYLIN-436] - Can't sync Hive table metadata from other database rather than "default"
-    * [KYLIN-508] - Too high cardinality is not suitable for dictionary!
-    * [KYLIN-509] - Order by on fact table not works correctly
-    * [KYLIN-517] - Always delete the last one of Add Lookup page buttom even if deleting the first join condition
-    * [KYLIN-524] - Exception will throw out if dimension is created on a lookup table, then deleting the lookup table.
-    * [KYLIN-547] - Create cube failed if column dictionary sets false and column length value greater than 0
-    * [KYLIN-556] - error tip enhance when cube detail return empty
-    * [KYLIN-570] - Need not to call API before sending login request
-    * [KYLIN-571] - Dimensions lost when creating cube though Joson Editor
-    * [KYLIN-572] - HTable size is wrong
-    * [KYLIN-581] - unable to build cube
-    * [KYLIN-583] - Dependency of Hive conf/jar in II branch will affect auto deploy
-    * [KYLIN-588] - Error when run package.sh
-    * [KYLIN-593] - angular.min.js.map and angular-resource.min.js.map are missing in kylin.war
-    * [KYLIN-594] - Making changes in build and packaging with respect to apache release process
-    * [KYLIN-595] - Kylin JDBC driver should not assume Kylin server listen on either 80 or 443
-    * [KYLIN-605] - Issue when install Kylin on a CLI which does not have yarn Resource Manager
-    * [KYLIN-614] - find hive dependency shell fine is unable to set the hive dependency correctly
-    * [KYLIN-615] - Unable add measures in Kylin web UI
-    * [KYLIN-619] - Cube build fails with hive+tez
-    * [KYLIN-620] - Wrong duration number
-    * [KYLIN-621] - SecurityException when running MR job
-    * [KYLIN-627] - Hive tables' partition column was not sync into Kylin
-    * [KYLIN-628] - Couldn't build a new created cube
-    * [KYLIN-629] - Kylin failed to run mapreduce job if there is no mapreduce.application.classpath in mapred-site.xml
-    * [KYLIN-630] - ArrayIndexOutOfBoundsException when merge cube segments 
-    * [KYLIN-638] - kylin.sh stop not working
-    * [KYLIN-639] - Get "Table 'xxxx' not found while executing SQL" error after a cube be successfully built
-    * [KYLIN-640] - sum of float not working
-    * [KYLIN-642] - Couldn't refresh cube segment
-    * [KYLIN-643] - JDBC couldn't connect to Kylin: "java.sql.SQLException: Authentication Failed"
-    * [KYLIN-644] - join table as null error when build the cube
-    * [KYLIN-652] - Lookup table alias will be set to null
-    * [KYLIN-657] - JDBC Driver not register into DriverManager
-    * [KYLIN-658] - java.lang.IllegalArgumentException: Cannot find rowkey column XXX in cube CubeDesc
-    * [KYLIN-659] - Couldn't adjust the rowkey sequence when create cube
-    * [KYLIN-666] - Select float type column got class cast exception
-    * [KYLIN-681] - Failed to build dictionary if the rowkey's dictionary property is "date(yyyy-mm-dd)"
-    * [KYLIN-682] - Got "No aggregator for func 'MIN' and return type 'decimal(19,4)'" error when build cube
-    * [KYLIN-684] - Remove holistic distinct count and multiple column distinct count from sample cube
-    * [KYLIN-691] - update tomcat download address in download-tomcat.sh
-    * [KYLIN-696] - Dictionary couldn't recognize a value and throw IllegalArgumentException: "Not a valid value"
-    * [KYLIN-703] - UT failed due to unknown host issue
-    * [KYLIN-711] - UT failure in REST module
-    * [KYLIN-739] - Dimension as metrics does not work with PK-FK derived column
-    * [KYLIN-761] - Tables are not shown in the "Query" tab, and couldn't run SQL query after cube be built
-
-__Improvement__
-
-    * [KYLIN-168] - Installation fails if multiple ZK
-    * [KYLIN-182] - Validation Rule: columns used in Join condition should have same datatype
-    * [KYLIN-204] - Kylin web not works properly in IE
-    * [KYLIN-217] - Enhance coprocessor with endpoints 
-    * [KYLIN-251] - job engine refactoring
-    * [KYLIN-261] - derived column validate when create cube
-    * [KYLIN-317] - note: grunt.json need to be configured when add new javascript or css file
-    * [KYLIN-324] - Refactor metadata to support InvertedIndex
-    * [KYLIN-407] - Validation: There's should no Hive table column using "binary" data type
-    * [KYLIN-445] - Rename cube_desc/cube folder
-    * [KYLIN-452] - Automatically create local cluster for running tests
-    * [KYLIN-498] - Merge metadata tables 
-    * [KYLIN-532] - Refactor data model in kylin front end
-    * [KYLIN-539] - use hbase command to launch tomcat
-    * [KYLIN-542] - add project property feature for cube
-    * [KYLIN-553] - From cube instance, couldn't easily find the project instance that it belongs to
-    * [KYLIN-563] - Wrap kylin start and stop with a script 
-    * [KYLIN-567] - More flexible validation of new segments
-    * [KYLIN-569] - Support increment+merge job
-    * [KYLIN-578] - add more generic configuration for ssh
-    * [KYLIN-601] - Extract content from kylin.tgz to "kylin" folder
-    * [KYLIN-616] - Validation Rule: partition date column should be in dimension columns
-    * [KYLIN-634] - Script to import sample data and cube metadata
-    * [KYLIN-636] - wiki/On-Hadoop-CLI-installation is not up to date
-    * [KYLIN-637] - add start&end date for hbase info in cubeDesigner
-    * [KYLIN-714] - Add Apache RAT to pom.xml
-    * [KYLIN-753] - Make the dependency on hbase-common to "provided"
-    * [KYLIN-758] - Updating port forwarding issue Hadoop Installation on Hortonworks Sandbox.
-    * [KYLIN-779] - [UI] jump to cube list after create cube
-    * [KYLIN-796] - Add REST API to trigger storage cleanup/GC
-
-__Wish__
-
-    * [KYLIN-608] - Distinct count for ii storage
-
-
-
-

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_posts/blog/2015-08-13-kylin-dictionary.md
----------------------------------------------------------------------
diff --git a/website/_posts/blog/2015-08-13-kylin-dictionary.md b/website/_posts/blog/2015-08-13-kylin-dictionary.md
deleted file mode 100644
index 7e3c8b8..0000000
--- a/website/_posts/blog/2015-08-13-kylin-dictionary.md
+++ /dev/null
@@ -1,68 +0,0 @@
----
-layout: post-blog
-title:  "Dictionary in Kylin"
-date:   2015-08-13 14:37:00
-author: Li Yang
-categories: blog
----
-
-### Purpose of Dictionary
-
-Dictionary is a [classic compression technique](https://en.wikipedia.org/wiki/Dictionary_coder) that can greatly reduce the size of data. Kylin apply dictionary to all dimension values stored in cube.
-
-Kylin's requirement to dictionary:
-
-* Compress cube size by storing IDs instead of real values
-* Bi-way mapping of dimension values from/to IDs
-* Preserving order to facilitate range query
-* Minimal memory & storage footprint
-
-### Dictionary Design
-
-Dictionary is implemented as a [trie](https://en.wikipedia.org/wiki/Trie) data structure. Dictionary ID (or "seq. no" below) is chosen in a way to preserve value order. Then at query time, predicate filters can be pushed down to storage and be evaluated on the IDs.
-
-* Trie node are labeled by 1) number of values underneath; 2) is end of value or not
-* Bi-way lookup between "value" <==> "seq. no" by top-down navigate
-* The "seq. no" preserves value order and is a minimal integer for space advantage
-* O(L) lookup time, where L=max(value length)
-
-An example of a trie dictionary.
-
-![](/images/develop/dictionary-trie-sample.png)
-
-### Memory structure
-
-Once built, the dictionary is serialized into a chunk of bytes. This is how it stays in memory and also in file.
-
-* Head
-   * magic, head len, body len, child_offset size, seq_no size, base ID, max value len, bytes converter
-* Body
-   * a flattened trie, where each node is
-      * child offset (size specified in head)
-         * 1st MSB: isLastChild
-         * 2nd MSB: isEndOfValue
-      * no. values beneath (size specified in head)
-      * value len (1 byte unsigned)
-      * value bytes
-
-![](/images/develop/dictionary-bytes-format.png)
-
-### Benchmark result
-
-We compared dictionary's size and performance with HashMap and ID Based Array. It's memory footprint is an order less and the throughput is very stable accross scales.
-
-|                                     | HashMap (value=>id) | Dictionary (value=>id) | IdArray (id=>value) | Dictionary (id=>value) |                      |     
-| ------------------------------------|---------------------|------------------------|---------------------|------------------------|----------------------|
-| 150K eng words footprint (bytes)    | 18.8M               | *1.7M*                 | 11.1M               | *1.7M*                 | 1.4M raw size        |
-| 150K eng words throughput (acc/s)   | 13M                 | 1.9M                   | 150M                | 1.96M                  | 31 max value len     |
-| 6.6K categories footprint (bytes)   | 0.94M               | 0.13M                  | 0.58M               | 0.12M                  | 0.1M raw size        |
-| 6.6K categories throughput (acc/s)  | 26M                 | 2.0M                   | 98M                 | 2.0M                   | 30 max value len     |
-| 6 words footprint (bytes)           | 792B                | 168B                   | 416B                | 168B                   | 33B raw size         |
-| 6 works throughput (acc/s)          | 68.5M               | 14.7M                  | 714M                | 11.1M                  | 9 max value len      |
-
-### Cache layer
-
-To achieve maximum lookup throughput, a cache layer (HashMap or IdArray) sits on top of dictionary using weak reference. The cache could be gone when memory runs short, then dictionary will be hit directly.
-
-![](/images/develop/dictionary-cache-layer.png)
-

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_posts/blog/2015-08-15-fast-cubing.md
----------------------------------------------------------------------
diff --git a/website/_posts/blog/2015-08-15-fast-cubing.md b/website/_posts/blog/2015-08-15-fast-cubing.md
deleted file mode 100644
index 5325b02..0000000
--- a/website/_posts/blog/2015-08-15-fast-cubing.md
+++ /dev/null
@@ -1,119 +0,0 @@
----
-layout: post-blog
-title:  "Fast Cubing Algorithm in Apache Kylin: Concept"
-date:   2015-08-15 12:28:00
-author: Shaofeng Shi
-categories: blog
----
-
-This article is the first post of a series document that introducing the fast cubing algorithm in Apache Kylin; The new algorithm will be released in the future release; Now it is under internal testing and pilot; All the source code has been published in Kylin's Git repository <https://github.com/apache/kylin/> on 0.8 branch. 
-
-
-## By Layer Cubing Algorithm
-Before introduce the fast cubing, I'd like to make a brief introduction on the as-is cubing algorithm, which is called "by layer cubing"; 
-
-As its name indicates, a full cube is calculated by layer: N-dimension, N-1 dimension, N-2 dimension, ... until 0 dimension; Each layer's calculation is based on its parent layer (except the first, which base on source data); So this algorithm need N rounds of MapReduce running in sequence;
-
-In the MapReduce, the key is the composite of the dimensions, the value is the composite of the measures; When the mapper reads a key-value pair, it calculates its possible child cuboids; for each child cuboid, remove 1 dimension from the key, and then output the new key and value to the reducer; 
-
-The reducer gets the values grouped by key; it aggregates the measures, and then output to HDFS; one layer's MR is finished;
-
-When all layers are finished, the cube is calculated;
-
-Figure 1 describes the flow:
-
-![]( /images/blog/by-layer-cubing.png)
-Figure 1: By layer cubing
-
-
-**Advantage**
-
-* This algorithm is clear as it mostly leverage the capability of Hadoop MapReduce; The sorting, grouping and shuffling are all taken care by MapReduce, developer just focus on the cubing logic;
-* Benefiting from Hadoop's maturity, this algorithm is very stable; In our experience there is seldom case that the mapper or reducer could fail; Even if your Hadoop cluster is small or is busy, it can finish in the end;
-
-
-**Disadvantage**
-
-* If the cube has many dimensions, it need the same number of MR jobs; As each Hadoop job scheduling need extra resource, the overhead cost to Hadoop is considerable; 
-* This algorithm causes too much shuffling to Hadoop; The mapper doesn't do aggregation, all the records that having same dimension values in next layer will be omitted to Hadoop, and then aggregated by combiner and reducer;
-* Many reads/writes on HDFS: each layer's cubing need write its output to HDFS for next layer MR to consume; In the end, Kylin need another round MR to convert these output files to HBase HFile for bulk load; These jobs generates many intermediate files in HDFS;
-* All in all: the performance is not good, especially when the cube has many dimensions; 
-
-
-
-## Fast Cubing Algorithm
-The fast cubing algorithm is also called "by segment cubing"; The core idea is, each mapper calculates the feed data block into a small cube segment (with all cuboids), and then output all key/values to reducer; The reducer aggregates them into one big cube segment, finishing the cubing; Figure 2 illustrates the flow;
-
-
-![]( /images/blog/by-segment-cubing.png)
-Figure 2: By segment cubing
-
-### Pre-aggregation in Mapper
-Compared with the origin algorithm, the fast cubing has two main differences:
-
-* The mapper will do pre-aggregation, this will reduce the number of records that the mapper output to Hadoop, and also reduce the number that reducer need to aggregate;
-* One round MR can calculate all cuboids; 
-
-Let take an example: a cube has 4 dimensions: A, B, C, D; Each mapper has 1 million source records to process; The column cardinality in the mapper is Car(A), Car(B), Car(C) and Car(D); 
-
-* When aggregate the source records to base cuboid (1111), with the old "by layer" algorithm, the mapper will output 1 million records to Hadoop; With the fast cubing algorithm, after the pre-aggregation, it only outputs the number of [distinct A, B, C, D] records to Hadoop, which is certainly smaller than source data; In a normal case, it can be 1/10 to 1/1000 of the source records size;
-
-* When aggregate from parent to a child cuboid, say from base cuboid (1111) to 3-dimension cuboid 0111, the dimension A will be aggregated; We assume the dimension A is independent with other dimensions, after aggregation, the cuboid 0111's size will be about 1/Card(A) of the base cuboid; So the output will be reduced to 1/Card(A) of the original one in this step. 
-
-Totally, assume the average cardinality of the dimensions is Card(N), the records that written from mapper to reducer can be reduced to 1/Card(N) of origin size; The less output to Hadoop, the less I/O and computing, the better performance.
-
-### Cuboid spanning tree
-Inner the mapper, there is another change in the cuboid spanning tree visiting order; In origin cubing, Kylin calculates the cuboids with Broad First Search order; In the fast cubing, it uses Depth First Search order, to reduce the cuboids that need be cached in memory.
-
-Let's look at the figure 3, which is a full spanning tree of a 4-dimension cube: before 0-Dimenson cuboid "\*" be aggregated, cuboid "ABCD", "BCD", "CD" and "D" will be cached in memory; As "\*" has no child, it will be outputted once be calculated, then comes "D"; After "C" be outputted, "CD" will be outputted as all of its children has finished; The memory for a cuboid can be released once it be outputted; The base cuboid ABCD will be the last one to output.
-
-![]( /images/blog/cube-spanning-tree.png)
-Figure 3: Cuboid spanning tree
-
-With the DFS visiting order, the output from a mapper is fully sorted (except some special cases), as the cuboid ID is at the beginning position of row key, and inner a cuboid the rows are already sorted:
-
-```
-0000
-0001[D0]
-0001[D1]
-....
-0010[C0]
-0010[C1]
-....
-0011[C0][D0]
-0011[C0][D1]
-....
-....
-1111[A0][B0][C0][D0]
-....
-```
-Since the outputs from mapper are already sorted, Hadoop's sort would be more efficient;  
-
-Besides, mapper's pre-aggregation happens in memory, this avoids unnecessary disk and network I/O, and the overhead to Hadoop is reduced; 
-
-###OutOfMemory error
-During the development phase, we encountered the OutOfMemory error in mappers; this could happen when: 
-
-	a) The mapper's JVM heap size is small;
-	b) "Distinct count" measure is used (HyperLogLog is space consuming) 
-	c) The spanning tree is too deep (too many dimensions);
-	d) The data blog feed to a mapper is too big;
-
-We realized that Kylin couldn't assume the mappers always have enough memory; The cubing algorithm need be self-adapting to various situations; A lot of efforts were put on optimizing the memory use and spilling data to disk when proactively detects an OutOfMemory error; The result is promising, the OOM error is rare to occur now;
-
-Here let's do a summary on the fast cubing algorithm;
-
-**Advantage**
-
-* It is faster than the old method; can reduce 30% to 50% overall building time from our comparison test; 
-* It produces less work load on Hadoop, and leaves less intermediate files on HDFS;
-* The cubing code can be easily reused by other cubing engines like Streaming and Spark;
-
-**Disadvantage**
-
-* The algorithm is a little complicated; This adds the maintenance effort;
-* Although the algorithm can spill data to disk automatically, it still wish the mapper has enough memory to gain best performance; User need more knowledge to tune the cubing; (this can be discussed in detail later)
-
-## Other Enhancements in Fast Cubing
-
-Together with the fast cubing algorithm, we also introduced several other enhancements in the cube build flow, like estimating the region splits by sampling, directly outputting HFile, and merging Cube over HBase Table, etc; I'd like to elaborate in the next articles, please keep posted here; If you have interest on Apache Kylin, welcome to visit our home page <http://kylin.apache.org/> and subscribe our development mailing list at <de...@kylin.apache.org>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_posts/blog/2015-09-06-release-v1.0-incubating.md
----------------------------------------------------------------------
diff --git a/website/_posts/blog/2015-09-06-release-v1.0-incubating.md b/website/_posts/blog/2015-09-06-release-v1.0-incubating.md
deleted file mode 100644
index 02025f8..0000000
--- a/website/_posts/blog/2015-09-06-release-v1.0-incubating.md
+++ /dev/null
@@ -1,44 +0,0 @@
----
-layout: post-blog
-title:  Apache Kylin v1.0 (incubating) Release Announcement
-date:   2015-09-06 17:28:00
-author: Luke Han
-categories: blog
----
-
-The Apache Kylin team is pleased to announce the release of Apache Kylin v1.0 (incubating). Apache Kylin is an open source Distributed Analytics Engine designed to provide SQL interface and multi-dimensional analysis (OLAP) on Hadoop supporting extremely large datasets.
-
-To download Apache Kylin v1.0 (incubating) visit the [download](http://kylin.apache.org/download) page.
-
-This is a major release which brings more stable, robust and well management version, Apache Kylin team resolved about 47 issues including bug fixes, improvements, and few new features.
-
-## Change Highlights
-
-__Kylin Core Improvement__
-
-* Dynamic Data Model has been added to supporting adding or removing column in data model without rebuild cube from the beginning [KYLIN-867](https://issues.apache.org/jira/browse/KYLIN-867)
-* Upgraded Apache Calcite to 1.3 for more bug fixes and new SQL functions [KYLIN-881](https://issues.apache.org/jira/browse/KYLIN-881) 
-* Cleanup job enhanced to make sure there’s no garbage files left in OS and HDFS/HBase after job build [KYLIN-926](https://issues.apache.org/jira/browse/KYLIN-926)
-* Added setting option for Hive intermediate tables created by Kylin [KYLIN-883](https://issues.apache.org/jira/browse/KYLIN-883) 
-* HBase coprocessor enhanced to imrpove query performance [KYLIN-857](https://issues.apache.org/jira/browse/KYLIN-857)
-* Kylin System Dashboard for usage, storage, performance [KYLIN-792](https://issues.apache.org/jira/browse/KYLIN-792)
-
-__Main Bug Fixes__
-
-* Can’t get source record size, especially when using other Hadoop distribution rather than HDP, like Cloudera and MapR [KYLIN-404](https://issues.apache.org/jira/browse/KYLIN-404)
-* Front-end cache cleanup issue [KYLIN-757](https://issues.apache.org/jira/browse/KYLIN-757)
-* Useless hive intermediate tables and HBase tables will be dropped after cube build/merge [KYLIN-805](https://issues.apache.org/jira/browse/KYLIN-805)
-* Support More than one HDFS files of lookup table, especially for large lookup table [KYLIN-889](https://issues.apache.org/jira/browse/KYLIN-889)
-* JDBC driver bug fixes [KYLIN-945](https://issues.apache.org/jira/browse/KYLIN-945), [KYLIN-626](https://issues.apache.org/jira/browse/KYLIN-626)
-* UI Bug fixes [KYLIN-946](https://issues.apache.org/jira/browse/KYLIN-946), [KYLIN-935](https://issues.apache.org/jira/browse/KYLIN-935)
-
-__Zeppelin Integration__
-
-[Apache Zeppelin](http://zeppelin.incubator.apache.org/) is a web-based notebook that enables interactive data analytics. The Apache Kylin team has contributed Kylin Interpreter which enables Zeppelin interaction with Kylin from notebook using ANSI SQL, this interpreter could be found from Zeppelin master code repo [here](https://github.com/apache/incubator-zeppelin/tree/master/kylin).
-
-__Upgrade__
-
-We recommend to upgrade to this version from v0.7.x or even more early version for better performance, stablility and clear one (most of the intermediate files will be cleaned up automatically). Also to keep up to date with community with latest features and supports.
-Any issue or question during upgrade, please send to Apache Kylin dev mailing list: [dev@kylin.apache.org](mailto:dev@kylin.apache.org)
-
-_Great thanks to everyone who contributed!_
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_posts/blog/2015-09-09-fast-cubing-on-spark.md
----------------------------------------------------------------------
diff --git a/website/_posts/blog/2015-09-09-fast-cubing-on-spark.md b/website/_posts/blog/2015-09-09-fast-cubing-on-spark.md
deleted file mode 100644
index c41523a..0000000
--- a/website/_posts/blog/2015-09-09-fast-cubing-on-spark.md
+++ /dev/null
@@ -1,117 +0,0 @@
----
-layout: post-blog
-title:  Fast Cubing on Spark in Apache Kylin
-date:   2015-09-09 15:28:00
-author: Qianhao Zhou
-categories: blog
----
-
-## Preparation
-
-In order to make POC phase as simple as possible, a standalone spark cluster is the best choice.
-So the environment setup is as below:
-
-1. hadoop sandbox (hortonworks hdp 2.2.0)
-	
-	(8 cores, 16G) * 1
-
-2. spark (1.4.1)
-
-	master:(4 cores, 8G)
-	
-	worker:(4 cores, 8G) * 2
-	
-The hadoop conf should also be in the SPARK_HOME/conf
-
-
-## Fast Cubing Implementation on Spark
-
-Spark as a computation framework has provided much richer operators than map-reduce. And some of them are quite suitable for the cubing algorithm, for instance **aggregate**.
-
-As the [Fast cubing algorithm](http://kylin.apache.org/blog/2015/08/15/fast-cubing/ "Fast Cubing Algorithm in Apache Kylin"), it contains several steps:
-
-1. build dictionary
-2. calculate region split for hbase
-3. build & output cuboid data
-
-----
-
-**build dictionary**
-
-In order to build dictionary, distinct values of the column are needed, which new API ***DataFrame*** has already provided(since spark 1.3.0).
-
-So after got the data from the hive through SparkSQL, it is quite natural to directly use the api to build dictionary.
-
-----
-
-**calculate region split**
-
-In order to calculate the distribution of all cuboids, Kylin use a HyperLogLog implementation. And each record will have a counter, whose size is by default 16KB each. So if the counter shuffles across the cluster, that will be very expensive.
-
-Spark has provided an operator ***aggregate*** to reduce shuffle size. It first does a map-reduce phase locally, and then another round of reduce to merge the data from each node.
-
-----
-
-**build & output cuboid data**
-
-In order to build cube, Kylin requires a small batch which can fit into memory in the same time.
-
-Previously in map-reduce implementation, Kylin leverage the life-cycle callback **cleanup** to gather all the input together as a batch. This cannot be directly applied in the map reduce operator in spark which we don't have such life-cycle callback.
-
-However spark has provided an operator ***glom*** which coalescing all elements within each partition into an array which is exactly Kylin want to build a small batch.
-
-Once the batch data is ready, we can just apply the Fast Cubing algorithm. 
-
-Then spark api ***saveAsNewAPIHadoopFile*** allow us to write hfile to hdfs and bulk load to HBase.
-
-
-## Statistics
-
-We use the sample data Kylin provided to build cube, total record count is 10000.
-
-Below are results(system environments are mentioned above)
-<table>
-    <tr>
-        <td></td>
-        <td>Spark</td>
-        <td>MR</td>
-    </tr>
-    <tr>
-        <td>Duration</td>
-        <td>5.5 min</td>
-        <td>10+ min</td>
-    </tr>
-</table>
-
-## Issues
-
-Since hdp 2.2+ requires Hive 0.14.0 while spark 1.3.0 only supports Hive 0.13.0. There are several compatibility problems in hive-site.xml we need to fix.
-
-1. some time-related settings
-
-    There are several settings, whose default value in hive 0.14.0 cannot be parsed in 0.13.0. Such as **hive.metastore.client.connect.retry.delay**, its default value is **5s**. And in hive 0.13.0, this value can only be in the format of Long value. So you have to manually change to from **5s** to **5**.
-
-2. hive.security.authorization.manager
-
-    If you have enabled this configuration, its default value is **org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdConfOnlyAuthorizerFactory** which is newly introduced in hive 0.14.0, it means you have to use the another implementation, such as **org.apache.hadoop.hive.ql.security.authorization.DefaultHiveAuthorizationProvider**
-
-3. hive.execution.engine
-
-    In hive 0.14.0, the default value of **hive.execution.engine** is **tez**, change it to **mr** in the Spark classpath, otherwise there will be NoClassDefFoundError.
-
-NOTE: Spark 1.4.0 has a [bug](https://issues.apache.org/jira/browse/SPARK-8368) which will lead to ClassNotFoundException. And it has been fixed in Spark 1.4.1. So if you are planning to run on Spark 1.4.0, you may need to upgrade to 1.4.1
-
-Last but not least, when you trying to run Spark application on YARN, make sure that you have hive-site.xml and hbase-site.xml in the  HADDOP_CONF_DIR or YARN_CONF_DIR. Since by default HDP lays these conf in separate directories.
-
-
-
-
-## Next move
-
-Clearly above is not a fair competition. The environment is not the same, test data size is too small, etc.
-
-However it showed that it is practical to migrate from MR to Spark, while some useful operators in Spark will save us quite a few codes.
-
-So the next move for us is to setup a cluster, do the benchmark on real data set for both MR and Spark.
-
-We will update the benchmark once we finished, please stay tuned.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_posts/blog/2015-09-22-hybrid-model.md
----------------------------------------------------------------------
diff --git a/website/_posts/blog/2015-09-22-hybrid-model.md b/website/_posts/blog/2015-09-22-hybrid-model.md
deleted file mode 100644
index d966fa5..0000000
--- a/website/_posts/blog/2015-09-22-hybrid-model.md
+++ /dev/null
@@ -1,136 +0,0 @@
----
-layout: post-blog
-title:  "Hybrid Model in Apache Kylin 1.0"
-date:   2015-09-25 16:00:00
-author: Shaofeng Shi
-categories: blog
----
-
-**Apache Kylin v1.0 introduces a new realization "hybrid model" (also called "dynamic model"); This post introduces the concept and how to create a hybrid instance.**
-
-# Problem
-
-For incoming SQL queries, Kylin picks one (and only one) realization to serve the query; Before the "hybrid", there is only one type of realization open for user: Cube. That to say, only 1 Cube would be selected to answer a query;
-
-Now let's start with a sample case; Assume user has a Cube called "Cube_V1", it has been built for a couple of months; Now the user wants to add new dimension or metrics to fulfill their business need; So he created a new Cube named "Cube_V2"; 
-
-Due to some reason user wants to keep "Cube_V1", and expects to build "Cube_V2" from the end date of "Cube_V1"; Possible reasons include:
-
-* History source data has been dropped from Hadoop, not possible to build "Cube_V2" from the very beginning;
-* The cube is large, rebuilding takes very long time;
-* New dimension/metrics is only available or applied since some day;
-* User feels okay that the result is empty for old days when the query uses new dimensions/metrics.
-
-For the queries against the common dimensions/metrics, user expects both "Cube_V1" and "Cube_V2" be scanned to get a full result set; Under such a background, the "hybrid model" is introduced to solve this problem.
-
-## Hybrid Model
-
-Hybrid model is a new realization which is a composite of one or multiple other realizations (cubes); See the figure below.
-
-![]( /images/blog/hybrid-model.png)
-
-Hybrid doesn't have its real storage; It is like a virtual database view over the tables; A hybrid instance acts as a delegator who forward the requests to its children realizations and then union the results when gets back from them.
-
-## How to add a hybrid instance
-
-So far there is no UI for creating/editing hybrid; if have the need, you need manually edit Kylin metadata;
-
-### Step 1: Take a backup of kylin metadata store 
-
-```
-export KYLIN_HOME="/path/to/kylin"
-
-$KYLIN_HOME/bin/metastore.sh backup
-
-```
-
-A backup folder will be created, assume it is $KYLIN_HOME/metadata_backup/2015-09-25/
- 
-### Step 2: Create sub-folder "hybrid"
-
-```
-mkdir -p $KYLIN_HOME/metadata_backup/2015-09-25/hybrid
-```
-
-### Step 3: Create a hybrid instance json file: 
-
-```
-vi $KYLIN_HOME/metadata_backup/2015-09-25/hybrid/my_hybrid.json
-
-```
-
-Input content like below, the "name" and "uuid" need be unique:
-
-```
-{
-  "uuid": "9iiu8590-64b6-4367-8fb5-7500eb95fd9c",
-  "name": "my_hybrid",
-  "realizations": [
-    {
-           "type": "CUBE",
-           "realization": "Cube_V1"
-    },
-    {
-            "type": "CUBE",
-            "realization": "Cube_V2"
-    }
-  ]
-}
-
-```
-Here "Cube_V1" and "Cube_V2" are the cubes that you want to combine.
-
-
-### Step 4: Add hybrid instance to project
-
-Open project json file (for example project "default") with text editor:
-
-```
-vi $KYLIN_HOME/metadata_backup/2015-09-25/project/default.json
-
-```
-
-In the "realizations" array, add one entry like below, the type need be "HYBRID", "realization" is the name of the hybrid instance:
-
-```
-    {
-      "name": "my_hybrid",
-      "type": "HYBRID",
-      "realization": "my_hybrid"
-    }
-```
-
-### Step 5: Upload the metadata:
-
-```
-  $KYLIN_HOME/bin/metastore.sh restore $KYLIN_HOME/metadata_backup/2015-09-25/
-
-```
-Please note, the "restore" action will upload the metadata from local to remote hbase store, which may overwrite the changes in remote; So please do this when there is no metadata change from Kylin server during this period (no building job, no cube creation/update, etc), or only pickup the changed files to an empty local folder before run "restore";
-
-### Step 6: Reload metadata
-
-Restart Kylin server, or click "Reload metadata" in the "Admin" tab on Kylin web UI to load the changes; Ideally the hybrid will start to work; You can do some verifications by writing some SQLs.
-
-## FAQ:
-
-**Question 1**: When will hybrid be selected to answer a SQL query?
-If one of its underlying cube can answer the query, the hybrid will be selected; 
-
-**Question 2**: How hybrid to answer the query?
-Hybrid will delegate the query to each of its children realizations; If a child cube is capable for this query (match all dimensions/metrics), it will return the results to the hybrid, otherwise it will be skipped; Finally query engine will aggregate the data from hybrid before return to user;
-
-**Question 3**: Will hybrid check the date/time duplication?
-No; it depends on user to ensure the cubes in a hybrid don't have date/time range duplication; For example, the "Cube_V1" is ended at 2015-9-20 (excluding), the "Cube_V2" should start from 2015-9-20 (including); 
-
-**Question 4**: Will hybrid restrict the children cubes having the same data model?
-No; To provide as much as flexibility, hybrid doesn't check whether the children cubes' fact/lookup tables and join conditions are the same; But user should understand what they're doing to avoid unexpected behavior.
-
-**Question 5**: Can hybrid have another hybrid as child?
-No; we don't see the need; so far it assumes all children are Cubes;
-
-**Question 6**: Can I use hybrid to join multiple cubes?
-No; the purpose of hybrid is to consolidate history cube and new cube, something like a "union", not "join";
-
-**Question 7**: If a child cube is disabled, will it be scanned via the hybrid?
-No; hybrid instance will check the child realization's status before sending query to it; so if the cube is disabled, it will not be scanned. 
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_posts/blog/2015-10-14-Apache-Kylin-Meetup.md
----------------------------------------------------------------------
diff --git a/website/_posts/blog/2015-10-14-Apache-Kylin-Meetup.md b/website/_posts/blog/2015-10-14-Apache-Kylin-Meetup.md
deleted file mode 100644
index 3e33da8..0000000
--- a/website/_posts/blog/2015-10-14-Apache-Kylin-Meetup.md
+++ /dev/null
@@ -1,63 +0,0 @@
----
-layout: post-blog
-title:  Apache Kylin Meetup @Shanghai Oct 10, 2015
-date:   2015-10-14 17:00:00
-author: Luke Han
-categories: blog
----
-
-{:.center}
-![]( /images/blog/meetup_1.jpeg)
-
-
-On Oct 10th, 2015, Apache Kylin Meetup was successfully held in German Centre, Shanghai. It was the first Kylin meetup held in Shanghai (the third time worldwide, the first in Silicon Valley, the second in Beijing). Mr. Chad Chun, Director of eBay ADI CCOE made a welcome speech to the meetup. After that, members of Kylin core development team, Kylin’s users in eBay, Apache Tez Committer, Apache Zeppelin Committer and technology partner of MiningLAMP shared rich topics in turn, offering a technology feast of big data to more than 100 attendees.
-
-{:.center}
-![]( /images/blog/meetup_2.jpeg)
-
-
-## 1. Apache Kylin Tech Deep Dive - Streaming and Plugin Architecture
--Yang Li, Sr. Architect eBay ADI, Apache Kylin Tech Leader, PMC member, introduced the detail design and implementation of Plugin architecture, Streaming Cubing, Fast Cubing, TopN, etc. in Kylin2 which will be released this year. Some new coming features were also mentioned.
-[Slides](http://www.slideshare.net/lukehan/1-apache-kylin-deep-dive-streaming-and-plugin-architecture-apache-kylin-meetup-shanghai)
-
-{:.center}
-![]( /images/blog/meetup_3.jpeg)
-
-## 2. How Kylin help eCG to analysis Google Analytics Data
--Naiqin Wu, Sr. PM of eBay Classifieds Group, described the difficulties and challenges encountered during the use of Google Analytics by eCG. Then she introduced how to download and organize Google Analytics Data into eBay internal Hadoop Cluster and then build analysis tools for business analysts using Kylin.
-
-{:.center}
-![]( /images/blog/meetup_4.jpeg)
-
-## 3. Apache Tez - Next Gen Execute Engine on Hadoop
--Jeff Zhang, Sr. Engineer of Hortonworks, Apache Tez committer, introduced the detail architecture and design of Apache Tez, comparison between Tez and MapReduce, community development and roadmap etc.
-[Slides](http://www.slideshare.net/lukehan/3-apache-tez-introducation-apache-kylin-meetup-shanghai)
-
-{:.center}
-![]( /images/blog/meetup_5.jpeg)
-
-## 4. Building Data Products with Apache Zeppelin
--Alexander Bezzubov, Sr. Engineer NFLabs (from Korea), Apache Zeppelin committer & PMC member, presented the method of how to leverage Zeppelin to build data product based on github open source. In the end he introduced Zeppelin’s Apache Kylin Intepreter.
-[Slides](http://www.slideshare.net/lukehan/4building-a-data-product-using-apache-zeppelin-apache-kylin-meetup-shanghai)
-
-{:.center}
-![]( /images/blog/meetup_6.jpeg)
-
-## 5. Kylin Use Case from Finance
--Hua Huang, technology partner of MiningLAMP, Apache Kylin committer, showed the result of the MiningLAMP’s recent research over big data, especially from finance field. He then introduced how MiningLAMP offer big data analysis product to its customers using Kylin. The product has been deployed and launched among its finance customers, supporting report and OLAP analysis capability based on more than 100 Kylin Cubes and dozens of TBs’ data.
-[Slides](http://www.slideshare.net/lukehan/5-apache-kylin-apache-kylin-meetup-shanghai)
-
-{:.center}
-![]( /images/blog/meetup_7.jpeg)
-
-## 6. Apache Kylin Roadmap and open source community
--Luke Han, Lead Product Manager of eBay ADI, Apache Kylin Product Lead, PMC member, introduced the development history of Apache Kylin and the future Roadmap, especially the upcoming release of StreamingOLAP and the correlated properties and release plan. The condition of community development since Kylin became open source was also referred, including newly added Committers (from eBay, Meituan, MiningLAMP), version release etc. Luke made a special mention that Apache Kylin has finished its first round voting, which means it is about to graduate from Apache Incubator and promoted as Apache top-level project soon.
-[Slides](http://www.slideshare.net/lukehan/6-apache-kylin-roadmap-and-community-apache-kylin-meetup-shanghai)
-
-{:.center}
-![]( /images/blog/meetup_8.jpeg)
-
-Finally, thanks the staffs from eBay and girls from Fudan University ;-)
-
-{:.center}
-![]( /images/blog/meetup_9.jpeg)

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_posts/blog/2015-10-25-release-v1.1-incubating.cn.md
----------------------------------------------------------------------
diff --git a/website/_posts/blog/2015-10-25-release-v1.1-incubating.cn.md b/website/_posts/blog/2015-10-25-release-v1.1-incubating.cn.md
deleted file mode 100644
index 3f799fb..0000000
--- a/website/_posts/blog/2015-10-25-release-v1.1-incubating.cn.md
+++ /dev/null
@@ -1,53 +0,0 @@
----
-layout: post-blog
-title:  Apache Kylin v1.1 (incubating) 正式发布
-date:   2015-10-25 17:28:00
-author: Luke Han
-categories: blog
----
-
-Apache Kylin社区非常高兴宣布Apache Kylin v1.1 (incubating)正式发布.
- 
-Apache Kylin一个开源的分布式分析引擎,提供Hadoop之上的SQL查询接口及多维分析(OLAP)能力以支持超大规模数据,最初由eBay Inc. 开发并贡献至开源社区。
-
-下载Apache Kylin v1.1 (incubating) 源代码及二进制安装包, 
-请访问[下载](http://kylin.apache.org/cn/download/)页面.
-
-这是一个主要的版本发布带来了更稳定,健壮及更好管理的版本,Apache Kylin社区解决了56个issue,包括Bug修复,功能增强及一些新特性等。
-
-## 主要变化
-
-__Kylin 核心功能增强__
-
-* 支持Cube数据保留时间设置 [KYLIN-906](https://issues.apache.org/jira/browse/KYLIN-906)
-* 升级Apache Calcite至1.4 [KYLIN-1047](https://issues.apache.org/jira/browse/KYLIN-1047) 
-* 在Cube构建成功后清理Hive中间文件 [KYLIN-589](https://issues.apache.org/jira/browse/KYLIN-589)
-* 当Hive返回空值时继续Cube构建任务 [KYLIN-772](https://issues.apache.org/jira/browse/KYLIN-772)
-* 支持可配置HBase压缩算法,包括Snappy及GZip [KYLIN-956](https://issues.apache.org/jira/browse/KYLIN-956) 
-* 支持将Cube数据导入到独立的HBase集群 [KYLIN-957](https://issues.apache.org/jira/browse/KYLIN-957)
-* 将Roaring bitmaps引入InvertedIndex模块 [KYLIN-1034](https://issues.apache.org/jira/browse/KYLIN-1034)
-
-__主要Bug修复__
-
-* 当有多个IN条件时SQL执行缓慢 [KYLIN-740](https://issues.apache.org/jira/browse/KYLIN-740)
-* Jobs页面加载Bug [KYLIN-950](https://issues.apache.org/jira/browse/KYLIN-950)
-* 查询缓存没有随元数据更新而刷新 [KYLIN-771](https://issues.apache.org/jira/browse/KYLIN-771)
-* 在事实表上执行“select * from fact”不工作 [KYLIN-847](https://issues.apache.org/jira/browse/KYLIN-847)
-* SQL执行时报Float 不能转换成为Double异常 [KYLIN-918](https://issues.apache.org/jira/browse/KYLIN-918)
-* 更新Cube数据模型失败后元数据状态不一致 [KYLIN-958](https://issues.apache.org/jira/browse/KYLIN-958)
-* SQL中关键字"offset"的Bug [KYLIN-983](https://issues.apache.org/jira/browse/KYLIN-983)
-* 平均值函数AVG不工作 [KYLIN-985](https://issues.apache.org/jira/browse/KYLIN-985)
-* 字典中空值''导致Cube合并失败 [KYLIN-1004](https://issues.apache.org/jira/browse/KYLIN-1004)
-
-
-__升级__  
-我们建议从v0.7.x及v1.0升级到此版本已获得更好的性能,稳定性及Bug修复等。
-并且与社区最新特性及支持保持同步。
-
-__支持__  
-升级和使用过程中有任何问题,请: 
-提交至Kylin的JIRA: [https://issues.apache.org/jira/browse/KYLIN/](https://issues.apache.org/jira/browse/KYLIN/)  
-或者  
-发送邮件到Apache Kylin邮件列表: [dev@kylin.apache.org](mailto:dev@kylin.apache.org)  
-
-_感谢各位的贡献!_
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_posts/blog/2015-10-25-release-v1.1-incubating.md
----------------------------------------------------------------------
diff --git a/website/_posts/blog/2015-10-25-release-v1.1-incubating.md b/website/_posts/blog/2015-10-25-release-v1.1-incubating.md
deleted file mode 100644
index 2468f42..0000000
--- a/website/_posts/blog/2015-10-25-release-v1.1-incubating.md
+++ /dev/null
@@ -1,53 +0,0 @@
----
-layout: post-blog
-title:  Apache Kylin v1.1 (incubating) Release Announcement
-date:   2015-10-25 17:28:00
-author: Luke Han
-categories: blog
----
-
-The Apache Kylin community is pleased to announce the release of Apache Kylin v1.1 (incubating).
- 
-Apache Kylin is an open source Distributed Analytics Engine designed to provide SQL interface and multi-dimensional analysis (OLAP) on Hadoop supporting extremely large datasets, original contributed from eBay Inc.
-
-To download Apache Kylin v1.1 (incubating) source code or binary package: 
-please visit the [download](http://kylin.apache.org/download) page.
-
-This is a major release which brings more stable, robust and well management version, Apache Kylin community resolved about 56 issues including bug fixes, improvements, and few new features.
-
-## Change Highlights
-
-__Kylin Core Improvement__
-
-* Support data retention by cube [KYLIN-906](https://issues.apache.org/jira/browse/KYLIN-906)
-* Upgraded Apache Calcite to 1.4 for more bug fixes and SQL functions [KYLIN-1047](https://issues.apache.org/jira/browse/KYLIN-1047) 
-* Cleanup intermediate Hive data after cube build [KYLIN-589](https://issues.apache.org/jira/browse/KYLIN-589)
-* Continue cube job when Hive return empty resultset [KYLIN-772](https://issues.apache.org/jira/browse/KYLIN-772)
-* Support setting for HBase compression with Snappy or GZip [KYLIN-956](https://issues.apache.org/jira/browse/KYLIN-956) 
-* Support load data to separated HBase cluster [KYLIN-957](https://issues.apache.org/jira/browse/KYLIN-957)
-* Introduced Roaring bitmaps for InvertedIndex, contributed by Daniel Lemire [KYLIN-1034](https://issues.apache.org/jira/browse/KYLIN-1034)
-
-__Main Bug Fixes__
-
-* Slowness with many IN() values [KYLIN-740](https://issues.apache.org/jira/browse/KYLIN-740)
-* Web UI "Jobs" issue [KYLIN-950](https://issues.apache.org/jira/browse/KYLIN-950)
-* Query cache is not evicted when metadata changed [KYLIN-771](https://issues.apache.org/jira/browse/KYLIN-771)
-* Select * from fact not work [KYLIN-847](https://issues.apache.org/jira/browse/KYLIN-847)
-* Float can't be cast to Double when execute SQL [KYLIN-918](https://issues.apache.org/jira/browse/KYLIN-918)
-* Update cube data model may fail and leave metadata in inconsistent state [KYLIN-958](https://issues.apache.org/jira/browse/KYLIN-958)
-* SQL keyword "offset" bug [KYLIN-983](https://issues.apache.org/jira/browse/KYLIN-983)
-* AVG not work [KYLIN-985](https://issues.apache.org/jira/browse/KYLIN-985)
-* Dictionary with '' value cause cube merge fail [KYLIN-1004](https://issues.apache.org/jira/browse/KYLIN-1004)
-
-
-__Upgrade__  
-We recommend to upgrade to this version from v0.7.x and v1.0 for better performance, stability and bug fixes.
-Also to keep up to date with community with latest features and supports.
-
-__Support__  
-Any issue or question during upgrade, please 
-open JIRA to Kylin project: [https://issues.apache.org/jira/browse/KYLIN/](https://issues.apache.org/jira/browse/KYLIN/)  
-or  
-send mail to Apache Kylin dev mailing list: [dev@kylin.apache.org](mailto:dev@kylin.apache.org)  
-
-_Great thanks to everyone who contributed!_
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_posts/blog/2015-12-23-release-v1.2.cn.md
----------------------------------------------------------------------
diff --git a/website/_posts/blog/2015-12-23-release-v1.2.cn.md b/website/_posts/blog/2015-12-23-release-v1.2.cn.md
deleted file mode 100644
index 37d12f0..0000000
--- a/website/_posts/blog/2015-12-23-release-v1.2.cn.md
+++ /dev/null
@@ -1,54 +0,0 @@
----
-layout: post-blog
-title:  Apache Kylin v1.2 正式发布
-date:   2015-12-23 22:28:00
-author: Luke Han
-categories: blog
----
-
-Apache Kylin社区非常高兴宣布Apache Kylin v1.2正式发布,这是自顺利毕业成Apache顶级项目后的第一个发布版本。
- 
-Apache Kylin一个开源的分布式分析引擎,提供Hadoop之上的SQL查询接口及多维分析(OLAP)能力以支持超大规模数据,最初由eBay Inc. 开发并贡献至开源社区。
-
-下载Apache Kylin v1.2源代码及二进制安装包, 
-请访问[下载](http://kylin.apache.org/cn/download/)页面.
-
-这是一个主要的版本发布带来了更稳定,健壮及更好管理的版本,Apache Kylin社区解决了44个issue,包括Bug修复,功能增强及一些新特性等。
-
-## 主要变化
-
-__Kylin 核心功能增强__
-
-* 支持Excel, Power BI 及 Tableau 9.1 [KYLIN-596](https://issues.apache.org/jira/browse/KYLIN-596),[KYLIN-1065](https://issues.apache.org/jira/browse/KYLIN-1065)
-* 增强HDFS小文件处理机制 [KYLIN-702](https://issues.apache.org/jira/browse/KYLIN-702) 
-* 环境检查脚本中对Hive HCatalog的增强 [KYLIN-1081](https://issues.apache.org/jira/browse/KYLIN-1081), [KYLIN-1119](https://issues.apache.org/jira/browse/KYLIN-1119)
-* 维度列字典编码支持超过千万以上基数 [KYLIN-1099](https://issues.apache.org/jira/browse/KYLIN-1099)
-* Job页面加载性能改进 [KYLIN-1154](https://issues.apache.org/jira/browse/KYLIN-1154) 
-* 基于每个查询分配内存预算 [KYLIN-1190](https://issues.apache.org/jira/browse/KYLIN-1190)
-
-__主要Bug修复__
-
-* 修复在编辑模式中保存Cube的Bug [KYLIN-1168](https://issues.apache.org/jira/browse/KYLIN-1168)
-* Cube创建后不能重命名 [KYLIN-693](https://issues.apache.org/jira/browse/KYLIN-693)
-* 项目页面中Cube列表消失 [KYLIN-930](https://issues.apache.org/jira/browse/KYLIN-930)
-* Join两个字查询时报错 [KYLIN-1033](https://issues.apache.org/jira/browse/KYLIN-1033)
-* 当过滤条件是 (A or false) 时导致错误结果 [KYLIN-1039](https://issues.apache.org/jira/browse/KYLIN-1039)
-* 支持通过ResourceManager HA环境中获取MapReduce任务状态 [KYLIN-1067](https://issues.apache.org/jira/browse/KYLIN-1067)
-* Build Base Cuboid Data出错后无法发送邮件 [KYLIN-1106](https://issues.apache.org/jira/browse/KYLIN-1106)
-* 二进制包中ResourceTool 下载/上传不工作 [KYLIN-1121](https://issues.apache.org/jira/browse/KYLIN-1121)
-* Kylin示例Cube "kylin_sales_cube"无法被保存 [KYLIN-1140](https://issues.apache.org/jira/browse/KYLIN-1140)
-* 1.x 分支中使用Minicluster的单元测试不工作 [KYLIN-1155](https://issues.apache.org/jira/browse/KYLIN-1155)
-* 在查询中无法解析如'YYYYMMDD'的日期格式 [KYLIN-1216](https://issues.apache.org/jira/browse/KYLIN-1216)
-
-
-__升级__  
-我们建议从早前颁布升级到此版本已获得更好的性能,稳定性及Bug修复等。
-并且与社区最新特性及支持保持同步。
-
-__支持__  
-升级和使用过程中有任何问题,请: 
-提交至Kylin的JIRA: [https://issues.apache.org/jira/browse/KYLIN/](https://issues.apache.org/jira/browse/KYLIN/)  
-或者  
-发送邮件到Apache Kylin邮件列表: [dev@kylin.apache.org](mailto:dev@kylin.apache.org)  
-
-_感谢每一位朋友的参与和贡献!_
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_posts/blog/2015-12-23-release-v1.2.md
----------------------------------------------------------------------
diff --git a/website/_posts/blog/2015-12-23-release-v1.2.md b/website/_posts/blog/2015-12-23-release-v1.2.md
deleted file mode 100644
index a248a18..0000000
--- a/website/_posts/blog/2015-12-23-release-v1.2.md
+++ /dev/null
@@ -1,54 +0,0 @@
----
-layout: post-blog
-title:  Apache Kylin v1.2 Release Announcement
-date:   2015-12-23 22:28:00
-author: Luke Han
-categories: blog
----
-
-The Apache Kylin community is pleased to announce the release of Apache Kylin v1.2, the first release after graduation.
- 
-Apache Kylin is an open source Distributed Analytics Engine designed to provide SQL interface and multi-dimensional analysis (OLAP) on Hadoop supporting extremely large datasets, original contributed from eBay Inc.
-
-To download Apache Kylin v1.2 source code or binary package: 
-please visit the [download](http://kylin.apache.org/download) page.
-
-This is a major release which brings more stable, robust and well management version, Apache Kylin community resolved about 44 issues including bug fixes, improvements, and few new features.
-
-## Change Highlights
-
-__Kylin Core Improvement__
-
-* Support Excel, Power BI and Tableau 9.1 [KYLIN-596](https://issues.apache.org/jira/browse/KYLIN-596),[KYLIN-1065](https://issues.apache.org/jira/browse/KYLIN-1065)
-* Improve small file management on HDFS [KYLIN-702](https://issues.apache.org/jira/browse/KYLIN-702) 
-* Env shell script enhance for Hive HCatalog [KYLIN-1081](https://issues.apache.org/jira/browse/KYLIN-1081), [KYLIN-1119](https://issues.apache.org/jira/browse/KYLIN-1119)
-* Dimenion column supports high cardinality over 10 million [KYLIN-1099](https://issues.apache.org/jira/browse/KYLIN-1099)
-* Enhance job page loading performance [KYLIN-1154](https://issues.apache.org/jira/browse/KYLIN-1154) 
-* Make memory budget per query configurable [KYLIN-1190](https://issues.apache.org/jira/browse/KYLIN-1190)
-
-__Main Bug Fixes__
-
-* Save cube issue in edit model [KYLIN-1168](https://issues.apache.org/jira/browse/KYLIN-1168)
-* Couldn't change a cube's name after it be created [KYLIN-693](https://issues.apache.org/jira/browse/KYLIN-693)
-* Cube list missing under project [KYLIN-930](https://issues.apache.org/jira/browse/KYLIN-930)
-* Error when join two sub-query [KYLIN-1033](https://issues.apache.org/jira/browse/KYLIN-1033)
-* Filter like (A or false) yields wrong result [KYLIN-1039](https://issues.apache.org/jira/browse/KYLIN-1039)
-* Support get MapReduce Job status for ResourceManager HA Env [KYLIN-1067](https://issues.apache.org/jira/browse/KYLIN-1067)
-* Can not send email caused by Build Base Cuboid Data step failed [KYLIN-1106](https://issues.apache.org/jira/browse/KYLIN-1106)
-* ResourceTool download/upload does not work in binary package [KYLIN-1121](https://issues.apache.org/jira/browse/KYLIN-1121)
-* Kylin's sample cube "kylin_sales_cube" couldn't be saved [KYLIN-1140](https://issues.apache.org/jira/browse/KYLIN-1140)
-* Unit test with minicluster doesn't work on 1.x [KYLIN-1155](https://issues.apache.org/jira/browse/KYLIN-1155)
-* Can't parse DateFormat like 'YYYYMMDD' correctly in query [KYLIN-1216](https://issues.apache.org/jira/browse/KYLIN-1216)
-
-
-__Upgrade__  
-We recommend to upgrade to this version for better performance, stability and bug fixes.
-Also to keep up to date with community with latest features and supports.
-
-__Support__  
-Any issue or question during upgrade, please 
-open JIRA to Kylin project: [https://issues.apache.org/jira/browse/KYLIN/](https://issues.apache.org/jira/browse/KYLIN/)  
-or  
-send mail to Apache Kylin dev mailing list: [dev@kylin.apache.org](mailto:dev@kylin.apache.org)  
-
-_Great thanks to everyone who contributed!_
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_posts/blog/2015-12-25-support-powerbi-tableau9.cn.md
----------------------------------------------------------------------
diff --git a/website/_posts/blog/2015-12-25-support-powerbi-tableau9.cn.md b/website/_posts/blog/2015-12-25-support-powerbi-tableau9.cn.md
deleted file mode 100644
index be26804..0000000
--- a/website/_posts/blog/2015-12-25-support-powerbi-tableau9.cn.md
+++ /dev/null
@@ -1,28 +0,0 @@
----
-layout: post-blog
-title:  Apache Kylin增加对Tableau 9及微软Excel, Power BI的支持
-date:   2015-12-25 23:23:00
-author: Luke Han
-categories: blog
----
-
-Apache Kylin社区更新了最新的ODBC Driver(v1.2),增加了对Tableau9.1、Microsoft Excel、Microsoft Power BI 等常用商业分析软件的支持。这次的更新为广大数据分析工作者带来了更多的选择
-
-### 微软Excel 及 Power BI
-Microsoft Excel是当今Windows平台上最流行的数据处理软件之一,支持多种数据处理功能,可以利用Power Query从ODBC数据源读取数据并返回到数据表中。
-
-Microsoft Power BI是由微软推出的商业智能的专业分析工具,给用户提供简单且丰富的数据可视化及分析功能。
-
-### Tableau 9
-Tableau是Windows平台上最流行的商业智能工具之一,它操作简洁、功能强大,通过简单地拖拽就可以将大量数据体现在可视化图表中,在最新的9.1版本中,用户体验得到了更进一步的提升
-
-### Kylin的支持
-从eBay及其他正在使用Kylin的用户中有很强烈的需求希望支持这几个工具,特别是每天使用这些工具的分析师们。 在Hadoop上通过其他工具,例如Hive,对大数据进行交互式分析给他们带来了巨大的挑战。通过Apache Kylin的亚秒级查询延迟特性,及这些漂亮的报表和可视化工具使得分析师能够在十亿以上规模数据上能够以秒级延迟进行交互式分析。
-
-### 教程
-迫不及待的想试试? 请参考以下教程:  
-* [微软Excel及Power BI教程](/cn/docs/tutorial/powerbi.html) 
-* [Tableau 9 教程](/cn/docs/tutorial/tableau_91.html)
-
-Enjoy!
-

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_posts/blog/2015-12-25-support-powerbi-tableau9.md
----------------------------------------------------------------------
diff --git a/website/_posts/blog/2015-12-25-support-powerbi-tableau9.md b/website/_posts/blog/2015-12-25-support-powerbi-tableau9.md
deleted file mode 100644
index e6f8fb3..0000000
--- a/website/_posts/blog/2015-12-25-support-powerbi-tableau9.md
+++ /dev/null
@@ -1,28 +0,0 @@
----
-layout: post-blog
-title:  Apache Kylin supports Tableau 9 and MS Excel, Power BI now
-date:   2015-12-25 23:23:00
-author: Luke Han
-categories: blog
----
-
-Apache Kylin Community has rolled out an updated ODBC Driver (v1.2), which enforced more support for Business Intelligence clients, including Microsoft Excel, Microsoft Power BI and Tableau 9. This update did bring more convenience and selectivity for analysts and end users.
-
-### MS Excel and Power BI
-Microsoft Excel is one of the most famous data tool on Windows platform, and has plenty of data analyzing functions. With Power Query installed as plug-in, excel can easily read data from ODBC data source and fill spreadsheets. 
-
-Microsoft Power BI is a business intelligence tool providing rich functionality and experience for data visualization and processing to user.
-
-### Tableau 9
-Tableau is one of the most famous business intelligence software on Windows platform. With simple operation and powerful functionality, users can easily drag and drop to visualize data in large scale. Besides, Tableau supports reading ODBC data sources, in a Live connection mode. In latest version 9.1, user experience gets better improved very much.
-
-### Support from Kylin community
-There are many requirements from eBay and other users who are using Kylin today, especially from analysts who using these tools everyday. The slow query performance of interactive with big data on Hadoop through other tool, like Hive, brings big chellenge for them.
-With sub-seconds query latency feature of Apache Kylin platform, analysts are able to run interactive query agaisting billions rows data in seconds, with these fancy reporting and visualization tools.
-
-### Tutorial
-Can't wait to try? Please refer to following tutorials:  
-* [MS Excel and Power BI Tutorial](/docs/tutorial/powerbi.html) 
-* [Tableau 9 Tutorial](/docs/tutorial/tableau_91.html)
-
-Enjoy!

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_sass/_base.scss
----------------------------------------------------------------------
diff --git a/website/_sass/_base.scss b/website/_sass/_base.scss
deleted file mode 100644
index 6fed5eb..0000000
--- a/website/_sass/_base.scss
+++ /dev/null
@@ -1,221 +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.
- */
-
-/**
- * Reset some basic elements
- */
-body, h1, h2, h3, h4, h5, h6,
-p, blockquote, pre, hr,
-dl, dd, ol, ul, figure {
-    margin: 0;
-    padding: 0;
-}
-
-
-
-/**
- * Basic styling
- */
-body {
-    font-family: $base-font-family;
-    font-size: $base-font-size;
-    line-height: $base-line-height;
-    font-weight: 300;
-    color: $text-color;
-    background-color: $background-color;
-    -webkit-text-size-adjust: 100%;
-}
-
-
-
-/**
- * Set `margin-bottom` to maintain vertical rhythm
- */
-h1, h2, h3, h4, h5, h6,
-p, blockquote, pre,
-ul, ol, dl, figure,
-%vertical-rhythm {
-    margin-bottom: $spacing-unit / 2;
-}
-
-
-
-/**
- * Images
- */
-img {
-    max-width: 100%;
-    vertical-align: middle;
-}
-
-
-
-/**
- * Figures
- */
-figure > img {
-    display: block;
-}
-
-figcaption {
-    font-size: $small-font-size;
-}
-
-
-
-/**
- * Lists
- */
-ul, ol {
-    margin-left: $spacing-unit;
-}
-
-li {
-    > ul,
-    > ol {
-         margin-bottom: 0;
-    }
-}
-
-
-
-/**
- * Headings
- */
-h1, h2, h3, h4, h5, h6 {
-    font-weight: 300;
-}
-
-
-
-/**
- * Links
- */
-a {
-    color: $brand-color;
-    text-decoration: none;
-
-    &:visited {
-        color: darken($brand-color, 15%);
-    }
-
-    &:hover {
-        color: $text-color;
-        text-decoration: underline;
-    }
-}
-
-
-
-/**
- * Blockquotes
- */
-blockquote {
-    color: $grey-color;
-    border-left: 4px solid $grey-color-light;
-    padding-left: $spacing-unit / 2;
-    font-size: 18px;
-    letter-spacing: -1px;
-    font-style: italic;
-
-    > :last-child {
-        margin-bottom: 0;
-    }
-}
-
-
-
-/**
- * Code formatting
- */
-pre,
-code {
-    font-size: 15px;
-    border: 1px solid $grey-color-light;
-    border-radius: 3px;
-    background-color: #eef;
-}
-
-code {
-    padding: 1px 5px;
-}
-
-pre {
-    padding: 8px 12px;
-    overflow-x: scroll;
-
-    > code {
-        border: 0;
-        padding-right: 0;
-        padding-left: 0;
-    }
-}
-
-
-
-/**
- * Wrapper
- */
-.wrapper {
-    max-width: -webkit-calc(#{$content-width} - (#{$spacing-unit} * 2));
-    max-width:         calc(#{$content-width} - (#{$spacing-unit} * 2));
-    margin-right: auto;
-    margin-left: auto;
-    padding-right: $spacing-unit;
-    padding-left: $spacing-unit;
-    @extend %clearfix;
-
-    @include media-query($on-laptop) {
-        max-width: -webkit-calc(#{$content-width} - (#{$spacing-unit}));
-        max-width:         calc(#{$content-width} - (#{$spacing-unit}));
-        padding-right: $spacing-unit / 2;
-        padding-left: $spacing-unit / 2;
-    }
-}
-
-
-
-/**
- * Clearfix
- */
-%clearfix {
-
-    &:after {
-        content: "";
-        display: table;
-        clear: both;
-    }
-}
-
-
-
-/**
- * Icons
- */
-.icon {
-
-    > svg {
-        display: inline-block;
-        width: 16px;
-        height: 16px;
-        vertical-align: middle;
-
-        path {
-            fill: $grey-color;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_sass/_layout.scss
----------------------------------------------------------------------
diff --git a/website/_sass/_layout.scss b/website/_sass/_layout.scss
deleted file mode 100644
index 01c4336..0000000
--- a/website/_sass/_layout.scss
+++ /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.
- */
-
-/**
- * Site header
- */
-.site-header {
-    border-top: 5px solid $grey-color-dark;
-    border-bottom: 1px solid $grey-color-light;
-    min-height: 56px;
-
-    // Positioning context for the mobile navigation icon
-    position: relative;
-}
-
-.site-title {
-    font-size: 26px;
-    line-height: 56px;
-    letter-spacing: -1px;
-    margin-bottom: 0;
-    float: left;
-
-    &,
-    &:visited {
-        color: $grey-color-dark;
-    }
-}
-
-.site-nav {
-    float: right;
-    line-height: 56px;
-
-    .menu-icon {
-        display: none;
-    }
-
-    .page-link {
-        color: $text-color;
-        line-height: $base-line-height;
-
-        // Gaps between nav items, but not on the first one
-        &:not(:first-child) {
-            margin-left: 20px;
-        }
-    }
-
-    @include media-query($on-palm) {
-        position: absolute;
-        top: 9px;
-        right: 30px;
-        background-color: $background-color;
-        border: 1px solid $grey-color-light;
-        border-radius: 5px;
-        text-align: right;
-
-        .menu-icon {
-            display: block;
-            float: right;
-            width: 36px;
-            height: 26px;
-            line-height: 0;
-            padding-top: 10px;
-            text-align: center;
-
-            > svg {
-                width: 18px;
-                height: 15px;
-
-                path {
-                    fill: $grey-color-dark;
-                }
-            }
-        }
-
-        .trigger {
-            clear: both;
-            display: none;
-        }
-
-        &:hover .trigger {
-            display: block;
-            padding-bottom: 5px;
-        }
-
-        .page-link {
-            display: block;
-            padding: 5px 10px;
-        }
-    }
-}
-
-
-
-/**
- * Site footer
- */
-.site-footer {
-    border-top: 1px solid $grey-color-light;
-    padding: $spacing-unit 0;
-}
-
-.footer-heading {
-    font-size: 18px;
-    margin-bottom: $spacing-unit / 2;
-}
-
-.contact-list,
-.social-media-list {
-    list-style: none;
-    margin-left: 0;
-}
-
-.footer-col-wrapper {
-    font-size: 15px;
-    color: $grey-color;
-    margin-left: -$spacing-unit / 2;
-    @extend %clearfix;
-}
-
-.footer-col {
-    float: left;
-    margin-bottom: $spacing-unit / 2;
-    padding-left: $spacing-unit / 2;
-}
-
-.footer-col-1 {
-    width: -webkit-calc(35% - (#{$spacing-unit} / 2));
-    width:         calc(35% - (#{$spacing-unit} / 2));
-}
-
-.footer-col-2 {
-    width: -webkit-calc(20% - (#{$spacing-unit} / 2));
-    width:         calc(20% - (#{$spacing-unit} / 2));
-}
-
-.footer-col-3 {
-    width: -webkit-calc(45% - (#{$spacing-unit} / 2));
-    width:         calc(45% - (#{$spacing-unit} / 2));
-}
-
-@include media-query($on-laptop) {
-    .footer-col-1,
-    .footer-col-2 {
-        width: -webkit-calc(50% - (#{$spacing-unit} / 2));
-        width:         calc(50% - (#{$spacing-unit} / 2));
-    }
-
-    .footer-col-3 {
-        width: -webkit-calc(100% - (#{$spacing-unit} / 2));
-        width:         calc(100% - (#{$spacing-unit} / 2));
-    }
-}
-
-@include media-query($on-palm) {
-    .footer-col {
-        float: none;
-        width: -webkit-calc(100% - (#{$spacing-unit} / 2));
-        width:         calc(100% - (#{$spacing-unit} / 2));
-    }
-}
-
-
-
-/**
- * Page content
- */
-.page-content {
-    padding: $spacing-unit 0;
-}
-
-.page-heading {
-    font-size: 20px;
-}
-
-.post-list {
-    margin-left: 0;
-    list-style: none;
-
-    > li {
-        margin-bottom: $spacing-unit;
-    }
-}
-
-.post-meta {
-    font-size: $small-font-size;
-    color: $grey-color;
-}
-
-.post-link {
-    display: block;
-    font-size: 24px;
-}
-
-
-
-/**
- * Posts
- */
-.post-header {
-    margin-bottom: $spacing-unit;
-}
-
-.post-title {
-    font-size: 42px;
-    letter-spacing: -1px;
-    line-height: 1;
-
-    @include media-query($on-laptop) {
-        font-size: 36px;
-    }
-}
-
-.post-content {
-    margin-bottom: $spacing-unit;
-
-    h2 {
-        font-size: 32px;
-
-        @include media-query($on-laptop) {
-            font-size: 28px;
-        }
-    }
-
-    h3 {
-        font-size: 26px;
-
-        @include media-query($on-laptop) {
-            font-size: 22px;
-        }
-    }
-
-    h4 {
-        font-size: 20px;
-
-        @include media-query($on-laptop) {
-            font-size: 18px;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/_sass/_syntax-highlighting.scss
----------------------------------------------------------------------
diff --git a/website/_sass/_syntax-highlighting.scss b/website/_sass/_syntax-highlighting.scss
deleted file mode 100644
index 44e482c..0000000
--- a/website/_sass/_syntax-highlighting.scss
+++ /dev/null
@@ -1,84 +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.
- */
-
-/**
- * Syntax highlighting styles
- */
-.highlight {
-    background: #fff;
-    @extend %vertical-rhythm;
-
-    .c     { color: #998; font-style: italic } // Comment
-    .err   { color: #a61717; background-color: #e3d2d2 } // Error
-    .k     { font-weight: bold } // Keyword
-    .o     { font-weight: bold } // Operator
-    .cm    { color: #998; font-style: italic } // Comment.Multiline
-    .cp    { color: #999; font-weight: bold } // Comment.Preproc
-    .c1    { color: #998; font-style: italic } // Comment.Single
-    .cs    { color: #999; font-weight: bold; font-style: italic } // Comment.Special
-    .gd    { color: #000; background-color: #fdd } // Generic.Deleted
-    .gd .x { color: #000; background-color: #faa } // Generic.Deleted.Specific
-    .ge    { font-style: italic } // Generic.Emph
-    .gr    { color: #a00 } // Generic.Error
-    .gh    { color: #999 } // Generic.Heading
-    .gi    { color: #000; background-color: #dfd } // Generic.Inserted
-    .gi .x { color: #000; background-color: #afa } // Generic.Inserted.Specific
-    .go    { color: #888 } // Generic.Output
-    .gp    { color: #555 } // Generic.Prompt
-    .gs    { font-weight: bold } // Generic.Strong
-    .gu    { color: #aaa } // Generic.Subheading
-    .gt    { color: #a00 } // Generic.Traceback
-    .kc    { font-weight: bold } // Keyword.Constant
-    .kd    { font-weight: bold } // Keyword.Declaration
-    .kp    { font-weight: bold } // Keyword.Pseudo
-    .kr    { font-weight: bold } // Keyword.Reserved
-    .kt    { color: #458; font-weight: bold } // Keyword.Type
-    .m     { color: #099 } // Literal.Number
-    .s     { color: #d14 } // Literal.String
-    .na    { color: #008080 } // Name.Attribute
-    .nb    { color: #0086B3 } // Name.Builtin
-    .nc    { color: #458; font-weight: bold } // Name.Class
-    .no    { color: #008080 } // Name.Constant
-    .ni    { color: #800080 } // Name.Entity
-    .ne    { color: #900; font-weight: bold } // Name.Exception
-    .nf    { color: #900; font-weight: bold } // Name.Function
-    .nn    { color: #555 } // Name.Namespace
-    .nt    { color: #000080 } // Name.Tag
-    .nv    { color: #008080 } // Name.Variable
-    .ow    { font-weight: bold } // Operator.Word
-    .w     { color: #bbb } // Text.Whitespace
-    .mf    { color: #099 } // Literal.Number.Float
-    .mh    { color: #099 } // Literal.Number.Hex
-    .mi    { color: #099 } // Literal.Number.Integer
-    .mo    { color: #099 } // Literal.Number.Oct
-    .sb    { color: #d14 } // Literal.String.Backtick
-    .sc    { color: #d14 } // Literal.String.Char
-    .sd    { color: #d14 } // Literal.String.Doc
-    .s2    { color: #d14 } // Literal.String.Double
-    .se    { color: #d14 } // Literal.String.Escape
-    .sh    { color: #d14 } // Literal.String.Heredoc
-    .si    { color: #d14 } // Literal.String.Interpol
-    .sx    { color: #d14 } // Literal.String.Other
-    .sr    { color: #009926 } // Literal.String.Regex
-    .s1    { color: #d14 } // Literal.String.Single
-    .ss    { color: #990073 } // Literal.String.Symbol
-    .bp    { color: #999 } // Name.Builtin.Pseudo
-    .vc    { color: #008080 } // Name.Variable.Class
-    .vg    { color: #008080 } // Name.Variable.Global
-    .vi    { color: #008080 } // Name.Variable.Instance
-    .il    { color: #099 } // Literal.Number.Integer.Long
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/about/index.md
----------------------------------------------------------------------
diff --git a/website/about/index.md b/website/about/index.md
deleted file mode 100644
index facf995..0000000
--- a/website/about/index.md
+++ /dev/null
@@ -1,35 +0,0 @@
----
-layout: default
-title: About
----
-
-
-<main id="main" >
-<section id="first" class="main">
-    <header style="padding:2em 0 4em 0;">
-      <div class="container" >
-        <h4 class="section-title"><span> About Kylin </span></h4>
-         <!-- second-->
-  <div class="row">
-          <div class="col-sm-12 col-md-12">
-            <div >
-            <p>Video on Youtube: <a href="http://youtu.be/xg4sm_N2_CM" target="_blank">Apache Kylin</a></p>
-            <p class="aboutkylin" style="font-size:1.2em">Apache Kylin, which is a distributed and scalable OLAP engine built on Hadoop to support
-extremely large datasets, developed and contributed by <a href="http://www.ebayinc.com/" target="_blank">eBay Inc</a> to open source community on Oct 1, 2014 and has been Apache Incubator Project since Nov 25, 2014. Refer to announcement on eBay Tech Blog for more detail: <a href="http://www.ebaytechblog.com/2014/10/20/announcing-kylin-extreme-olap-engine-for-big-data" target="_blank">Announcing Kylin: Extreme OLAP Engine for Big Data</a> </p>
-            
-           
-          </div>
-        </div>
-         </div>
-      </div>
-      <!-- /container --> 
-      
-    </header>
-  </section>
-    <!-- / section --> 
-  </div>
-  <!-- /container -->
-  
-  </header>
-  </section>
-</main>


[22/52] [abbrv] kylin git commit: KYLIN-1402 StringIndexOutOfBoundsException in Kylin Hive Column Cardinality Job

Posted by li...@apache.org.
KYLIN-1402 StringIndexOutOfBoundsException in Kylin Hive Column Cardinality Job

Project: http://git-wip-us.apache.org/repos/asf/kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/d19eb0dc
Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/d19eb0dc
Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/d19eb0dc

Branch: refs/heads/1.x-HBase1.1.3
Commit: d19eb0dc48122861077caa4b37ff604c216076c1
Parents: 85d55f4
Author: shaofengshi <sh...@apache.org>
Authored: Fri Feb 5 17:30:35 2016 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Fri Feb 5 17:30:35 2016 +0800

----------------------------------------------------------------------
 .../job/hadoop/cardinality/HiveColumnCardinalityUpdateJob.java     | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/d19eb0dc/job/src/main/java/org/apache/kylin/job/hadoop/cardinality/HiveColumnCardinalityUpdateJob.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/cardinality/HiveColumnCardinalityUpdateJob.java b/job/src/main/java/org/apache/kylin/job/hadoop/cardinality/HiveColumnCardinalityUpdateJob.java
index cad3c0b..7bd3814 100644
--- a/job/src/main/java/org/apache/kylin/job/hadoop/cardinality/HiveColumnCardinalityUpdateJob.java
+++ b/job/src/main/java/org/apache/kylin/job/hadoop/cardinality/HiveColumnCardinalityUpdateJob.java
@@ -55,7 +55,7 @@ public class HiveColumnCardinalityUpdateJob extends AbstractHadoopJob {
     @SuppressWarnings("static-access")
     protected static final Option OPTION_TABLE = OptionBuilder.withArgName("table name").hasArg().isRequired(true).withDescription("The hive table name").create("table");
 
-    private static final Logger logger = LoggerFactory.getLog(HiveColumnCardinalityUpdateJob.class);
+    private static final Logger logger = LoggerFactory.getLogger(HiveColumnCardinalityUpdateJob.class);
     private String table;
 
     public HiveColumnCardinalityUpdateJob() {


[31/52] [abbrv] kylin git commit: KYLIN-1416 remove website from coding branch

Posted by li...@apache.org.
http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/assets/fonts/opensans-light-webfont.woff
----------------------------------------------------------------------
diff --git a/website/assets/fonts/opensans-light-webfont.woff b/website/assets/fonts/opensans-light-webfont.woff
deleted file mode 100644
index ef59474..0000000
Binary files a/website/assets/fonts/opensans-light-webfont.woff and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/assets/fonts/opensans-regular-webfont.eot
----------------------------------------------------------------------
diff --git a/website/assets/fonts/opensans-regular-webfont.eot b/website/assets/fonts/opensans-regular-webfont.eot
deleted file mode 100644
index 51e112f..0000000
Binary files a/website/assets/fonts/opensans-regular-webfont.eot and /dev/null differ


[21/52] [abbrv] kylin git commit: KYLIN-1402 StringIndexOutOfBoundsException in Kylin Hive Column Cardinality Job

Posted by li...@apache.org.
KYLIN-1402 StringIndexOutOfBoundsException in Kylin Hive Column Cardinality Job

Project: http://git-wip-us.apache.org/repos/asf/kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/85d55f4f
Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/85d55f4f
Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/85d55f4f

Branch: refs/heads/1.x-HBase1.1.3
Commit: 85d55f4f8e4de440c7764cd1820917ddbef3836b
Parents: 474dbfa
Author: shaofengshi <sh...@apache.org>
Authored: Fri Feb 5 16:35:25 2016 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Fri Feb 5 16:35:25 2016 +0800

----------------------------------------------------------------------
 .../HiveColumnCardinalityUpdateJob.java         | 44 +++++++++++---------
 1 file changed, 25 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/85d55f4f/job/src/main/java/org/apache/kylin/job/hadoop/cardinality/HiveColumnCardinalityUpdateJob.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/cardinality/HiveColumnCardinalityUpdateJob.java b/job/src/main/java/org/apache/kylin/job/hadoop/cardinality/HiveColumnCardinalityUpdateJob.java
index 115753e..cad3c0b 100644
--- a/job/src/main/java/org/apache/kylin/job/hadoop/cardinality/HiveColumnCardinalityUpdateJob.java
+++ b/job/src/main/java/org/apache/kylin/job/hadoop/cardinality/HiveColumnCardinalityUpdateJob.java
@@ -18,14 +18,6 @@
 
 package org.apache.kylin.job.hadoop.cardinality;
 
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.StringWriter;
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-
 import org.apache.commons.cli.Option;
 import org.apache.commons.cli.OptionBuilder;
 import org.apache.commons.cli.Options;
@@ -41,11 +33,21 @@ import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.job.hadoop.AbstractHadoopJob;
 import org.apache.kylin.metadata.MetadataConstants;
 import org.apache.kylin.metadata.MetadataManager;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.StringWriter;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
 
 /**
  * This job will update save the cardinality result into Kylin table metadata store.
- * @author shaoshi
  *
+ * @author shaoshi
  */
 public class HiveColumnCardinalityUpdateJob extends AbstractHadoopJob {
     public static final String JOB_TITLE = "Kylin Hive Column Cardinality Update Job";
@@ -53,6 +55,7 @@ public class HiveColumnCardinalityUpdateJob extends AbstractHadoopJob {
     @SuppressWarnings("static-access")
     protected static final Option OPTION_TABLE = OptionBuilder.withArgName("table name").hasArg().isRequired(true).withDescription("The hive table name").create("table");
 
+    private static final Logger logger = LoggerFactory.getLog(HiveColumnCardinalityUpdateJob.class);
     private String table;
 
     public HiveColumnCardinalityUpdateJob() {
@@ -73,7 +76,7 @@ public class HiveColumnCardinalityUpdateJob extends AbstractHadoopJob {
             this.table = getOptionValue(OPTION_TABLE).toUpperCase();
             // start job
             String jobName = JOB_TITLE + getOptionsAsString();
-            System.out.println("Starting: " + jobName);
+            logger.info("Starting: " + jobName);
             Configuration conf = getConf();
             Path output = new Path(getOptionValue(OPTION_OUTPUT_PATH));
 
@@ -91,30 +94,33 @@ public class HiveColumnCardinalityUpdateJob extends AbstractHadoopJob {
         try {
             columns = readLines(new Path(outPath), config);
         } catch (Exception e) {
-            e.printStackTrace();
-            System.out.println("Failed to resolve cardinality for " + tableName + " from " + outPath);
+            logger.error("Failed to resolve cardinality for " + tableName + " from " + outPath, e);
             return;
         }
 
         StringBuffer cardi = new StringBuffer();
         Iterator<String> it = columns.iterator();
         while (it.hasNext()) {
-            String string = (String) it.next();
+            String string = it.next();
             String[] ss = StringUtils.split(string, "\t");
 
             if (ss.length != 2) {
-                System.out.println("The hadoop cardinality value is not valid " + string);
+                logger.info("The hadoop cardinality value is not valid " + string);
                 continue;
             }
             cardi.append(ss[1]);
             cardi.append(",");
         }
         String scardi = cardi.toString();
-        scardi = scardi.substring(0, scardi.length() - 1);
-        MetadataManager metaMgr = MetadataManager.getInstance(KylinConfig.getInstanceFromEnv());
-        Map<String, String> tableExd = metaMgr.getTableDescExd(tableName);
-        tableExd.put(MetadataConstants.TABLE_EXD_CARDINALITY, scardi);
-        metaMgr.saveTableExd(tableName.toUpperCase(), tableExd);
+        if (scardi.length() > 0) {
+            scardi = scardi.substring(0, scardi.length() - 1);
+            MetadataManager metaMgr = MetadataManager.getInstance(KylinConfig.getInstanceFromEnv());
+            Map<String, String> tableExd = metaMgr.getTableDescExd(tableName);
+            tableExd.put(MetadataConstants.TABLE_EXD_CARDINALITY, scardi);
+            metaMgr.saveTableExd(tableName.toUpperCase(), tableExd);
+        } else {
+            throw new IllegalArgumentException("No cardinality data is collected for table " + tableName);
+        }
     }
 
     private static List<String> readLines(Path location, Configuration conf) throws Exception {


[27/52] [abbrv] kylin git commit: KYLIN-1416 remove website from coding branch

Posted by li...@apache.org.
http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/assets/js/main.js
----------------------------------------------------------------------
diff --git a/website/assets/js/main.js b/website/assets/js/main.js
deleted file mode 100644
index df30c43..0000000
--- a/website/assets/js/main.js
+++ /dev/null
@@ -1,74 +0,0 @@
-jQuery(document).ready(function ($) {
-	
-	
-  
-	
-
-    //parallax effect
-    var parallax_animation = function () {
-        $('.parallax').each(function (i, obj) {
-            var speed = $(this).attr('parallax-speed');
-            if (speed) {
-                var background_pos = '-' + (window.pageYOffset / speed) + "px";
-                $(this).css('background-position', 'center ' + background_pos);
-            }
-        });
-    }
-
-
-
-
-   // page elements animation 
-    var image_animation = function () {
-        //var diagramTop = $('#diagram').offset().top;
-		var coremTop = $('#core').offset().top;
-
-        
-        var scroll_top = $(window).scrollTop();
-        var currentPosition = scroll_top + 320;
-
-        
-
-        if (coremTop< currentPosition) {
-
-            $('#core').addClass("animated fadeInRight");
-        } else {
-            $('#core').removeClass("animated fadeInRight");
-        }
-
-    }
-	
-	
-	
-	
-	//document page 
-	$( "#content-container" ).load("docs/What-should-I-use-Kylin-for.md");
-	$( "#left-menu li" ).eq(0).css("background-color", "#efefef");
-
-    $( "#left-menu li" ).click(function(){
-		var selectedID = $(this).attr("id");
-		$("#content-container").load( "docs/"+selectedID+"-content.html", function() { $(this).fadeIn(500);});
-		$(this).css("background-color", "#efefef");
-		$(this).siblings().css("background-color", "transparent")
-		});
- 
-  $('#nav-wrapper').height($("#nav").height());
-    
-    $('#nav').affix({
-        offset: { top: $('#nav').offset().top }
-    });
-	
-
-
-
-    $(document).scroll(function () {
-        
-        parallax_animation();
-		image_animation();
-       
-
-    });
-
-
-
-});
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/assets/js/nnav.js
----------------------------------------------------------------------
diff --git a/website/assets/js/nnav.js b/website/assets/js/nnav.js
deleted file mode 100644
index 3554118..0000000
--- a/website/assets/js/nnav.js
+++ /dev/null
@@ -1,16 +0,0 @@
-$(function() {
-	var myNav = $("#bs-example-navbar-collapse-1 a"),i;
-	for(i=0;i<myNav.length;i++) {
-		var links = myNav.eq(i).attr("href"), myURL = document.URL;
-		if (links != "/" & links != "/cn") {
-			if(myURL.indexOf(links) != -1) {
-				myNav.eq(i).attr('class','nnav');
-				myNav.eq(0).attr('class','nnav-home');
-				break;
-			}
-		}
-		else if (links == "/" | links == "/cn")  {
-			myNav.eq(0).attr('class','nnav');
-		}
-	}
-})

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/assets/js/nside.js
----------------------------------------------------------------------
diff --git a/website/assets/js/nside.js b/website/assets/js/nside.js
deleted file mode 100644
index 35ec49e..0000000
--- a/website/assets/js/nside.js
+++ /dev/null
@@ -1,13 +0,0 @@
-$(function() {
-	var myNav1 = $("#nside1 .list-group"),i,j;
-	for(i=0;i<myNav1.length;i++) {
-		var myid = myNav1.eq(i).attr("id");
-		var myNav2 = $("#" + myid + " a");
-		for(j=0;j<myNav2.length;j++) {
-			var links = myNav2.eq(j).attr("href"), myURL = document.URL;
-			if(myURL.indexOf(links) != -1) {
-				myNav2.eq(j).attr('class','nside');
-			}
-		}
-	}
-})

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/blog/index.md
----------------------------------------------------------------------
diff --git a/website/blog/index.md b/website/blog/index.md
deleted file mode 100644
index 0129cec..0000000
--- a/website/blog/index.md
+++ /dev/null
@@ -1,39 +0,0 @@
----
-layout: default
-title: Blog
----
-
-<main id="main" >
-<section id="first" class="main">
-    <header style="padding:2em 0 4em 0;">
-      <div class="container" >
-        <h4 class="section-title"><span>Apache Kylin™ Technical Blog </span></h4>
-         <!-- second-->
-          <div id="content-container" class="animated fadeIn">
-            <div>
-             <ul class="post-list">
-            {% for category in site.categories %}     <!-- categories -->
-            {% if category[0]  == 'blog' %}
-            {% for post in category[1] %}
-            <li>
-        <h2 align="left" style="margin:0px">
-          <a class="post-link" href="{{ post.url | prepend: site.baseurl }}">{{ post.title }}</a></h2><div align="left" class="post-meta" >posted: {{ post.date | date: "%b %-d, %Y" }}</div>
-        
-      </li>
-    {% endfor %}
-    {% endif %}
-    {% endfor %}
-  </ul>
-
-  <p class="rss-subscribe">subscribe <a href="{{ "/feed.xml" | prepend: site.baseurl }}">via RSS</a></p>
-          </div>
-        </div>
-      </div>
-      <!-- /container --> 
-      
-    </header>
-  </section>
-
-  
-    
-</main>

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/community/index.md
----------------------------------------------------------------------
diff --git a/website/community/index.md b/website/community/index.md
deleted file mode 100644
index 700f2f9..0000000
--- a/website/community/index.md
+++ /dev/null
@@ -1,60 +0,0 @@
----
-layout: default
-title: Community
-permalink: /community/index.html
----
-
-### Powered By Apache Kylin™
-For information about who are using Apache Kylin™, please refer to [Powered By](/community/poweredby.html) page.
-
-
-### Apache Kylin Mailing List
-
-These are the mailing lists that have been established for this project. For each list, there is a subscribe, unsubscribe, and an archive link.
-
-| Name  | Subscribe | Unsubscribe | Post | Archive |
-|------ |-----------|-------------|------|---------|
-| User Mailing List | [Subscribe](mailto:user-subscribe@kylin.apache.org) | [Unsubscribe](mailto:user-unsubscribe@kylin.apache.org) | [Post](mailto:user@kylin.apache.org) | [mail-archives.apache.org](http://mail-archives.apache.org/mod_mbox/kylin-user/) |
-| Developers Mailing List | [Subscribe](mailto:dev-subscribe@kylin.apache.org) | [Unsubscribe](mailto:dev-unsubscribe@kylin.apache.org) | [Post](mailto:dev@kylin.apache.org) | [mail-archives.apache.org](http://mail-archives.apache.org/mod_mbox/kylin-dev/) |
-| Issues Mailing List | [Subscribe](mailto:issues-subscribe@kylin.apache.org) | [Unsubscribe](mailto:issues-unsubscribe@kylin.apache.org) | N/A | [mail-archives.apache.org](http://mail-archives.apache.org/mod_mbox/kylin-issues/) |
-| Commits Mailing List | [Subscribe](mailto:commits-subscribe@kylin.apache.org) | [Unsubscribe](mailto:commits-unsubscribe@kylin.apache.org) | N/A | [mail-archives.apache.org](http://mail-archives.apache.org/mod_mbox/kylin-commits/) |
-
-### Mailing List Archives
-For convenience, there's a forum style mailing list archives which not part of offical Apache archives:
-
-* [Developer List archive on Nabble](http://apache-kylin.74782.x6.nabble.com)
-
-### Social Media 
-The official Kylin Twitter account: [@ApacheKylin](https://twitter.com/ApacheKylin)
-
-## Apache Kylin Team
-A successful project requires many people to play many roles. Some members write code, provide project mentorship, or author documentation. Others are valuable as testers, submitting patches and suggestions.
-
-### PMC Members
-
-| Name  | Apache ID    | Github    |  Role |
-|:----- |:-------------|:----------|:------|
-{% for c in site.data.contributors %}  | {{ c.name }} | <a href="http://people.apache.org/committer-index#{{ c.apacheId }}">{{ c.apacheId }}</a> | <a href="http://github.com/{{ c.githubId }}"><img width="48" src="{% unless c.avatar %}http://github.com/{{ c.githubId }}.png{% else %}{{ c.avatar }}{% endunless %}"></a> |  {{ c.role }} |
-{% endfor %}
-
-### Contributors
-Contributors has commited code could be found [here](https://github.com/apache/kylin/graphs/contributors).
-__Other contributors__
-
-| Name  | Github    |   |
-|:----- |:----------|:------|
-|Rui Feng | [fengrui129](https://github.com/fengrui129) | Website Design, Kylin Logo|
-|Luffy Xiao | [luffy-xiao](http://github.com/luffy-xiao) | Kylin Web application, REST service |
-|Kejia Wang |  [Kejia-Wang](https://github.com/Kejia-Wang)  | Web aplication, Website|
-|Yue Yang |  | Web aplication UI design |
-
-### Credits
-
-* Thanks [eBay Inc.](https://www.ebayinc.com/) to donated this project to open source community, first announement at [eBay Techblog](http://www.ebaytechblog.com/2014/10/20/announcing-kylin-extreme-olap-engine-for-big-data/).  
-* Thanks [JetBrains](https://www.jetbrains.com/) for providing us a free license of [IntelliJ IDEA](https://www.jetbrains.com/idea/).
-* Thanks to [Vikash Agarwal](vikash_agarwal@hotmail.com), his artical __[ODBC Driver Development](http://www.drdobbs.com/windows/odbc-driver-development/184416434?pgno=5)__ and sample code introdued the basic idea about how to write an ODBC driver from scratch.
-
-
-
-
-

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/community/poweredby.md
----------------------------------------------------------------------
diff --git a/website/community/poweredby.md b/website/community/poweredby.md
deleted file mode 100644
index 2dcd70d..0000000
--- a/website/community/poweredby.md
+++ /dev/null
@@ -1,33 +0,0 @@
----
-layout: default
-title: PoweredBy
-permalink: /community/poweredby.html
----		
-
-## Powered By Apache Kylin™
-
-__How to be listed here?__
-Send a quick description of your organization and usage to the [mailing list](mailto:user@kylin.apache.org) or to [@apachekylin](https://twitter.com/apachekylin) or [@lukehq](https://twitter.com/lukehq) on twitter and we'll add you.
-
-__Companies & Organizations__
-
-* [eBay](http://www.ebay.com)  (_NASDAQ: EBAY_)
-    * Apache Kylin is used at eBay for Big Data Analytics on Hadoop. This powers various data products including Behavior Analytics, Traffic Reporting, Account Manager Application and Streaming Dashboard.
-* [JD.com, Inc.](http://www.jd.com)  (_NASDAQ: JD_)
-    * Apache Kylin as Data Analytics Engine to analysis [JOS](http://jos.jd.com) API access behavior and report in [JCloud](http://www.jcloud.com).
-* [MiningLAMP](http://www.mininglamp.com)
-    * Apache Kylin is part of MiningLAMP's Big Data Analytics product offering OLAP capability for customers from Banking, Finance and others industuies.
-* [Meituan.com](http://www.meituan.com) (_[CrunchBase:Meituan.com](https://www.crunchbase.com/organization/meituan-com)_)
-    * Apache Kylin is powering various OLAP scenarios at Meituan's Data Platform, such as traffic analysis and payment channel analysis. It's quite stable and super fast. We are very willing to try Kylin on other product as well.
-* [VIP.com](http://www.vip.com)  (_NYSE: VIPS_)
-    * Apache Kylin is used at VIP.com’s big data self-service analysis platform. Apache Kylin gives us a new option to make user experience more ad-hoc friendly and give us the opportunity to introduce Cube back to Big Data.
-* [DreamSoft](http://www.dream-it.cn/)
-    * DreamSoft is using Apache Kylin as Big Data Analytics Engine for their customers coming from Retail, Manufacturing, Finance and others.
-* [Exponential](http://www.exponential.com)
-	* Using Kylin as a component in it's data platform LAMBDA to power inventory, campaign, behavior and demand analysis for advertising. It has been great at handling the size and speed constraints of old gen systems. With the first version already in production, we are looking forward to pushing more data into kylin and achieve near real time updates. 
-* [Baidu Map](http://map.baidu.com/)  (_NASDAQ: BIDU_)
-	* Apache Kylin is used for analysis Baidu Map's huge amounts of data about users, channels and etc, it performs quite fast with its own pre-calculation and index technologies along with the cubes.
-* [NetEase](http://www.163.com/)  (_NASDAQ: NTES_)
-	* Apache Kylin has been introduced as an OLAP engine for the data platform at NetEase, powering data analysis for various products & business with great performance.
-
-    

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/css/main.scss
----------------------------------------------------------------------
diff --git a/website/css/main.scss b/website/css/main.scss
deleted file mode 100755
index beee4e3..0000000
--- a/website/css/main.scss
+++ /dev/null
@@ -1,52 +0,0 @@
----
-# Only the main Sass file needs front matter (the dashes are enough)
----
-@charset "utf-8";
-
-
-
-// Our variables
-$base-font-family: Helvetica, Arial, sans-serif;
-$base-font-size:   16px;
-$small-font-size:  $base-font-size * 0.875;
-$base-line-height: 1.5;
-
-$spacing-unit:     30px;
-
-$text-color:       #111;
-$background-color: #fdfdfd;
-$brand-color:      #2a7ae2;
-
-$grey-color:       #828282;
-$grey-color-light: lighten($grey-color, 40%);
-$grey-color-dark:  darken($grey-color, 25%);
-
-// Width of the content area
-$content-width:    800px;
-
-$on-palm:          600px;
-$on-laptop:        800px;
-
-
-
-// Using media queries with like this:
-// @include media-query($on-palm) {
-//     .wrapper {
-//         padding-right: $spacing-unit / 2;
-//         padding-left: $spacing-unit / 2;
-//     }
-// }
-@mixin media-query($device) {
-    @media screen and (max-width: $device) {
-        @content;
-    }
-}
-
-
-
-// Import partials from `sass_dir` (defaults to `_sass`)
-@import
-        "base",
-        "layout",
-        "syntax-highlighting"
-;

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/download/index.cn.md
----------------------------------------------------------------------
diff --git a/website/download/index.cn.md b/website/download/index.cn.md
deleted file mode 100644
index 913d358..0000000
--- a/website/download/index.cn.md
+++ /dev/null
@@ -1,40 +0,0 @@
----
-layout: download-cn
-title: 下载
----
-
-__最新发布(源代码)__  
-最新发布的Apache Kylin™可以从ASF网站下载::
-
-* [Apache Kylin v1.2](http://www.apache.org/dyn/closer.cgi/kylin/apache-kylin-1.2/)
-* [发布日志](http://kylin.apache.org/docs/release_notes.html)
-* Git 标签: [kylin-1.2](https://github.com/apache/kylin/tree/kylin-1.2)
-* Git Commit: [c2589aee4ac5537b460b3b02fa89cdb3a922d64e](https://github.com/apache/kylin/commit/c2589aee4ac5537b460b3b02fa89cdb3a922d64e)
-
-__二进制包 for HBase 0.98/0.99__
-为方便使用,我们提供预打包的二进制安装包:
-
-  * [apache-kylin-1.2-bin.tar.gz](https://dist.apache.org/repos/dist/release/kylin/apache-kylin-1.2/apache-kylin-1.2-bin.tar.gz)
-  * [安装帮助](http://kylin.apache.org/docs/install)
-
-__二进制包 for HBase 1.1.3及更高版本__
-越来越多的用户开始部署使用HBase 1.1或更高版本,我们提供一个在HBase 1.1上编译的Kylin快照二进制包;
-请注意此安装包需要HBase 1.1.3或更高版本,之前版本中有一个已知的关于fuzzy key过滤器的缺陷,会导致Kylin查询结果缺少记录: [HBASE-14269](https://issues.apache.org/jira/browse/HBASE-14269)
-此外请注意,这不是一个正式的发布版,没有经过完整的测试。
-
-  * [apache-kylin-1.3-HBase-1.1-SNAPSHOT-bin.tar.gz](https://dist.apache.org/repos/dist/dev/kylin/apache-kylin-1.3-snapshot/apache-kylin-1.3-HBase-1.1-SNAPSHOT-bin.tar.gz)
-  * Git commit [a3b8eb04955310abec158ea30f61deb0119679d1](https://github.com/apache/kylin/commit/a3b8eb04955310abec158ea30f61deb0119679d1) 
-
-对于HBase 1.0用户,建议您升级到1.1.3或降级到0.98/0.99.
-
-__构建二进制包__
-可以从各个版本或当前最新的开发分支中生成二进制包,请参考这篇[帮助文档](https://kylin.apache.org/development/howto_package.html)
-
-__以前的版本__
-Apache Kylin的旧版本可以从[归档](https://archive.apache.org/dist/kylin/)中下载。
-
-__ODBC 驱动__  
-Kylin ODBC 驱动要求首先安装[Microsoft Visual C++ 2012 Redistributable]()。 
-Kylin ODBC 驱动可以从这里下载:
-
-  * [Kylin ODBC 驱动 v1.2](http://kylin.apache.org/download/KylinODBCDriver-1.2.zip)

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/download/index.md
----------------------------------------------------------------------
diff --git a/website/download/index.md b/website/download/index.md
deleted file mode 100644
index d5ab443..0000000
--- a/website/download/index.md
+++ /dev/null
@@ -1,44 +0,0 @@
----
-layout: download
-title: Download
-permalink: /download/index.html
----
-
-__Latest Release(Source Code)__  
-The latest release of Apache Kylin™ can be downloaded from the ASF:
-
-* [Apache Kylin v1.2](http://www.apache.org/dyn/closer.cgi/kylin/apache-kylin-1.2/)
-* [Release Notes](http://kylin.apache.org/docs/release_notes.html)
-* Git Tag: [kylin-1.2](https://github.com/apache/kylin/tree/kylin-1.2)
-* Git Commit: [c2589aee4ac5537b460b3b02fa89cdb3a922d64e](https://github.com/apache/kylin/commit/c2589aee4ac5537b460b3b02fa89cdb3a922d64e)
-
-__Binary Package (for running on HBase 0.98/0.99)__
-For convenience, there’s binary package also available: 
-
-* [apache-kylin-1.2-bin.tar.gz](https://dist.apache.org/repos/dist/release/kylin/apache-kylin-1.2/apache-kylin-1.2-bin.tar.gz)
-* [Installation Guide](http://kylin.apache.org/docs/install)
-
-__Binary Package (for running on HBase 1.1.3 or above)__
-As there are more and more HBase 1.1 deployments, an binary snapshot build for HBase 1.1.3+ is provided; 
-Note the requirement of HBase version 1.1.3 (or above). There is a known bug in HBase earlier versions about fuzzy key filter that will cause
-missing rows or lesser aggregations in Kylin query result: [HBASE-14269](https://issues.apache.org/jira/browse/HBASE-14269)
-Also, please aware this is not a formal release, and it is not fully tested:
-
-* [apache-kylin-1.3-HBase-1.1-SNAPSHOT-bin.tar.gz](https://dist.apache.org/repos/dist/dev/kylin/apache-kylin-1.3-snapshot/apache-kylin-1.3-HBase-1.1-SNAPSHOT-bin.tar.gz)
-* Git commit [a3b8eb04955310abec158ea30f61deb0119679d1](https://github.com/apache/kylin/commit/a3b8eb04955310abec158ea30f61deb0119679d1) 
-
-If you're using HBase 1.0, we suggest you to upgrade to 1.1.3+ or downgrade to 0.98/0.99.
-
-__Build Binary Package__
-To build binary package from any version even latest development branch, please refer to this [guide](https://kylin.apache.org/development/howto_package.html)
-
-__Previous Release__  
- Older releases may be found in the [archives](https://archive.apache.org/dist/kylin/).
-    
-__ODBC Driver__  
-Kylin ODBC driver requires [Microsoft Visual C++ 2012 Redistributable](http://www.microsoft.com/en-us/download/details.aspx?id=30679) installed first. 
-And Kylin ODBC Driver could be downloaded here: 
-
-* [Kylin ODBC Driver v1.2](http://kylin.apache.org/download/KylinODBCDriver-1.2.zip)
-
-

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/fav.png
----------------------------------------------------------------------
diff --git a/website/fav.png b/website/fav.png
deleted file mode 100644
index 821bdd2..0000000
Binary files a/website/fav.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/favicon.ico
----------------------------------------------------------------------
diff --git a/website/favicon.ico b/website/favicon.ico
deleted file mode 100644
index 9485685..0000000
Binary files a/website/favicon.ico and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/feed.xml
----------------------------------------------------------------------
diff --git a/website/feed.xml b/website/feed.xml
deleted file mode 100644
index d2d4a6e..0000000
--- a/website/feed.xml
+++ /dev/null
@@ -1,44 +0,0 @@
----
-layout: null
----
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-  Licensed 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. See accompanying LICENSE file.
--->
-
-<rss version="2.0" xmlns:atom="http://www.w3.org/2005/Atom">
-  <channel>
-    <title>{{ site.title | xml_escape }}</title>
-    <description>{{ site.description | xml_escape }}</description>
-    <link>{{ site.url }}{{ site.baseurl }}/</link>
-    <atom:link href="{{ "/feed.xml" | prepend: site.baseurl | prepend: site.url }}" rel="self" type="application/rss+xml"/>
-    <pubDate>{{ site.time | date_to_rfc822 }}</pubDate>
-    <lastBuildDate>{{ site.time | date_to_rfc822 }}</lastBuildDate>
-    <generator>Jekyll v{{ jekyll.version }}</generator>
-    {% for post in site.posts limit:10 %}
-      <item>
-        <title>{{ post.title | xml_escape }}</title>
-        <description>{{ post.content | xml_escape }}</description>
-        <pubDate>{{ post.date | date_to_rfc822 }}</pubDate>
-        <link>{{ post.url | prepend: site.baseurl | prepend: site.url }}</link>
-        <guid isPermaLink="true">{{ post.url | prepend: site.baseurl | prepend: site.url }}</guid>
-        {% for tag in post.tags %}
-        <category>{{ tag | xml_escape }}</category>
-        {% endfor %}
-        {% for cat in post.categories %}
-        <category>{{ cat | xml_escape }}</category>
-        {% endfor %}
-      </item>
-    {% endfor %}
-  </channel>
-</rss>

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/google0f781a4ad9402c21.html
----------------------------------------------------------------------
diff --git a/website/google0f781a4ad9402c21.html b/website/google0f781a4ad9402c21.html
deleted file mode 100644
index 1bdaf0b..0000000
--- a/website/google0f781a4ad9402c21.html
+++ /dev/null
@@ -1,19 +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.
--->
-
-google-site-verification: google0f781a4ad9402c21.html
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/Kylin-Cube-Build-and-Job-Monitoring-Tutorial/1 action-build.png
----------------------------------------------------------------------
diff --git a/website/images/Kylin-Cube-Build-and-Job-Monitoring-Tutorial/1 action-build.png b/website/images/Kylin-Cube-Build-and-Job-Monitoring-Tutorial/1 action-build.png
deleted file mode 100644
index 136b3de..0000000
Binary files a/website/images/Kylin-Cube-Build-and-Job-Monitoring-Tutorial/1 action-build.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/Kylin-Cube-Build-and-Job-Monitoring-Tutorial/2 pop-up.png
----------------------------------------------------------------------
diff --git a/website/images/Kylin-Cube-Build-and-Job-Monitoring-Tutorial/2 pop-up.png b/website/images/Kylin-Cube-Build-and-Job-Monitoring-Tutorial/2 pop-up.png
deleted file mode 100644
index 8bae515..0000000
Binary files a/website/images/Kylin-Cube-Build-and-Job-Monitoring-Tutorial/2 pop-up.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/Kylin-Cube-Build-and-Job-Monitoring-Tutorial/3 end-date.png
----------------------------------------------------------------------
diff --git a/website/images/Kylin-Cube-Build-and-Job-Monitoring-Tutorial/3 end-date.png b/website/images/Kylin-Cube-Build-and-Job-Monitoring-Tutorial/3 end-date.png
deleted file mode 100644
index e102bd8..0000000
Binary files a/website/images/Kylin-Cube-Build-and-Job-Monitoring-Tutorial/3 end-date.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/Kylin-Cube-Build-and-Job-Monitoring-Tutorial/4 submit.png
----------------------------------------------------------------------
diff --git a/website/images/Kylin-Cube-Build-and-Job-Monitoring-Tutorial/4 submit.png b/website/images/Kylin-Cube-Build-and-Job-Monitoring-Tutorial/4 submit.png
deleted file mode 100644
index 3dc0c7e..0000000
Binary files a/website/images/Kylin-Cube-Build-and-Job-Monitoring-Tutorial/4 submit.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/Kylin-Cube-Build-and-Job-Monitoring-Tutorial/4.1 success.png
----------------------------------------------------------------------
diff --git a/website/images/Kylin-Cube-Build-and-Job-Monitoring-Tutorial/4.1 success.png b/website/images/Kylin-Cube-Build-and-Job-Monitoring-Tutorial/4.1 success.png
deleted file mode 100644
index 00a944c..0000000
Binary files a/website/images/Kylin-Cube-Build-and-Job-Monitoring-Tutorial/4.1 success.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/Kylin-Cube-Build-and-Job-Monitoring-Tutorial/5 jobs-page.png
----------------------------------------------------------------------
diff --git a/website/images/Kylin-Cube-Build-and-Job-Monitoring-Tutorial/5 jobs-page.png b/website/images/Kylin-Cube-Build-and-Job-Monitoring-Tutorial/5 jobs-page.png
deleted file mode 100644
index 175d231..0000000
Binary files a/website/images/Kylin-Cube-Build-and-Job-Monitoring-Tutorial/5 jobs-page.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/Kylin-Cube-Build-and-Job-Monitoring-Tutorial/6 discard.png
----------------------------------------------------------------------
diff --git a/website/images/Kylin-Cube-Build-and-Job-Monitoring-Tutorial/6 discard.png b/website/images/Kylin-Cube-Build-and-Job-Monitoring-Tutorial/6 discard.png
deleted file mode 100644
index 792d1b6..0000000
Binary files a/website/images/Kylin-Cube-Build-and-Job-Monitoring-Tutorial/6 discard.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/Kylin-Cube-Build-and-Job-Monitoring-Tutorial/7 job-steps.png
----------------------------------------------------------------------
diff --git a/website/images/Kylin-Cube-Build-and-Job-Monitoring-Tutorial/7 job-steps.png b/website/images/Kylin-Cube-Build-and-Job-Monitoring-Tutorial/7 job-steps.png
deleted file mode 100644
index 664b097..0000000
Binary files a/website/images/Kylin-Cube-Build-and-Job-Monitoring-Tutorial/7 job-steps.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/Kylin-Cube-Build-and-Job-Monitoring-Tutorial/8 hover-step.png
----------------------------------------------------------------------
diff --git a/website/images/Kylin-Cube-Build-and-Job-Monitoring-Tutorial/8 hover-step.png b/website/images/Kylin-Cube-Build-and-Job-Monitoring-Tutorial/8 hover-step.png
deleted file mode 100644
index ffd6750..0000000
Binary files a/website/images/Kylin-Cube-Build-and-Job-Monitoring-Tutorial/8 hover-step.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/Kylin-Cube-Build-and-Job-Monitoring-Tutorial/9 log-d.png
----------------------------------------------------------------------
diff --git a/website/images/Kylin-Cube-Build-and-Job-Monitoring-Tutorial/9 log-d.png b/website/images/Kylin-Cube-Build-and-Job-Monitoring-Tutorial/9 log-d.png
deleted file mode 100644
index dae85c9..0000000
Binary files a/website/images/Kylin-Cube-Build-and-Job-Monitoring-Tutorial/9 log-d.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/Kylin-Cube-Build-and-Job-Monitoring-Tutorial/9 log.png
----------------------------------------------------------------------
diff --git a/website/images/Kylin-Cube-Build-and-Job-Monitoring-Tutorial/9 log.png b/website/images/Kylin-Cube-Build-and-Job-Monitoring-Tutorial/9 log.png
deleted file mode 100644
index b844ad4..0000000
Binary files a/website/images/Kylin-Cube-Build-and-Job-Monitoring-Tutorial/9 log.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/Kylin-Cube-Build-and-Job-Monitoring-Tutorial/9 mrjob-d.png
----------------------------------------------------------------------
diff --git a/website/images/Kylin-Cube-Build-and-Job-Monitoring-Tutorial/9 mrjob-d.png b/website/images/Kylin-Cube-Build-and-Job-Monitoring-Tutorial/9 mrjob-d.png
deleted file mode 100644
index e57ae2e..0000000
Binary files a/website/images/Kylin-Cube-Build-and-Job-Monitoring-Tutorial/9 mrjob-d.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/Kylin-Cube-Build-and-Job-Monitoring-Tutorial/9 mrjob.png
----------------------------------------------------------------------
diff --git a/website/images/Kylin-Cube-Build-and-Job-Monitoring-Tutorial/9 mrjob.png b/website/images/Kylin-Cube-Build-and-Job-Monitoring-Tutorial/9 mrjob.png
deleted file mode 100644
index 2621695..0000000
Binary files a/website/images/Kylin-Cube-Build-and-Job-Monitoring-Tutorial/9 mrjob.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/Kylin-Cube-Build-and-Job-Monitoring-Tutorial/9 parameters-d.png
----------------------------------------------------------------------
diff --git a/website/images/Kylin-Cube-Build-and-Job-Monitoring-Tutorial/9 parameters-d.png b/website/images/Kylin-Cube-Build-and-Job-Monitoring-Tutorial/9 parameters-d.png
deleted file mode 100644
index f4fd6d4..0000000
Binary files a/website/images/Kylin-Cube-Build-and-Job-Monitoring-Tutorial/9 parameters-d.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/Kylin-Cube-Build-and-Job-Monitoring-Tutorial/9 parameters.png
----------------------------------------------------------------------
diff --git a/website/images/Kylin-Cube-Build-and-Job-Monitoring-Tutorial/9 parameters.png b/website/images/Kylin-Cube-Build-and-Job-Monitoring-Tutorial/9 parameters.png
deleted file mode 100644
index 1b6053f..0000000
Binary files a/website/images/Kylin-Cube-Build-and-Job-Monitoring-Tutorial/9 parameters.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/Kylin-Cube-Creation-Tutorial/1 manage-prject.png
----------------------------------------------------------------------
diff --git a/website/images/Kylin-Cube-Creation-Tutorial/1 manage-prject.png b/website/images/Kylin-Cube-Creation-Tutorial/1 manage-prject.png
deleted file mode 100644
index 0127f7a..0000000
Binary files a/website/images/Kylin-Cube-Creation-Tutorial/1 manage-prject.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/Kylin-Cube-Creation-Tutorial/10 filter.png
----------------------------------------------------------------------
diff --git a/website/images/Kylin-Cube-Creation-Tutorial/10 filter.png b/website/images/Kylin-Cube-Creation-Tutorial/10 filter.png
deleted file mode 100644
index 6ae086b..0000000
Binary files a/website/images/Kylin-Cube-Creation-Tutorial/10 filter.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/Kylin-Cube-Creation-Tutorial/11 refresh-setting1.png
----------------------------------------------------------------------
diff --git a/website/images/Kylin-Cube-Creation-Tutorial/11 refresh-setting1.png b/website/images/Kylin-Cube-Creation-Tutorial/11 refresh-setting1.png
deleted file mode 100644
index 5fedf87..0000000
Binary files a/website/images/Kylin-Cube-Creation-Tutorial/11 refresh-setting1.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/Kylin-Cube-Creation-Tutorial/11 refresh-setting2.png
----------------------------------------------------------------------
diff --git a/website/images/Kylin-Cube-Creation-Tutorial/11 refresh-setting2.png b/website/images/Kylin-Cube-Creation-Tutorial/11 refresh-setting2.png
deleted file mode 100644
index 8f2dd0e..0000000
Binary files a/website/images/Kylin-Cube-Creation-Tutorial/11 refresh-setting2.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/Kylin-Cube-Creation-Tutorial/12 advanced.png
----------------------------------------------------------------------
diff --git a/website/images/Kylin-Cube-Creation-Tutorial/12 advanced.png b/website/images/Kylin-Cube-Creation-Tutorial/12 advanced.png
deleted file mode 100644
index 7663ac3..0000000
Binary files a/website/images/Kylin-Cube-Creation-Tutorial/12 advanced.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/Kylin-Cube-Creation-Tutorial/13 overview.png
----------------------------------------------------------------------
diff --git a/website/images/Kylin-Cube-Creation-Tutorial/13 overview.png b/website/images/Kylin-Cube-Creation-Tutorial/13 overview.png
deleted file mode 100644
index a86c269..0000000
Binary files a/website/images/Kylin-Cube-Creation-Tutorial/13 overview.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/Kylin-Cube-Creation-Tutorial/2 +project.png
----------------------------------------------------------------------
diff --git a/website/images/Kylin-Cube-Creation-Tutorial/2 +project.png b/website/images/Kylin-Cube-Creation-Tutorial/2 +project.png
deleted file mode 100644
index 8b36313..0000000
Binary files a/website/images/Kylin-Cube-Creation-Tutorial/2 +project.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/Kylin-Cube-Creation-Tutorial/3 new-project.png
----------------------------------------------------------------------
diff --git a/website/images/Kylin-Cube-Creation-Tutorial/3 new-project.png b/website/images/Kylin-Cube-Creation-Tutorial/3 new-project.png
deleted file mode 100644
index dcb574b..0000000
Binary files a/website/images/Kylin-Cube-Creation-Tutorial/3 new-project.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/Kylin-Cube-Creation-Tutorial/3.1 pj-created.png
----------------------------------------------------------------------
diff --git a/website/images/Kylin-Cube-Creation-Tutorial/3.1 pj-created.png b/website/images/Kylin-Cube-Creation-Tutorial/3.1 pj-created.png
deleted file mode 100644
index aabaea8..0000000
Binary files a/website/images/Kylin-Cube-Creation-Tutorial/3.1 pj-created.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/Kylin-Cube-Creation-Tutorial/4 +table.png
----------------------------------------------------------------------
diff --git a/website/images/Kylin-Cube-Creation-Tutorial/4 +table.png b/website/images/Kylin-Cube-Creation-Tutorial/4 +table.png
deleted file mode 100644
index 5e8d896..0000000
Binary files a/website/images/Kylin-Cube-Creation-Tutorial/4 +table.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/Kylin-Cube-Creation-Tutorial/5 hive-table.png
----------------------------------------------------------------------
diff --git a/website/images/Kylin-Cube-Creation-Tutorial/5 hive-table.png b/website/images/Kylin-Cube-Creation-Tutorial/5 hive-table.png
deleted file mode 100644
index 7b4d67a..0000000
Binary files a/website/images/Kylin-Cube-Creation-Tutorial/5 hive-table.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/Kylin-Cube-Creation-Tutorial/6 +cube.png
----------------------------------------------------------------------
diff --git a/website/images/Kylin-Cube-Creation-Tutorial/6 +cube.png b/website/images/Kylin-Cube-Creation-Tutorial/6 +cube.png
deleted file mode 100644
index b5e3cc9..0000000
Binary files a/website/images/Kylin-Cube-Creation-Tutorial/6 +cube.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/Kylin-Cube-Creation-Tutorial/7 cube-info.png
----------------------------------------------------------------------
diff --git a/website/images/Kylin-Cube-Creation-Tutorial/7 cube-info.png b/website/images/Kylin-Cube-Creation-Tutorial/7 cube-info.png
deleted file mode 100644
index 51bbc0e..0000000
Binary files a/website/images/Kylin-Cube-Creation-Tutorial/7 cube-info.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/Kylin-Cube-Creation-Tutorial/8 dim-+dim.png
----------------------------------------------------------------------
diff --git a/website/images/Kylin-Cube-Creation-Tutorial/8 dim-+dim.png b/website/images/Kylin-Cube-Creation-Tutorial/8 dim-+dim.png
deleted file mode 100644
index f4b835e..0000000
Binary files a/website/images/Kylin-Cube-Creation-Tutorial/8 dim-+dim.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/Kylin-Cube-Creation-Tutorial/8 dim-edit.png
----------------------------------------------------------------------
diff --git a/website/images/Kylin-Cube-Creation-Tutorial/8 dim-edit.png b/website/images/Kylin-Cube-Creation-Tutorial/8 dim-edit.png
deleted file mode 100644
index 6fe3846..0000000
Binary files a/website/images/Kylin-Cube-Creation-Tutorial/8 dim-edit.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/Kylin-Cube-Creation-Tutorial/8 dim-factable.png
----------------------------------------------------------------------
diff --git a/website/images/Kylin-Cube-Creation-Tutorial/8 dim-factable.png b/website/images/Kylin-Cube-Creation-Tutorial/8 dim-factable.png
deleted file mode 100644
index 6994f11..0000000
Binary files a/website/images/Kylin-Cube-Creation-Tutorial/8 dim-factable.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/Kylin-Cube-Creation-Tutorial/8 dim-typeA.png
----------------------------------------------------------------------
diff --git a/website/images/Kylin-Cube-Creation-Tutorial/8 dim-typeA.png b/website/images/Kylin-Cube-Creation-Tutorial/8 dim-typeA.png
deleted file mode 100644
index f7c042f..0000000
Binary files a/website/images/Kylin-Cube-Creation-Tutorial/8 dim-typeA.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/Kylin-Cube-Creation-Tutorial/8 dim-typeB-1.png
----------------------------------------------------------------------
diff --git a/website/images/Kylin-Cube-Creation-Tutorial/8 dim-typeB-1.png b/website/images/Kylin-Cube-Creation-Tutorial/8 dim-typeB-1.png
deleted file mode 100644
index bb19bb9..0000000
Binary files a/website/images/Kylin-Cube-Creation-Tutorial/8 dim-typeB-1.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/Kylin-Cube-Creation-Tutorial/8 dim-typeB-2.png
----------------------------------------------------------------------
diff --git a/website/images/Kylin-Cube-Creation-Tutorial/8 dim-typeB-2.png b/website/images/Kylin-Cube-Creation-Tutorial/8 dim-typeB-2.png
deleted file mode 100644
index 95f80b6..0000000
Binary files a/website/images/Kylin-Cube-Creation-Tutorial/8 dim-typeB-2.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/Kylin-Cube-Creation-Tutorial/8 dim-typeC.png
----------------------------------------------------------------------
diff --git a/website/images/Kylin-Cube-Creation-Tutorial/8 dim-typeC.png b/website/images/Kylin-Cube-Creation-Tutorial/8 dim-typeC.png
deleted file mode 100644
index 1bf1c93..0000000
Binary files a/website/images/Kylin-Cube-Creation-Tutorial/8 dim-typeC.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/Kylin-Cube-Creation-Tutorial/8 dim-typeD.png
----------------------------------------------------------------------
diff --git a/website/images/Kylin-Cube-Creation-Tutorial/8 dim-typeD.png b/website/images/Kylin-Cube-Creation-Tutorial/8 dim-typeD.png
deleted file mode 100644
index 430f0fb..0000000
Binary files a/website/images/Kylin-Cube-Creation-Tutorial/8 dim-typeD.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/Kylin-Cube-Creation-Tutorial/9 meas-+meas.png
----------------------------------------------------------------------
diff --git a/website/images/Kylin-Cube-Creation-Tutorial/9 meas-+meas.png b/website/images/Kylin-Cube-Creation-Tutorial/9 meas-+meas.png
deleted file mode 100644
index 8c0ab35..0000000
Binary files a/website/images/Kylin-Cube-Creation-Tutorial/9 meas-+meas.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/Kylin-Cube-Creation-Tutorial/9 meas-count.png
----------------------------------------------------------------------
diff --git a/website/images/Kylin-Cube-Creation-Tutorial/9 meas-count.png b/website/images/Kylin-Cube-Creation-Tutorial/9 meas-count.png
deleted file mode 100644
index af24812..0000000
Binary files a/website/images/Kylin-Cube-Creation-Tutorial/9 meas-count.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/Kylin-Cube-Creation-Tutorial/9 meas-distinct.png
----------------------------------------------------------------------
diff --git a/website/images/Kylin-Cube-Creation-Tutorial/9 meas-distinct.png b/website/images/Kylin-Cube-Creation-Tutorial/9 meas-distinct.png
deleted file mode 100644
index 6772d03..0000000
Binary files a/website/images/Kylin-Cube-Creation-Tutorial/9 meas-distinct.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/Kylin-Cube-Creation-Tutorial/9 meas-max.png
----------------------------------------------------------------------
diff --git a/website/images/Kylin-Cube-Creation-Tutorial/9 meas-max.png b/website/images/Kylin-Cube-Creation-Tutorial/9 meas-max.png
deleted file mode 100644
index 9c8e599..0000000
Binary files a/website/images/Kylin-Cube-Creation-Tutorial/9 meas-max.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/Kylin-Cube-Creation-Tutorial/9 meas-min.png
----------------------------------------------------------------------
diff --git a/website/images/Kylin-Cube-Creation-Tutorial/9 meas-min.png b/website/images/Kylin-Cube-Creation-Tutorial/9 meas-min.png
deleted file mode 100644
index 6ddab38..0000000
Binary files a/website/images/Kylin-Cube-Creation-Tutorial/9 meas-min.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/Kylin-Cube-Creation-Tutorial/9 meas-sum.png
----------------------------------------------------------------------
diff --git a/website/images/Kylin-Cube-Creation-Tutorial/9 meas-sum.png b/website/images/Kylin-Cube-Creation-Tutorial/9 meas-sum.png
deleted file mode 100644
index 09cfbab..0000000
Binary files a/website/images/Kylin-Cube-Creation-Tutorial/9 meas-sum.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/Kylin-Cube-Permission-Grant-Tutorial/14 +grant.png
----------------------------------------------------------------------
diff --git a/website/images/Kylin-Cube-Permission-Grant-Tutorial/14 +grant.png b/website/images/Kylin-Cube-Permission-Grant-Tutorial/14 +grant.png
deleted file mode 100644
index ad65a01..0000000
Binary files a/website/images/Kylin-Cube-Permission-Grant-Tutorial/14 +grant.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/Kylin-Cube-Permission-Grant-Tutorial/15 grantInfo.png
----------------------------------------------------------------------
diff --git a/website/images/Kylin-Cube-Permission-Grant-Tutorial/15 grantInfo.png b/website/images/Kylin-Cube-Permission-Grant-Tutorial/15 grantInfo.png
deleted file mode 100644
index 1a393d6..0000000
Binary files a/website/images/Kylin-Cube-Permission-Grant-Tutorial/15 grantInfo.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/Kylin-Cube-Permission-Grant-Tutorial/16 grant-user.png
----------------------------------------------------------------------
diff --git a/website/images/Kylin-Cube-Permission-Grant-Tutorial/16 grant-user.png b/website/images/Kylin-Cube-Permission-Grant-Tutorial/16 grant-user.png
deleted file mode 100644
index 5bcc2ff..0000000
Binary files a/website/images/Kylin-Cube-Permission-Grant-Tutorial/16 grant-user.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/Kylin-Cube-Permission-Grant-Tutorial/16 user-update.png
----------------------------------------------------------------------
diff --git a/website/images/Kylin-Cube-Permission-Grant-Tutorial/16 user-update.png b/website/images/Kylin-Cube-Permission-Grant-Tutorial/16 user-update.png
deleted file mode 100644
index 98ea4d3..0000000
Binary files a/website/images/Kylin-Cube-Permission-Grant-Tutorial/16 user-update.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/Kylin-ODBC-DSN/1.png
----------------------------------------------------------------------
diff --git a/website/images/Kylin-ODBC-DSN/1.png b/website/images/Kylin-ODBC-DSN/1.png
deleted file mode 100644
index dfaaeee..0000000
Binary files a/website/images/Kylin-ODBC-DSN/1.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/Kylin-ODBC-DSN/2.png
----------------------------------------------------------------------
diff --git a/website/images/Kylin-ODBC-DSN/2.png b/website/images/Kylin-ODBC-DSN/2.png
deleted file mode 100644
index 30fe4e5..0000000
Binary files a/website/images/Kylin-ODBC-DSN/2.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/Kylin-ODBC-DSN/3.png
----------------------------------------------------------------------
diff --git a/website/images/Kylin-ODBC-DSN/3.png b/website/images/Kylin-ODBC-DSN/3.png
deleted file mode 100644
index 8e4ac09..0000000
Binary files a/website/images/Kylin-ODBC-DSN/3.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/Kylin-ODBC-DSN/4.png
----------------------------------------------------------------------
diff --git a/website/images/Kylin-ODBC-DSN/4.png b/website/images/Kylin-ODBC-DSN/4.png
deleted file mode 100644
index 5c0255f..0000000
Binary files a/website/images/Kylin-ODBC-DSN/4.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/Kylin-Web-Tutorial/1 login.png
----------------------------------------------------------------------
diff --git a/website/images/Kylin-Web-Tutorial/1 login.png b/website/images/Kylin-Web-Tutorial/1 login.png
deleted file mode 100644
index 8356341..0000000
Binary files a/website/images/Kylin-Web-Tutorial/1 login.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/Kylin-Web-Tutorial/10 query-result.png
----------------------------------------------------------------------
diff --git a/website/images/Kylin-Web-Tutorial/10 query-result.png b/website/images/Kylin-Web-Tutorial/10 query-result.png
deleted file mode 100644
index 7445314..0000000
Binary files a/website/images/Kylin-Web-Tutorial/10 query-result.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/Kylin-Web-Tutorial/11 save-query-2.png
----------------------------------------------------------------------
diff --git a/website/images/Kylin-Web-Tutorial/11 save-query-2.png b/website/images/Kylin-Web-Tutorial/11 save-query-2.png
deleted file mode 100644
index 5ce14d5..0000000
Binary files a/website/images/Kylin-Web-Tutorial/11 save-query-2.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/Kylin-Web-Tutorial/11 save-query.png
----------------------------------------------------------------------
diff --git a/website/images/Kylin-Web-Tutorial/11 save-query.png b/website/images/Kylin-Web-Tutorial/11 save-query.png
deleted file mode 100644
index ab63aa7..0000000
Binary files a/website/images/Kylin-Web-Tutorial/11 save-query.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/Kylin-Web-Tutorial/12 general.png
----------------------------------------------------------------------
diff --git a/website/images/Kylin-Web-Tutorial/12 general.png b/website/images/Kylin-Web-Tutorial/12 general.png
deleted file mode 100644
index 090df22..0000000
Binary files a/website/images/Kylin-Web-Tutorial/12 general.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/Kylin-Web-Tutorial/13 results.png
----------------------------------------------------------------------
diff --git a/website/images/Kylin-Web-Tutorial/13 results.png b/website/images/Kylin-Web-Tutorial/13 results.png
deleted file mode 100644
index cc17841..0000000
Binary files a/website/images/Kylin-Web-Tutorial/13 results.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/Kylin-Web-Tutorial/14 drag.png
----------------------------------------------------------------------
diff --git a/website/images/Kylin-Web-Tutorial/14 drag.png b/website/images/Kylin-Web-Tutorial/14 drag.png
deleted file mode 100644
index e1b889e..0000000
Binary files a/website/images/Kylin-Web-Tutorial/14 drag.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/Kylin-Web-Tutorial/15 bar-chart.png
----------------------------------------------------------------------
diff --git a/website/images/Kylin-Web-Tutorial/15 bar-chart.png b/website/images/Kylin-Web-Tutorial/15 bar-chart.png
deleted file mode 100644
index dc7a75d..0000000
Binary files a/website/images/Kylin-Web-Tutorial/15 bar-chart.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/Kylin-Web-Tutorial/16 pie-chart.png
----------------------------------------------------------------------
diff --git a/website/images/Kylin-Web-Tutorial/16 pie-chart.png b/website/images/Kylin-Web-Tutorial/16 pie-chart.png
deleted file mode 100644
index d66dced..0000000
Binary files a/website/images/Kylin-Web-Tutorial/16 pie-chart.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/Kylin-Web-Tutorial/17 line-chart.png
----------------------------------------------------------------------
diff --git a/website/images/Kylin-Web-Tutorial/17 line-chart.png b/website/images/Kylin-Web-Tutorial/17 line-chart.png
deleted file mode 100644
index 12d1696..0000000
Binary files a/website/images/Kylin-Web-Tutorial/17 line-chart.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/Kylin-Web-Tutorial/2 tables.png
----------------------------------------------------------------------
diff --git a/website/images/Kylin-Web-Tutorial/2 tables.png b/website/images/Kylin-Web-Tutorial/2 tables.png
deleted file mode 100644
index b7fd530..0000000
Binary files a/website/images/Kylin-Web-Tutorial/2 tables.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/Kylin-Web-Tutorial/3 cubes.png
----------------------------------------------------------------------
diff --git a/website/images/Kylin-Web-Tutorial/3 cubes.png b/website/images/Kylin-Web-Tutorial/3 cubes.png
deleted file mode 100644
index 34eee2e..0000000
Binary files a/website/images/Kylin-Web-Tutorial/3 cubes.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/Kylin-Web-Tutorial/4 form-view.png
----------------------------------------------------------------------
diff --git a/website/images/Kylin-Web-Tutorial/4 form-view.png b/website/images/Kylin-Web-Tutorial/4 form-view.png
deleted file mode 100644
index ab5cbd1..0000000
Binary files a/website/images/Kylin-Web-Tutorial/4 form-view.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/Kylin-Web-Tutorial/5 sql-view.png
----------------------------------------------------------------------
diff --git a/website/images/Kylin-Web-Tutorial/5 sql-view.png b/website/images/Kylin-Web-Tutorial/5 sql-view.png
deleted file mode 100644
index 69a4cc6..0000000
Binary files a/website/images/Kylin-Web-Tutorial/5 sql-view.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/Kylin-Web-Tutorial/6 visualization.png
----------------------------------------------------------------------
diff --git a/website/images/Kylin-Web-Tutorial/6 visualization.png b/website/images/Kylin-Web-Tutorial/6 visualization.png
deleted file mode 100644
index 745f2d8..0000000
Binary files a/website/images/Kylin-Web-Tutorial/6 visualization.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/Kylin-Web-Tutorial/7 access.png
----------------------------------------------------------------------
diff --git a/website/images/Kylin-Web-Tutorial/7 access.png b/website/images/Kylin-Web-Tutorial/7 access.png
deleted file mode 100644
index 9db1da8..0000000
Binary files a/website/images/Kylin-Web-Tutorial/7 access.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/Kylin-Web-Tutorial/7 job-steps.png
----------------------------------------------------------------------
diff --git a/website/images/Kylin-Web-Tutorial/7 job-steps.png b/website/images/Kylin-Web-Tutorial/7 job-steps.png
deleted file mode 100644
index 664b097..0000000
Binary files a/website/images/Kylin-Web-Tutorial/7 job-steps.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/Kylin-Web-Tutorial/8 query.png
----------------------------------------------------------------------
diff --git a/website/images/Kylin-Web-Tutorial/8 query.png b/website/images/Kylin-Web-Tutorial/8 query.png
deleted file mode 100644
index f30889e..0000000
Binary files a/website/images/Kylin-Web-Tutorial/8 query.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/Kylin-Web-Tutorial/9 query-table.png
----------------------------------------------------------------------
diff --git a/website/images/Kylin-Web-Tutorial/9 query-table.png b/website/images/Kylin-Web-Tutorial/9 query-table.png
deleted file mode 100644
index 1fecda4..0000000
Binary files a/website/images/Kylin-Web-Tutorial/9 query-table.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/Kylin-and-Tableau-Tutorial/1 odbc.png
----------------------------------------------------------------------
diff --git a/website/images/Kylin-and-Tableau-Tutorial/1 odbc.png b/website/images/Kylin-and-Tableau-Tutorial/1 odbc.png
deleted file mode 100644
index 1a9988a..0000000
Binary files a/website/images/Kylin-and-Tableau-Tutorial/1 odbc.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/Kylin-and-Tableau-Tutorial/10 connectLive.jpg
----------------------------------------------------------------------
diff --git a/website/images/Kylin-and-Tableau-Tutorial/10 connectLive.jpg b/website/images/Kylin-and-Tableau-Tutorial/10 connectLive.jpg
deleted file mode 100644
index d28fa06..0000000
Binary files a/website/images/Kylin-and-Tableau-Tutorial/10 connectLive.jpg and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/Kylin-and-Tableau-Tutorial/11 analysis.jpg
----------------------------------------------------------------------
diff --git a/website/images/Kylin-and-Tableau-Tutorial/11 analysis.jpg b/website/images/Kylin-and-Tableau-Tutorial/11 analysis.jpg
deleted file mode 100644
index 53bf70d..0000000
Binary files a/website/images/Kylin-and-Tableau-Tutorial/11 analysis.jpg and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/Kylin-and-Tableau-Tutorial/12 edit tables.jpg
----------------------------------------------------------------------
diff --git a/website/images/Kylin-and-Tableau-Tutorial/12 edit tables.jpg b/website/images/Kylin-and-Tableau-Tutorial/12 edit tables.jpg
deleted file mode 100644
index bb7d4c6..0000000
Binary files a/website/images/Kylin-and-Tableau-Tutorial/12 edit tables.jpg and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/Kylin-and-Tableau-Tutorial/13 visual.jpg
----------------------------------------------------------------------
diff --git a/website/images/Kylin-and-Tableau-Tutorial/13 visual.jpg b/website/images/Kylin-and-Tableau-Tutorial/13 visual.jpg
deleted file mode 100644
index 74930ce..0000000
Binary files a/website/images/Kylin-and-Tableau-Tutorial/13 visual.jpg and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/Kylin-and-Tableau-Tutorial/14 publish.jpg
----------------------------------------------------------------------
diff --git a/website/images/Kylin-and-Tableau-Tutorial/14 publish.jpg b/website/images/Kylin-and-Tableau-Tutorial/14 publish.jpg
deleted file mode 100644
index b118966..0000000
Binary files a/website/images/Kylin-and-Tableau-Tutorial/14 publish.jpg and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/Kylin-and-Tableau-Tutorial/16 prepare-publish.png
----------------------------------------------------------------------
diff --git a/website/images/Kylin-and-Tableau-Tutorial/16 prepare-publish.png b/website/images/Kylin-and-Tableau-Tutorial/16 prepare-publish.png
deleted file mode 100644
index 8a99ab6..0000000
Binary files a/website/images/Kylin-and-Tableau-Tutorial/16 prepare-publish.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/Kylin-and-Tableau-Tutorial/17 embedded-pwd.png
----------------------------------------------------------------------
diff --git a/website/images/Kylin-and-Tableau-Tutorial/17 embedded-pwd.png b/website/images/Kylin-and-Tableau-Tutorial/17 embedded-pwd.png
deleted file mode 100644
index 3f9227b..0000000
Binary files a/website/images/Kylin-and-Tableau-Tutorial/17 embedded-pwd.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/Kylin-and-Tableau-Tutorial/18 groupby-folder.jpg
----------------------------------------------------------------------
diff --git a/website/images/Kylin-and-Tableau-Tutorial/18 groupby-folder.jpg b/website/images/Kylin-and-Tableau-Tutorial/18 groupby-folder.jpg
deleted file mode 100644
index 3e4b246..0000000
Binary files a/website/images/Kylin-and-Tableau-Tutorial/18 groupby-folder.jpg and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/Kylin-and-Tableau-Tutorial/19 custom.jpg
----------------------------------------------------------------------
diff --git a/website/images/Kylin-and-Tableau-Tutorial/19 custom.jpg b/website/images/Kylin-and-Tableau-Tutorial/19 custom.jpg
deleted file mode 100644
index 93e058d..0000000
Binary files a/website/images/Kylin-and-Tableau-Tutorial/19 custom.jpg and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/Kylin-and-Tableau-Tutorial/2 serverhost.jpg
----------------------------------------------------------------------
diff --git a/website/images/Kylin-and-Tableau-Tutorial/2 serverhost.jpg b/website/images/Kylin-and-Tableau-Tutorial/2 serverhost.jpg
deleted file mode 100644
index f06755f..0000000
Binary files a/website/images/Kylin-and-Tableau-Tutorial/2 serverhost.jpg and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/Kylin-and-Tableau-Tutorial/3 project.jpg
----------------------------------------------------------------------
diff --git a/website/images/Kylin-and-Tableau-Tutorial/3 project.jpg b/website/images/Kylin-and-Tableau-Tutorial/3 project.jpg
deleted file mode 100644
index b127a3d..0000000
Binary files a/website/images/Kylin-and-Tableau-Tutorial/3 project.jpg and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/Kylin-and-Tableau-Tutorial/4 done.jpg
----------------------------------------------------------------------
diff --git a/website/images/Kylin-and-Tableau-Tutorial/4 done.jpg b/website/images/Kylin-and-Tableau-Tutorial/4 done.jpg
deleted file mode 100644
index 1777013..0000000
Binary files a/website/images/Kylin-and-Tableau-Tutorial/4 done.jpg and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/Kylin-and-Tableau-Tutorial/5 multipleTable.jpg
----------------------------------------------------------------------
diff --git a/website/images/Kylin-and-Tableau-Tutorial/5 multipleTable.jpg b/website/images/Kylin-and-Tableau-Tutorial/5 multipleTable.jpg
deleted file mode 100644
index 9e13460..0000000
Binary files a/website/images/Kylin-and-Tableau-Tutorial/5 multipleTable.jpg and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/Kylin-and-Tableau-Tutorial/6 facttable.jpg
----------------------------------------------------------------------
diff --git a/website/images/Kylin-and-Tableau-Tutorial/6 facttable.jpg b/website/images/Kylin-and-Tableau-Tutorial/6 facttable.jpg
deleted file mode 100644
index f08ab52..0000000
Binary files a/website/images/Kylin-and-Tableau-Tutorial/6 facttable.jpg and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/Kylin-and-Tableau-Tutorial/6 facttable2.jpg
----------------------------------------------------------------------
diff --git a/website/images/Kylin-and-Tableau-Tutorial/6 facttable2.jpg b/website/images/Kylin-and-Tableau-Tutorial/6 facttable2.jpg
deleted file mode 100644
index 509d46b..0000000
Binary files a/website/images/Kylin-and-Tableau-Tutorial/6 facttable2.jpg and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/Kylin-and-Tableau-Tutorial/7 lkptable.jpg
----------------------------------------------------------------------
diff --git a/website/images/Kylin-and-Tableau-Tutorial/7 lkptable.jpg b/website/images/Kylin-and-Tableau-Tutorial/7 lkptable.jpg
deleted file mode 100644
index cd4aad6..0000000
Binary files a/website/images/Kylin-and-Tableau-Tutorial/7 lkptable.jpg and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/Kylin-and-Tableau-Tutorial/8 join.jpg
----------------------------------------------------------------------
diff --git a/website/images/Kylin-and-Tableau-Tutorial/8 join.jpg b/website/images/Kylin-and-Tableau-Tutorial/8 join.jpg
deleted file mode 100644
index 05f7780..0000000
Binary files a/website/images/Kylin-and-Tableau-Tutorial/8 join.jpg and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/Kylin-and-Tableau-Tutorial/9 connName.jpg
----------------------------------------------------------------------
diff --git a/website/images/Kylin-and-Tableau-Tutorial/9 connName.jpg b/website/images/Kylin-and-Tableau-Tutorial/9 connName.jpg
deleted file mode 100644
index 5c816d3..0000000
Binary files a/website/images/Kylin-and-Tableau-Tutorial/9 connName.jpg and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/blog/by-layer-cubing.png
----------------------------------------------------------------------
diff --git a/website/images/blog/by-layer-cubing.png b/website/images/blog/by-layer-cubing.png
deleted file mode 100644
index b5238d2..0000000
Binary files a/website/images/blog/by-layer-cubing.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/blog/by-segment-cubing.png
----------------------------------------------------------------------
diff --git a/website/images/blog/by-segment-cubing.png b/website/images/blog/by-segment-cubing.png
deleted file mode 100644
index 02fbec3..0000000
Binary files a/website/images/blog/by-segment-cubing.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/blog/cube-spanning-tree.png
----------------------------------------------------------------------
diff --git a/website/images/blog/cube-spanning-tree.png b/website/images/blog/cube-spanning-tree.png
deleted file mode 100644
index 71c464c..0000000
Binary files a/website/images/blog/cube-spanning-tree.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/blog/hybrid-model.png
----------------------------------------------------------------------
diff --git a/website/images/blog/hybrid-model.png b/website/images/blog/hybrid-model.png
deleted file mode 100644
index 5fd476c..0000000
Binary files a/website/images/blog/hybrid-model.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/blog/meetup_1.jpeg
----------------------------------------------------------------------
diff --git a/website/images/blog/meetup_1.jpeg b/website/images/blog/meetup_1.jpeg
deleted file mode 100644
index 71d637a..0000000
Binary files a/website/images/blog/meetup_1.jpeg and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/blog/meetup_2.jpeg
----------------------------------------------------------------------
diff --git a/website/images/blog/meetup_2.jpeg b/website/images/blog/meetup_2.jpeg
deleted file mode 100644
index 21b0c1b..0000000
Binary files a/website/images/blog/meetup_2.jpeg and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/blog/meetup_3.jpeg
----------------------------------------------------------------------
diff --git a/website/images/blog/meetup_3.jpeg b/website/images/blog/meetup_3.jpeg
deleted file mode 100644
index 05b73a2..0000000
Binary files a/website/images/blog/meetup_3.jpeg and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/blog/meetup_4.jpeg
----------------------------------------------------------------------
diff --git a/website/images/blog/meetup_4.jpeg b/website/images/blog/meetup_4.jpeg
deleted file mode 100644
index b69cd83..0000000
Binary files a/website/images/blog/meetup_4.jpeg and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/blog/meetup_5.jpeg
----------------------------------------------------------------------
diff --git a/website/images/blog/meetup_5.jpeg b/website/images/blog/meetup_5.jpeg
deleted file mode 100644
index e65123c..0000000
Binary files a/website/images/blog/meetup_5.jpeg and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/blog/meetup_6.jpeg
----------------------------------------------------------------------
diff --git a/website/images/blog/meetup_6.jpeg b/website/images/blog/meetup_6.jpeg
deleted file mode 100644
index 56f38ae..0000000
Binary files a/website/images/blog/meetup_6.jpeg and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/blog/meetup_7.jpeg
----------------------------------------------------------------------
diff --git a/website/images/blog/meetup_7.jpeg b/website/images/blog/meetup_7.jpeg
deleted file mode 100644
index 4186c56..0000000
Binary files a/website/images/blog/meetup_7.jpeg and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/blog/meetup_8.jpeg
----------------------------------------------------------------------
diff --git a/website/images/blog/meetup_8.jpeg b/website/images/blog/meetup_8.jpeg
deleted file mode 100644
index 4045763..0000000
Binary files a/website/images/blog/meetup_8.jpeg and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/blog/meetup_9.jpeg
----------------------------------------------------------------------
diff --git a/website/images/blog/meetup_9.jpeg b/website/images/blog/meetup_9.jpeg
deleted file mode 100644
index 438e956..0000000
Binary files a/website/images/blog/meetup_9.jpeg and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/develop/dictionary-bytes-format.png
----------------------------------------------------------------------
diff --git a/website/images/develop/dictionary-bytes-format.png b/website/images/develop/dictionary-bytes-format.png
deleted file mode 100644
index d3641f6..0000000
Binary files a/website/images/develop/dictionary-bytes-format.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/develop/dictionary-cache-layer.png
----------------------------------------------------------------------
diff --git a/website/images/develop/dictionary-cache-layer.png b/website/images/develop/dictionary-cache-layer.png
deleted file mode 100644
index 99b3dcc..0000000
Binary files a/website/images/develop/dictionary-cache-layer.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/develop/dictionary-trie-sample.png
----------------------------------------------------------------------
diff --git a/website/images/develop/dictionary-trie-sample.png b/website/images/develop/dictionary-trie-sample.png
deleted file mode 100644
index 7b63d04..0000000
Binary files a/website/images/develop/dictionary-trie-sample.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/develop/disable_import_on_the_fly.png
----------------------------------------------------------------------
diff --git a/website/images/develop/disable_import_on_the_fly.png b/website/images/develop/disable_import_on_the_fly.png
deleted file mode 100644
index 7b2412d..0000000
Binary files a/website/images/develop/disable_import_on_the_fly.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/develop/eclipse_code_formatter_config.png
----------------------------------------------------------------------
diff --git a/website/images/develop/eclipse_code_formatter_config.png b/website/images/develop/eclipse_code_formatter_config.png
deleted file mode 100644
index 32d8952..0000000
Binary files a/website/images/develop/eclipse_code_formatter_config.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/docs/concepts/AggregationGroup.png
----------------------------------------------------------------------
diff --git a/website/images/docs/concepts/AggregationGroup.png b/website/images/docs/concepts/AggregationGroup.png
deleted file mode 100644
index 0e563fc..0000000
Binary files a/website/images/docs/concepts/AggregationGroup.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/docs/concepts/CubeAction.png
----------------------------------------------------------------------
diff --git a/website/images/docs/concepts/CubeAction.png b/website/images/docs/concepts/CubeAction.png
deleted file mode 100644
index bbc6ef5..0000000
Binary files a/website/images/docs/concepts/CubeAction.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/docs/concepts/CubeDesc.png
----------------------------------------------------------------------
diff --git a/website/images/docs/concepts/CubeDesc.png b/website/images/docs/concepts/CubeDesc.png
deleted file mode 100644
index a0736a4..0000000
Binary files a/website/images/docs/concepts/CubeDesc.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/docs/concepts/CubeInstance.png
----------------------------------------------------------------------
diff --git a/website/images/docs/concepts/CubeInstance.png b/website/images/docs/concepts/CubeInstance.png
deleted file mode 100644
index 7748afa..0000000
Binary files a/website/images/docs/concepts/CubeInstance.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/docs/concepts/CubeSegment.png
----------------------------------------------------------------------
diff --git a/website/images/docs/concepts/CubeSegment.png b/website/images/docs/concepts/CubeSegment.png
deleted file mode 100644
index 6f57720..0000000
Binary files a/website/images/docs/concepts/CubeSegment.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/docs/concepts/DataModel.png
----------------------------------------------------------------------
diff --git a/website/images/docs/concepts/DataModel.png b/website/images/docs/concepts/DataModel.png
deleted file mode 100644
index dd959f5..0000000
Binary files a/website/images/docs/concepts/DataModel.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/docs/concepts/DataSource.png
----------------------------------------------------------------------
diff --git a/website/images/docs/concepts/DataSource.png b/website/images/docs/concepts/DataSource.png
deleted file mode 100644
index 1933fa3..0000000
Binary files a/website/images/docs/concepts/DataSource.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/docs/concepts/Dimension.png
----------------------------------------------------------------------
diff --git a/website/images/docs/concepts/Dimension.png b/website/images/docs/concepts/Dimension.png
deleted file mode 100644
index 65e5810..0000000
Binary files a/website/images/docs/concepts/Dimension.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/docs/concepts/Job.png
----------------------------------------------------------------------
diff --git a/website/images/docs/concepts/Job.png b/website/images/docs/concepts/Job.png
deleted file mode 100644
index a790239..0000000
Binary files a/website/images/docs/concepts/Job.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/docs/concepts/JobAction.png
----------------------------------------------------------------------
diff --git a/website/images/docs/concepts/JobAction.png b/website/images/docs/concepts/JobAction.png
deleted file mode 100644
index 1ec370b..0000000
Binary files a/website/images/docs/concepts/JobAction.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/docs/concepts/Measure.png
----------------------------------------------------------------------
diff --git a/website/images/docs/concepts/Measure.png b/website/images/docs/concepts/Measure.png
deleted file mode 100644
index 34542f6..0000000
Binary files a/website/images/docs/concepts/Measure.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/docs/concepts/Partition.png
----------------------------------------------------------------------
diff --git a/website/images/docs/concepts/Partition.png b/website/images/docs/concepts/Partition.png
deleted file mode 100644
index 636eaed..0000000
Binary files a/website/images/docs/concepts/Partition.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/install/Picture1.png
----------------------------------------------------------------------
diff --git a/website/images/install/Picture1.png b/website/images/install/Picture1.png
deleted file mode 100644
index 5b07d25..0000000
Binary files a/website/images/install/Picture1.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/install/Picture2.png
----------------------------------------------------------------------
diff --git a/website/images/install/Picture2.png b/website/images/install/Picture2.png
deleted file mode 100644
index 26e9e4f..0000000
Binary files a/website/images/install/Picture2.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/install/cube.png
----------------------------------------------------------------------
diff --git a/website/images/install/cube.png b/website/images/install/cube.png
deleted file mode 100644
index bb84b58..0000000
Binary files a/website/images/install/cube.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/install/installation.pptx
----------------------------------------------------------------------
diff --git a/website/images/install/installation.pptx b/website/images/install/installation.pptx
deleted file mode 100644
index 548d276..0000000
Binary files a/website/images/install/installation.pptx and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/install/job.png
----------------------------------------------------------------------
diff --git a/website/images/install/job.png b/website/images/install/job.png
deleted file mode 100644
index 0ff57f8..0000000
Binary files a/website/images/install/job.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/install/kylin_server_modes.png
----------------------------------------------------------------------
diff --git a/website/images/install/kylin_server_modes.png b/website/images/install/kylin_server_modes.png
deleted file mode 100644
index bc4a398..0000000
Binary files a/website/images/install/kylin_server_modes.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/install/off_cli_install_scene.png
----------------------------------------------------------------------
diff --git a/website/images/install/off_cli_install_scene.png b/website/images/install/off_cli_install_scene.png
deleted file mode 100644
index 26e9e4f..0000000
Binary files a/website/images/install/off_cli_install_scene.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/install/on_cli_install_scene.png
----------------------------------------------------------------------
diff --git a/website/images/install/on_cli_install_scene.png b/website/images/install/on_cli_install_scene.png
deleted file mode 100644
index 5b07d25..0000000
Binary files a/website/images/install/on_cli_install_scene.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/install/query.png
----------------------------------------------------------------------
diff --git a/website/images/install/query.png b/website/images/install/query.png
deleted file mode 100644
index dd30e27..0000000
Binary files a/website/images/install/query.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/install/starthbase.png
----------------------------------------------------------------------
diff --git a/website/images/install/starthbase.png b/website/images/install/starthbase.png
deleted file mode 100644
index ca5e825..0000000
Binary files a/website/images/install/starthbase.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/tutorial/odbc/ms_tool/Picture1.png
----------------------------------------------------------------------
diff --git a/website/images/tutorial/odbc/ms_tool/Picture1.png b/website/images/tutorial/odbc/ms_tool/Picture1.png
deleted file mode 100755
index de8bbb7..0000000
Binary files a/website/images/tutorial/odbc/ms_tool/Picture1.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/tutorial/odbc/ms_tool/Picture2.png
----------------------------------------------------------------------
diff --git a/website/images/tutorial/odbc/ms_tool/Picture2.png b/website/images/tutorial/odbc/ms_tool/Picture2.png
deleted file mode 100755
index 1e45b27..0000000
Binary files a/website/images/tutorial/odbc/ms_tool/Picture2.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/tutorial/odbc/ms_tool/Picture3.png
----------------------------------------------------------------------
diff --git a/website/images/tutorial/odbc/ms_tool/Picture3.png b/website/images/tutorial/odbc/ms_tool/Picture3.png
deleted file mode 100755
index c59a705..0000000
Binary files a/website/images/tutorial/odbc/ms_tool/Picture3.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/tutorial/odbc/ms_tool/Picture4.png
----------------------------------------------------------------------
diff --git a/website/images/tutorial/odbc/ms_tool/Picture4.png b/website/images/tutorial/odbc/ms_tool/Picture4.png
deleted file mode 100755
index 4a06b6c..0000000
Binary files a/website/images/tutorial/odbc/ms_tool/Picture4.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/tutorial/odbc/ms_tool/Picture5.png
----------------------------------------------------------------------
diff --git a/website/images/tutorial/odbc/ms_tool/Picture5.png b/website/images/tutorial/odbc/ms_tool/Picture5.png
deleted file mode 100755
index e96e627..0000000
Binary files a/website/images/tutorial/odbc/ms_tool/Picture5.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/tutorial/odbc/ms_tool/Picture6.png
----------------------------------------------------------------------
diff --git a/website/images/tutorial/odbc/ms_tool/Picture6.png b/website/images/tutorial/odbc/ms_tool/Picture6.png
deleted file mode 100755
index 010a2f9..0000000
Binary files a/website/images/tutorial/odbc/ms_tool/Picture6.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/tutorial/odbc/ms_tool/Picture7.png
----------------------------------------------------------------------
diff --git a/website/images/tutorial/odbc/ms_tool/Picture7.png b/website/images/tutorial/odbc/ms_tool/Picture7.png
deleted file mode 100755
index 24ec92a..0000000
Binary files a/website/images/tutorial/odbc/ms_tool/Picture7.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/tutorial/odbc/ms_tool/Picture8.png
----------------------------------------------------------------------
diff --git a/website/images/tutorial/odbc/ms_tool/Picture8.png b/website/images/tutorial/odbc/ms_tool/Picture8.png
deleted file mode 100755
index 69914c2..0000000
Binary files a/website/images/tutorial/odbc/ms_tool/Picture8.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/tutorial/odbc/tableau_91/1.png
----------------------------------------------------------------------
diff --git a/website/images/tutorial/odbc/tableau_91/1.png b/website/images/tutorial/odbc/tableau_91/1.png
deleted file mode 100755
index 231deb9..0000000
Binary files a/website/images/tutorial/odbc/tableau_91/1.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/tutorial/odbc/tableau_91/2.png
----------------------------------------------------------------------
diff --git a/website/images/tutorial/odbc/tableau_91/2.png b/website/images/tutorial/odbc/tableau_91/2.png
deleted file mode 100755
index 5fc8092..0000000
Binary files a/website/images/tutorial/odbc/tableau_91/2.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/tutorial/odbc/tableau_91/3.png
----------------------------------------------------------------------
diff --git a/website/images/tutorial/odbc/tableau_91/3.png b/website/images/tutorial/odbc/tableau_91/3.png
deleted file mode 100755
index 049798c..0000000
Binary files a/website/images/tutorial/odbc/tableau_91/3.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/tutorial/odbc/tableau_91/4.png
----------------------------------------------------------------------
diff --git a/website/images/tutorial/odbc/tableau_91/4.png b/website/images/tutorial/odbc/tableau_91/4.png
deleted file mode 100755
index 777bdb8..0000000
Binary files a/website/images/tutorial/odbc/tableau_91/4.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/tutorial/odbc/tableau_91/5.png
----------------------------------------------------------------------
diff --git a/website/images/tutorial/odbc/tableau_91/5.png b/website/images/tutorial/odbc/tableau_91/5.png
deleted file mode 100755
index 3703cad..0000000
Binary files a/website/images/tutorial/odbc/tableau_91/5.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/tutorial/odbc/tableau_91/6.png
----------------------------------------------------------------------
diff --git a/website/images/tutorial/odbc/tableau_91/6.png b/website/images/tutorial/odbc/tableau_91/6.png
deleted file mode 100755
index e8fe866..0000000
Binary files a/website/images/tutorial/odbc/tableau_91/6.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/images/tutorial/odbc/tableau_91/7.png
----------------------------------------------------------------------
diff --git a/website/images/tutorial/odbc/tableau_91/7.png b/website/images/tutorial/odbc/tableau_91/7.png
deleted file mode 100755
index 1dd727d..0000000
Binary files a/website/images/tutorial/odbc/tableau_91/7.png and /dev/null differ


[45/52] [abbrv] kylin git commit: KYLIN-1419 NPE occurs when query from subqueries with order by (revised by Yang)

Posted by li...@apache.org.
KYLIN-1419 NPE occurs when query from subqueries with order by (revised by Yang)


Project: http://git-wip-us.apache.org/repos/asf/kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/b1fe5eac
Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/b1fe5eac
Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/b1fe5eac

Branch: refs/heads/1.x-HBase1.1.3
Commit: b1fe5eac8555f3dda580c455d124f54f703c8ec9
Parents: 24b7aa7
Author: nichunen <ni...@mininglamp.com>
Authored: Wed Feb 17 11:07:32 2016 +0800
Committer: Li, Yang <li...@apache.org>
Committed: Wed Feb 17 11:08:35 2016 +0800

----------------------------------------------------------------------
 .../apache/kylin/query/relnode/OLAPSortRel.java |  5 +++
 .../resources/query/sql_subquery/query09.sql    | 33 ++++++++++++++++++++
 2 files changed, 38 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/b1fe5eac/query/src/main/java/org/apache/kylin/query/relnode/OLAPSortRel.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/org/apache/kylin/query/relnode/OLAPSortRel.java b/query/src/main/java/org/apache/kylin/query/relnode/OLAPSortRel.java
index e8b7d17..6d13b07 100644
--- a/query/src/main/java/org/apache/kylin/query/relnode/OLAPSortRel.java
+++ b/query/src/main/java/org/apache/kylin/query/relnode/OLAPSortRel.java
@@ -81,6 +81,11 @@ public class OLAPSortRel extends Sort implements OLAPRel {
     public void implementRewrite(RewriteImplementor implementor) {
         implementor.visitChild(this, getInput());
 
+        // No need to rewrite "order by" applied on non-olap context.
+        // Occurs in sub-query like "select ... from (...) inner join (...) order by ..."
+        if (this.context.realization == null)
+            return;
+        
         for (RelFieldCollation fieldCollation : this.collation.getFieldCollations()) {
             int index = fieldCollation.getFieldIndex();
             StorageContext.OrderEnum order = getOrderEnum(fieldCollation.getDirection());

http://git-wip-us.apache.org/repos/asf/kylin/blob/b1fe5eac/query/src/test/resources/query/sql_subquery/query09.sql
----------------------------------------------------------------------
diff --git a/query/src/test/resources/query/sql_subquery/query09.sql b/query/src/test/resources/query/sql_subquery/query09.sql
new file mode 100644
index 0000000..3502cd7
--- /dev/null
+++ b/query/src/test/resources/query/sql_subquery/query09.sql
@@ -0,0 +1,33 @@
+--
+-- 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.
+--
+
+SELECT t1.week_beg_dt, t1.sum_price, t2.cnt
+FROM (
+  select test_cal_dt.week_beg_dt, sum(test_kylin_fact.price) as sum_price
+  from test_kylin_fact
+  inner join edw.test_cal_dt as test_cal_dt ON test_kylin_fact.cal_dt = test_cal_dt.cal_dt
+  group by test_cal_dt.week_beg_dt
+) t1
+inner join  (
+  select test_cal_dt.week_beg_dt, count(*) as cnt
+  from test_kylin_fact
+  inner join edw.test_cal_dt as test_cal_dt ON test_kylin_fact.cal_dt = test_cal_dt.cal_dt
+  group by test_cal_dt.week_beg_dt
+) t2
+on t1.week_beg_dt=t2.week_beg_dt
+order by t1.week_beg_dt


[06/52] [abbrv] kylin git commit: KYLIN-1343 fix Job test cases running in local mode

Posted by li...@apache.org.
KYLIN-1343 fix Job test cases running in local mode


Project: http://git-wip-us.apache.org/repos/asf/kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/dfdeb92e
Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/dfdeb92e
Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/dfdeb92e

Branch: refs/heads/1.x-HBase1.1.3
Commit: dfdeb92ecc9fa89b0bc488e2f355eb7698537ef7
Parents: 2bd3b4e
Author: Yang Li <li...@apache.org>
Authored: Sun Jan 24 09:18:25 2016 +0800
Committer: Yang Li <li...@apache.org>
Committed: Sun Jan 24 09:18:33 2016 +0800

----------------------------------------------------------------------
 .../kylin/job/hadoop/cube/BaseCuboidJobTest.java    | 16 ++++++++++------
 .../job/hadoop/cube/KeyDistributionJobTest.java     |  4 +---
 .../kylin/job/hadoop/cube/MergeCuboidJobTest.java   |  6 +-----
 .../kylin/job/hadoop/cube/NDCuboidJobTest.java      |  9 ++-------
 .../hadoop/cube/RangeKeyDistributionJobTest.java    |  6 +-----
 .../kylin/job/hadoop/hbase/CreateHTableTest.java    |  6 +++---
 .../kylin/job/tools/ColumnCardinalityJobTest.java   |  5 ++---
 7 files changed, 20 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/dfdeb92e/job/src/test/java/org/apache/kylin/job/hadoop/cube/BaseCuboidJobTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/org/apache/kylin/job/hadoop/cube/BaseCuboidJobTest.java b/job/src/test/java/org/apache/kylin/job/hadoop/cube/BaseCuboidJobTest.java
index 1775eae..8d22efb 100644
--- a/job/src/test/java/org/apache/kylin/job/hadoop/cube/BaseCuboidJobTest.java
+++ b/job/src/test/java/org/apache/kylin/job/hadoop/cube/BaseCuboidJobTest.java
@@ -31,18 +31,22 @@ import org.junit.Before;
 import org.junit.Test;
 
 public class BaseCuboidJobTest extends LocalFileMetadataTestCase {
+    
+    public static void setLocalMR(Configuration conf) {
+        conf.set("fs.default.name", "file:///");
+        conf.set("mapreduce.framework.name", "local");
+        conf.set("mapreduce.application.framework.path", "");
+
+        // for local runner out-of-memory issue
+        conf.set("mapreduce.task.io.sort.mb", "10");
+    }
 
     private Configuration conf;
 
     @Before
     public void setup() throws Exception {
         conf = new Configuration();
-        conf.set("fs.default.name", "file:///");
-        conf.set("mapred.job.tracker", "local");
-        conf.set("mapreduce.application.framework.path", "");
-
-        // for local runner out-of-memory issue
-        conf.set("mapreduce.task.io.sort.mb", "10");
+        BaseCuboidJobTest.setLocalMR(conf);
 
         createTestMetadata();
     }

http://git-wip-us.apache.org/repos/asf/kylin/blob/dfdeb92e/job/src/test/java/org/apache/kylin/job/hadoop/cube/KeyDistributionJobTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/org/apache/kylin/job/hadoop/cube/KeyDistributionJobTest.java b/job/src/test/java/org/apache/kylin/job/hadoop/cube/KeyDistributionJobTest.java
index f5b3b7d..6f7db6a 100644
--- a/job/src/test/java/org/apache/kylin/job/hadoop/cube/KeyDistributionJobTest.java
+++ b/job/src/test/java/org/apache/kylin/job/hadoop/cube/KeyDistributionJobTest.java
@@ -60,9 +60,7 @@ package org.apache.kylin.job.hadoop.cube;
 //    @Before
 //    public void setup() throws IOException {
 //        conf = new Configuration();
-//        conf.set("fs.default.name", "file:///");
-//        conf.set("mapred.job.tracker", "local");
-//        conf.set("mapreduce.application.framework.path", "");
+//        BaseCuboidJobTest.setLocalMR(conf);
 //    }
 //
 //    @Test

http://git-wip-us.apache.org/repos/asf/kylin/blob/dfdeb92e/job/src/test/java/org/apache/kylin/job/hadoop/cube/MergeCuboidJobTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/org/apache/kylin/job/hadoop/cube/MergeCuboidJobTest.java b/job/src/test/java/org/apache/kylin/job/hadoop/cube/MergeCuboidJobTest.java
index 012b3a1..6c17c69 100644
--- a/job/src/test/java/org/apache/kylin/job/hadoop/cube/MergeCuboidJobTest.java
+++ b/job/src/test/java/org/apache/kylin/job/hadoop/cube/MergeCuboidJobTest.java
@@ -38,12 +38,8 @@ public class MergeCuboidJobTest extends LocalFileMetadataTestCase {
     @Before
     public void setup() throws Exception {
         conf = new Configuration();
-        conf.set("fs.default.name", "file:///");
-        conf.set("mapred.job.tracker", "local");
-        conf.set("mapreduce.application.framework.path", "");
+        BaseCuboidJobTest.setLocalMR(conf);
 
-        // for local runner out-of-memory issue
-        conf.set("mapreduce.task.io.sort.mb", "10");
         createTestMetadata();
     }
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/dfdeb92e/job/src/test/java/org/apache/kylin/job/hadoop/cube/NDCuboidJobTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/org/apache/kylin/job/hadoop/cube/NDCuboidJobTest.java b/job/src/test/java/org/apache/kylin/job/hadoop/cube/NDCuboidJobTest.java
index 5ba3ec6..8b5e78d 100644
--- a/job/src/test/java/org/apache/kylin/job/hadoop/cube/NDCuboidJobTest.java
+++ b/job/src/test/java/org/apache/kylin/job/hadoop/cube/NDCuboidJobTest.java
@@ -18,7 +18,7 @@
 
 package org.apache.kylin.job.hadoop.cube;
 
-import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.*;
 
 import java.io.File;
 
@@ -37,12 +37,7 @@ public class NDCuboidJobTest extends LocalFileMetadataTestCase {
     @Before
     public void setup() throws Exception {
         conf = new Configuration();
-        conf.set("fs.default.name", "file:///");
-        conf.set("mapred.job.tracker", "local");
-        conf.set("mapreduce.application.framework.path", "");
-
-        // for local runner out-of-memory issue
-        conf.set("mapreduce.task.io.sort.mb", "10");
+        BaseCuboidJobTest.setLocalMR(conf);
 
         createTestMetadata();
     }

http://git-wip-us.apache.org/repos/asf/kylin/blob/dfdeb92e/job/src/test/java/org/apache/kylin/job/hadoop/cube/RangeKeyDistributionJobTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/org/apache/kylin/job/hadoop/cube/RangeKeyDistributionJobTest.java b/job/src/test/java/org/apache/kylin/job/hadoop/cube/RangeKeyDistributionJobTest.java
index 586bdf0..8e0c3bb 100644
--- a/job/src/test/java/org/apache/kylin/job/hadoop/cube/RangeKeyDistributionJobTest.java
+++ b/job/src/test/java/org/apache/kylin/job/hadoop/cube/RangeKeyDistributionJobTest.java
@@ -41,12 +41,8 @@ public class RangeKeyDistributionJobTest extends LocalFileMetadataTestCase {
     @Before
     public void setup() throws Exception {
         conf = new Configuration();
-        conf.set("fs.default.name", "file:///");
-        conf.set("mapred.job.tracker", "local");
-        conf.set("mapreduce.application.framework.path", "");
+        BaseCuboidJobTest.setLocalMR(conf);
 
-        // for local runner out-of-memory issue
-        conf.set("mapreduce.task.io.sort.mb", "10");
         createTestMetadata();
     }
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/dfdeb92e/job/src/test/java/org/apache/kylin/job/hadoop/hbase/CreateHTableTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/org/apache/kylin/job/hadoop/hbase/CreateHTableTest.java b/job/src/test/java/org/apache/kylin/job/hadoop/hbase/CreateHTableTest.java
index a87c66b..365a0d8 100644
--- a/job/src/test/java/org/apache/kylin/job/hadoop/hbase/CreateHTableTest.java
+++ b/job/src/test/java/org/apache/kylin/job/hadoop/hbase/CreateHTableTest.java
@@ -24,6 +24,7 @@ import static org.junit.Assert.assertEquals;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.kylin.common.util.LocalFileMetadataTestCase;
+import org.apache.kylin.job.hadoop.cube.BaseCuboidJobTest;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -39,9 +40,8 @@ public class CreateHTableTest extends LocalFileMetadataTestCase {
     @Before
     public void setup() throws Exception {
         conf = new Configuration();
-        conf.set("fs.default.name", "file:///");
-        conf.set("mapred.job.tracker", "local");
-        conf.set("mapreduce.application.framework.path", "");
+        BaseCuboidJobTest.setLocalMR(conf);
+
         this.createTestMetadata();
 
     }

http://git-wip-us.apache.org/repos/asf/kylin/blob/dfdeb92e/job/src/test/java/org/apache/kylin/job/tools/ColumnCardinalityJobTest.java
----------------------------------------------------------------------
diff --git a/job/src/test/java/org/apache/kylin/job/tools/ColumnCardinalityJobTest.java b/job/src/test/java/org/apache/kylin/job/tools/ColumnCardinalityJobTest.java
index 9d95a88..2c2f16f 100644
--- a/job/src/test/java/org/apache/kylin/job/tools/ColumnCardinalityJobTest.java
+++ b/job/src/test/java/org/apache/kylin/job/tools/ColumnCardinalityJobTest.java
@@ -27,6 +27,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.util.ToolRunner;
 import org.apache.kylin.job.hadoop.cardinality.HiveColumnCardinalityJob;
+import org.apache.kylin.job.hadoop.cube.BaseCuboidJobTest;
 import org.junit.Before;
 import org.junit.Ignore;
 import org.junit.Test;
@@ -43,9 +44,7 @@ public class ColumnCardinalityJobTest {
     @Before
     public void setup() throws IOException {
         conf = new Configuration();
-        conf.set("fs.default.name", "file:///");
-        conf.set("mapred.job.tracker", "local");
-        conf.set("mapreduce.application.framework.path", "");
+        BaseCuboidJobTest.setLocalMR(conf);
     }
 
     @Test


[18/52] [abbrv] kylin git commit: Add conf/kylin_job_conf.xml for HiveColumnCardinalityJob

Posted by li...@apache.org.
Add conf/kylin_job_conf.xml for HiveColumnCardinalityJob

Signed-off-by: shaofengshi <sh...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/05df7136
Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/05df7136
Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/05df7136

Branch: refs/heads/1.x-HBase1.1.3
Commit: 05df71369f53673aea6134dd74138e7c11edeac0
Parents: b507b44
Author: John Zhao <yu...@ebay.com>
Authored: Tue Feb 2 13:55:28 2016 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Wed Feb 3 10:23:06 2016 +0800

----------------------------------------------------------------------
 .../main/java/org/apache/kylin/job/engine/JobEngineConfig.java | 2 +-
 .../kylin/job/hadoop/cardinality/HiveColumnCardinalityJob.java | 6 ++++++
 2 files changed, 7 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/05df7136/job/src/main/java/org/apache/kylin/job/engine/JobEngineConfig.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/engine/JobEngineConfig.java b/job/src/main/java/org/apache/kylin/job/engine/JobEngineConfig.java
index 75d897e..8edc8a0 100644
--- a/job/src/main/java/org/apache/kylin/job/engine/JobEngineConfig.java
+++ b/job/src/main/java/org/apache/kylin/job/engine/JobEngineConfig.java
@@ -53,7 +53,7 @@ public class JobEngineConfig {
 
     private String getHadoopJobConfFilePath(RealizationCapacity capaticy, boolean appendSuffix) throws IOException {
         String hadoopJobConfFile;
-        if (appendSuffix) {
+        if (capaticy != null && appendSuffix) {
             hadoopJobConfFile = (HADOOP_JOB_CONF_FILENAME + "_" + capaticy.toString().toLowerCase() + ".xml");
         } else {
             hadoopJobConfFile = (HADOOP_JOB_CONF_FILENAME + ".xml");

http://git-wip-us.apache.org/repos/asf/kylin/blob/05df7136/job/src/main/java/org/apache/kylin/job/hadoop/cardinality/HiveColumnCardinalityJob.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/hadoop/cardinality/HiveColumnCardinalityJob.java b/job/src/main/java/org/apache/kylin/job/hadoop/cardinality/HiveColumnCardinalityJob.java
index cf1871f..f27d074 100644
--- a/job/src/main/java/org/apache/kylin/job/hadoop/cardinality/HiveColumnCardinalityJob.java
+++ b/job/src/main/java/org/apache/kylin/job/hadoop/cardinality/HiveColumnCardinalityJob.java
@@ -30,8 +30,10 @@ import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
 import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat;
 import org.apache.hive.hcatalog.mapreduce.HCatInputFormat;
+import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.util.HadoopUtil;
 import org.apache.kylin.job.constant.BatchConstants;
+import org.apache.kylin.job.engine.JobEngineConfig;
 import org.apache.kylin.job.hadoop.AbstractHadoopJob;
 
 /**
@@ -65,6 +67,10 @@ public class HiveColumnCardinalityJob extends AbstractHadoopJob {
             String jobName = JOB_TITLE + getOptionsAsString();
             System.out.println("Starting: " + jobName);
             Configuration conf = getConf();
+
+            JobEngineConfig jobEngineConfig = new JobEngineConfig(KylinConfig.getInstanceFromEnv());
+            conf.addResource(jobEngineConfig.getHadoopJobConfFilePath(null));
+
             job = Job.getInstance(conf, jobName);
 
             setJobClasspath(job);


[02/52] [abbrv] kylin git commit: KYLIN-1343 Compile pass, pending IT

Posted by li...@apache.org.
http://git-wip-us.apache.org/repos/asf/kylin/blob/df7ae17a/atopcalcite/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
----------------------------------------------------------------------
diff --git a/atopcalcite/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java b/atopcalcite/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
index c184f0b..76203e8 100644
--- a/atopcalcite/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
+++ b/atopcalcite/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
@@ -1,13 +1,12 @@
 /*
- * 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
+ * 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
+ * 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,
@@ -15,9 +14,9 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.calcite.sql2rel;
 
+import org.apache.calcite.avatica.util.Spaces;
 import org.apache.calcite.linq4j.Ord;
 import org.apache.calcite.plan.Convention;
 import org.apache.calcite.plan.RelOptCluster;
@@ -25,26 +24,32 @@ import org.apache.calcite.plan.RelOptPlanner;
 import org.apache.calcite.plan.RelOptSamplingParameters;
 import org.apache.calcite.plan.RelOptTable;
 import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.plan.RelTraitSet;
 import org.apache.calcite.prepare.Prepare;
 import org.apache.calcite.prepare.RelOptTableImpl;
 import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelCollationTraitDef;
 import org.apache.calcite.rel.RelCollations;
 import org.apache.calcite.rel.RelFieldCollation;
 import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.RelShuttle;
+import org.apache.calcite.rel.RelRoot;
+import org.apache.calcite.rel.SingleRel;
 import org.apache.calcite.rel.core.Aggregate;
 import org.apache.calcite.rel.core.AggregateCall;
 import org.apache.calcite.rel.core.Collect;
 import org.apache.calcite.rel.core.CorrelationId;
+import org.apache.calcite.rel.core.Filter;
 import org.apache.calcite.rel.core.Join;
 import org.apache.calcite.rel.core.JoinInfo;
 import org.apache.calcite.rel.core.JoinRelType;
 import org.apache.calcite.rel.core.Project;
 import org.apache.calcite.rel.core.RelFactories;
 import org.apache.calcite.rel.core.Sample;
+import org.apache.calcite.rel.core.Sort;
 import org.apache.calcite.rel.core.Uncollect;
 import org.apache.calcite.rel.logical.LogicalAggregate;
 import org.apache.calcite.rel.logical.LogicalCorrelate;
+import org.apache.calcite.rel.logical.LogicalFilter;
 import org.apache.calcite.rel.logical.LogicalIntersect;
 import org.apache.calcite.rel.logical.LogicalJoin;
 import org.apache.calcite.rel.logical.LogicalMinus;
@@ -56,6 +61,7 @@ import org.apache.calcite.rel.logical.LogicalTableScan;
 import org.apache.calcite.rel.logical.LogicalUnion;
 import org.apache.calcite.rel.logical.LogicalValues;
 import org.apache.calcite.rel.metadata.RelColumnMapping;
+import org.apache.calcite.rel.stream.Delta;
 import org.apache.calcite.rel.stream.LogicalDelta;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
@@ -72,8 +78,8 @@ import org.apache.calcite.rex.RexLiteral;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexRangeRef;
 import org.apache.calcite.rex.RexShuttle;
+import org.apache.calcite.rex.RexSubQuery;
 import org.apache.calcite.rex.RexUtil;
-import org.apache.calcite.rex.RexVisitorImpl;
 import org.apache.calcite.rex.RexWindowBound;
 import org.apache.calcite.schema.ModifiableTable;
 import org.apache.calcite.schema.ModifiableView;
@@ -85,6 +91,7 @@ import org.apache.calcite.sql.SemiJoinType;
 import org.apache.calcite.sql.SqlAggFunction;
 import org.apache.calcite.sql.SqlBasicCall;
 import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlCallBinding;
 import org.apache.calcite.sql.SqlDataTypeSpec;
 import org.apache.calcite.sql.SqlDelete;
 import org.apache.calcite.sql.SqlDynamicParam;
@@ -102,10 +109,12 @@ import org.apache.calcite.sql.SqlNodeList;
 import org.apache.calcite.sql.SqlNumericLiteral;
 import org.apache.calcite.sql.SqlOperator;
 import org.apache.calcite.sql.SqlOperatorTable;
+import org.apache.calcite.sql.SqlOrderBy;
 import org.apache.calcite.sql.SqlSampleSpec;
 import org.apache.calcite.sql.SqlSelect;
 import org.apache.calcite.sql.SqlSelectKeyword;
 import org.apache.calcite.sql.SqlSetOperator;
+import org.apache.calcite.sql.SqlUnnestOperator;
 import org.apache.calcite.sql.SqlUpdate;
 import org.apache.calcite.sql.SqlUtil;
 import org.apache.calcite.sql.SqlValuesOperator;
@@ -138,8 +147,10 @@ import org.apache.calcite.sql.validate.SqlValidatorImpl;
 import org.apache.calcite.sql.validate.SqlValidatorNamespace;
 import org.apache.calcite.sql.validate.SqlValidatorScope;
 import org.apache.calcite.sql.validate.SqlValidatorUtil;
+import org.apache.calcite.tools.RelBuilder;
 import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.ImmutableIntList;
+import org.apache.calcite.util.Litmus;
 import org.apache.calcite.util.NlsString;
 import org.apache.calcite.util.NumberUtil;
 import org.apache.calcite.util.Pair;
@@ -158,15 +169,16 @@ import com.google.common.collect.Sets;
 import java.lang.reflect.Type;
 import java.math.BigDecimal;
 import java.util.AbstractList;
+import java.util.ArrayDeque;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.Deque;
 import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
-import java.util.Stack;
 import java.util.TreeSet;
 import java.util.logging.Level;
 import java.util.logging.Logger;
@@ -178,7 +190,7 @@ import static org.apache.calcite.util.Static.RESOURCE;
  * OVERRIDE POINT:
  * - getInSubqueryThreshold(), was `20`, now `Integer.MAX_VALUE`
  * - isTrimUnusedFields(), override to false
- * - AggConverter.visit(SqlCall), skip column reading for COUNT(COL), for https://jirap.corp.ebay.com/browse/KYLIN-104
+ * - AggConverter.translateAgg(...), skip column reading for COUNT(COL), for https://jirap.corp.ebay.com/browse/KYLIN-104
  */
 
 /**
@@ -190,3851 +202,4825 @@ import static org.apache.calcite.util.Static.RESOURCE;
  * {@link #convertExpression(SqlNode)}.
  */
 public class SqlToRelConverter {
-    //~ Static fields/initializers ---------------------------------------------
-
-    protected static final Logger SQL2REL_LOGGER = CalciteTrace.getSqlToRelTracer();
-
-    private static final BigDecimal TWO = BigDecimal.valueOf(2L);
-
-    //~ Instance fields --------------------------------------------------------
-
-    protected final SqlValidator validator;
-    protected final RexBuilder rexBuilder;
-    protected final Prepare.CatalogReader catalogReader;
-    protected final RelOptCluster cluster;
-    private DefaultValueFactory defaultValueFactory;
-    private SubqueryConverter subqueryConverter;
-    protected final List<RelNode> leaves = new ArrayList<>();
-    private final List<SqlDynamicParam> dynamicParamSqlNodes = new ArrayList<>();
-    private final SqlOperatorTable opTab;
-    private boolean shouldConvertTableAccess;
-    protected final RelDataTypeFactory typeFactory;
-    private final SqlNodeToRexConverter exprConverter;
-    private boolean decorrelationEnabled;
-    private boolean trimUnusedFields;
-    private boolean shouldCreateValuesRel;
-    private boolean isExplain;
-    private int nDynamicParamsInExplain;
-
-    /**
-     * Fields used in name resolution for correlated subqueries.
-     */
-    private final Map<String, DeferredLookup> mapCorrelToDeferred = new HashMap<>();
-    private int nextCorrel = 0;
-
-    private static final String CORREL_PREFIX = "$cor";
-
-    /**
-     * Stack of names of datasets requested by the <code>
-     * TABLE(SAMPLE(&lt;datasetName&gt;, &lt;query&gt;))</code> construct.
-     */
-    private final Stack<String> datasetStack = new Stack<>();
-
-    /**
-     * Mapping of non-correlated subqueries that have been converted to their
-     * equivalent constants. Used to avoid re-evaluating the subquery if it's
-     * already been evaluated.
-     */
-    private final Map<SqlNode, RexNode> mapConvertedNonCorrSubqs = new HashMap<>();
-
-    public final RelOptTable.ViewExpander viewExpander;
-
-    //~ Constructors -----------------------------------------------------------
-    /**
-     * Creates a converter.
-     *
-     * @param viewExpander    Preparing statement
-     * @param validator       Validator
-     * @param catalogReader   Schema
-     * @param planner         Planner
-     * @param rexBuilder      Rex builder
-     * @param convertletTable Expression converter
-     */
-    @Deprecated
-    // will be removed before 2.0
-    public SqlToRelConverter(RelOptTable.ViewExpander viewExpander, SqlValidator validator, Prepare.CatalogReader catalogReader, RelOptPlanner planner, RexBuilder rexBuilder, SqlRexConvertletTable convertletTable) {
-        this(viewExpander, validator, catalogReader, RelOptCluster.create(planner, rexBuilder), convertletTable);
-    }
-
-    /* Creates a converter. */
-    public SqlToRelConverter(RelOptTable.ViewExpander viewExpander, SqlValidator validator, Prepare.CatalogReader catalogReader, RelOptCluster cluster, SqlRexConvertletTable convertletTable) {
-        this.viewExpander = viewExpander;
-        this.opTab = (validator == null) ? SqlStdOperatorTable.instance() : validator.getOperatorTable();
-        this.validator = validator;
-        this.catalogReader = catalogReader;
-        this.defaultValueFactory = new NullDefaultValueFactory();
-        this.subqueryConverter = new NoOpSubqueryConverter();
-        this.rexBuilder = cluster.getRexBuilder();
-        this.typeFactory = rexBuilder.getTypeFactory();
-        this.cluster = Preconditions.checkNotNull(cluster);
-        this.shouldConvertTableAccess = true;
-        this.exprConverter = new SqlNodeToRexConverterImpl(convertletTable);
-        decorrelationEnabled = true;
-        trimUnusedFields = false;
-        shouldCreateValuesRel = true;
-        isExplain = false;
-        nDynamicParamsInExplain = 0;
-    }
-
-    //~ Methods ----------------------------------------------------------------
+  //~ Static fields/initializers ---------------------------------------------
+
+  protected static final Logger SQL2REL_LOGGER =
+      CalciteTrace.getSqlToRelTracer();
+
+  private static final BigDecimal TWO = BigDecimal.valueOf(2L);
+
+  /** Size of the smallest IN list that will be converted to a semijoin to a
+   * static table. */
+  public static final int IN_SUBQUERY_THRESHOLD = 20;
+
+  //~ Instance fields --------------------------------------------------------
+
+  protected final SqlValidator validator;
+  protected final RexBuilder rexBuilder;
+  protected final Prepare.CatalogReader catalogReader;
+  protected final RelOptCluster cluster;
+  private DefaultValueFactory defaultValueFactory;
+  private SubqueryConverter subqueryConverter;
+  protected final List<RelNode> leaves = new ArrayList<>();
+  private final List<SqlDynamicParam> dynamicParamSqlNodes = new ArrayList<>();
+  private final SqlOperatorTable opTab;
+  private boolean shouldConvertTableAccess;
+  protected final RelDataTypeFactory typeFactory;
+  private final SqlNodeToRexConverter exprConverter;
+  private boolean decorrelationEnabled;
+  private boolean trimUnusedFields;
+  private boolean shouldCreateValuesRel;
+  private boolean isExplain;
+  private int nDynamicParamsInExplain;
+
+  /**
+   * Fields used in name resolution for correlated subqueries.
+   */
+  private final Map<CorrelationId, DeferredLookup> mapCorrelToDeferred =
+      new HashMap<>();
+
+  /**
+   * Stack of names of datasets requested by the <code>
+   * TABLE(SAMPLE(&lt;datasetName&gt;, &lt;query&gt;))</code> construct.
+   */
+  private final Deque<String> datasetStack = new ArrayDeque<>();
+
+  /**
+   * Mapping of non-correlated subqueries that have been converted to their
+   * equivalent constants. Used to avoid re-evaluating the subquery if it's
+   * already been evaluated.
+   */
+  private final Map<SqlNode, RexNode> mapConvertedNonCorrSubqs =
+      new HashMap<>();
+
+  public final RelOptTable.ViewExpander viewExpander;
+
+  /** Whether to expand sub-queries. If false, each sub-query becomes a
+   * {@link org.apache.calcite.rex.RexSubQuery}. */
+  private boolean expand = true;
+
+  //~ Constructors -----------------------------------------------------------
+  /**
+   * Creates a converter.
+   *
+   * @param viewExpander    Preparing statement
+   * @param validator       Validator
+   * @param catalogReader   Schema
+   * @param planner         Planner
+   * @param rexBuilder      Rex builder
+   * @param convertletTable Expression converter
+   */
+  @Deprecated // will be removed before 2.0
+  public SqlToRelConverter(
+      RelOptTable.ViewExpander viewExpander,
+      SqlValidator validator,
+      Prepare.CatalogReader catalogReader,
+      RelOptPlanner planner,
+      RexBuilder rexBuilder,
+      SqlRexConvertletTable convertletTable) {
+    this(viewExpander, validator, catalogReader,
+        RelOptCluster.create(planner, rexBuilder), convertletTable);
+  }
+
+  /* Creates a converter. */
+  public SqlToRelConverter(
+      RelOptTable.ViewExpander viewExpander,
+      SqlValidator validator,
+      Prepare.CatalogReader catalogReader,
+      RelOptCluster cluster,
+      SqlRexConvertletTable convertletTable) {
+    this.viewExpander = viewExpander;
+    this.opTab =
+        (validator
+            == null) ? SqlStdOperatorTable.instance()
+            : validator.getOperatorTable();
+    this.validator = validator;
+    this.catalogReader = catalogReader;
+    this.defaultValueFactory = new NullDefaultValueFactory();
+    this.subqueryConverter = new NoOpSubqueryConverter();
+    this.rexBuilder = cluster.getRexBuilder();
+    this.typeFactory = rexBuilder.getTypeFactory();
+    this.cluster = Preconditions.checkNotNull(cluster);
+    this.shouldConvertTableAccess = true;
+    this.exprConverter =
+        new SqlNodeToRexConverterImpl(convertletTable);
+    decorrelationEnabled = true;
+    trimUnusedFields = false;
+    shouldCreateValuesRel = true;
+    isExplain = false;
+    nDynamicParamsInExplain = 0;
+  }
+
+  //~ Methods ----------------------------------------------------------------
+
+  /**
+   * @return the RelOptCluster in use.
+   */
+  public RelOptCluster getCluster() {
+    return cluster;
+  }
+
+  /**
+   * Returns the row-expression builder.
+   */
+  public RexBuilder getRexBuilder() {
+    return rexBuilder;
+  }
+
+  /**
+   * Returns the number of dynamic parameters encountered during translation;
+   * this must only be called after {@link #convertQuery}.
+   *
+   * @return number of dynamic parameters
+   */
+  public int getDynamicParamCount() {
+    return dynamicParamSqlNodes.size();
+  }
+
+  /**
+   * Returns the type inferred for a dynamic parameter.
+   *
+   * @param index 0-based index of dynamic parameter
+   * @return inferred type, never null
+   */
+  public RelDataType getDynamicParamType(int index) {
+    SqlNode sqlNode = dynamicParamSqlNodes.get(index);
+    if (sqlNode == null) {
+      throw Util.needToImplement("dynamic param type inference");
+    }
+    return validator.getValidatedNodeType(sqlNode);
+  }
+
+  /**
+   * Returns the current count of the number of dynamic parameters in an
+   * EXPLAIN PLAN statement.
+   *
+   * @param increment if true, increment the count
+   * @return the current count before the optional increment
+   */
+  public int getDynamicParamCountInExplain(boolean increment) {
+    int retVal = nDynamicParamsInExplain;
+    if (increment) {
+      ++nDynamicParamsInExplain;
+    }
+    return retVal;
+  }
+
+  /**
+   * @return mapping of non-correlated subqueries that have been converted to
+   * the constants that they evaluate to
+   */
+  public Map<SqlNode, RexNode> getMapConvertedNonCorrSubqs() {
+    return mapConvertedNonCorrSubqs;
+  }
+
+  /**
+   * Adds to the current map of non-correlated converted subqueries the
+   * elements from another map that contains non-correlated subqueries that
+   * have been converted by another SqlToRelConverter.
+   *
+   * @param alreadyConvertedNonCorrSubqs the other map
+   */
+  public void addConvertedNonCorrSubqs(
+      Map<SqlNode, RexNode> alreadyConvertedNonCorrSubqs) {
+    mapConvertedNonCorrSubqs.putAll(alreadyConvertedNonCorrSubqs);
+  }
+
+  /**
+   * Set a new DefaultValueFactory. To have any effect, this must be called
+   * before any convert method.
+   *
+   * @param factory new DefaultValueFactory
+   */
+  public void setDefaultValueFactory(DefaultValueFactory factory) {
+    defaultValueFactory = factory;
+  }
+
+  /**
+   * Sets a new SubqueryConverter. To have any effect, this must be called
+   * before any convert method.
+   *
+   * @param converter new SubqueryConverter
+   */
+  public void setSubqueryConverter(SubqueryConverter converter) {
+    subqueryConverter = converter;
+  }
+
+  /**
+   * Indicates that the current statement is part of an EXPLAIN PLAN statement
+   *
+   * @param nDynamicParams number of dynamic parameters in the statement
+   */
+  public void setIsExplain(int nDynamicParams) {
+    isExplain = true;
+    nDynamicParamsInExplain = nDynamicParams;
+  }
+
+  /**
+   * Controls whether table access references are converted to physical rels
+   * immediately. The optimizer doesn't like leaf rels to have
+   * {@link Convention#NONE}. However, if we are doing further conversion
+   * passes (e.g. {@link RelStructuredTypeFlattener}), then we may need to
+   * defer conversion. To have any effect, this must be called before any
+   * convert method.
+   *
+   * @param enabled true for immediate conversion (the default); false to
+   *                generate logical LogicalTableScan instances
+   */
+  public void enableTableAccessConversion(boolean enabled) {
+    shouldConvertTableAccess = enabled;
+  }
+
+  /**
+   * Controls whether instances of
+   * {@link org.apache.calcite.rel.logical.LogicalValues} are generated. These
+   * may not be supported by all physical implementations. To have any effect,
+   * this must be called before any convert method.
+   *
+   * @param enabled true to allow LogicalValues to be generated (the default);
+   *                false to force substitution of Project+OneRow instead
+   */
+  public void enableValuesRelCreation(boolean enabled) {
+    shouldCreateValuesRel = enabled;
+  }
+
+  private void checkConvertedType(SqlNode query, RelNode result) {
+    if (query.isA(SqlKind.DML)) {
+      return;
+    }
+    // Verify that conversion from SQL to relational algebra did
+    // not perturb any type information.  (We can't do this if the
+    // SQL statement is something like an INSERT which has no
+    // validator type information associated with its result,
+    // hence the namespace check above.)
+    final List<RelDataTypeField> validatedFields =
+        validator.getValidatedNodeType(query).getFieldList();
+    final RelDataType validatedRowType =
+        validator.getTypeFactory().createStructType(
+            Pair.right(validatedFields),
+            SqlValidatorUtil.uniquify(Pair.left(validatedFields)));
+
+    final List<RelDataTypeField> convertedFields =
+        result.getRowType().getFieldList().subList(0, validatedFields.size());
+    final RelDataType convertedRowType =
+        validator.getTypeFactory().createStructType(convertedFields);
+
+    if (!RelOptUtil.equal("validated row type", validatedRowType,
+        "converted row type", convertedRowType, Litmus.IGNORE)) {
+      throw new AssertionError("Conversion to relational algebra failed to "
+          + "preserve datatypes:\n"
+          + "validated type:\n"
+          + validatedRowType.getFullTypeString()
+          + "\nconverted type:\n"
+          + convertedRowType.getFullTypeString()
+          + "\nrel:\n"
+          + RelOptUtil.toString(result));
+    }
+  }
+
+  public RelNode flattenTypes(
+      RelNode rootRel,
+      boolean restructure) {
+    RelStructuredTypeFlattener typeFlattener =
+        new RelStructuredTypeFlattener(rexBuilder, createToRelContext());
+    return typeFlattener.rewrite(rootRel, restructure);
+  }
+
+  /**
+   * If subquery is correlated and decorrelation is enabled, performs
+   * decorrelation.
+   *
+   * @param query   Query
+   * @param rootRel Root relational expression
+   * @return New root relational expression after decorrelation
+   */
+  public RelNode decorrelate(SqlNode query, RelNode rootRel) {
+    if (!enableDecorrelation()) {
+      return rootRel;
+    }
+    final RelNode result = decorrelateQuery(rootRel);
+    if (result != rootRel) {
+      checkConvertedType(query, result);
+    }
+    return result;
+  }
+
+  /**
+   * Walks over a tree of relational expressions, replacing each
+   * {@link RelNode} with a 'slimmed down' relational expression that projects
+   * only the fields required by its consumer.
+   *
+   * <p>This may make things easier for the optimizer, by removing crud that
+   * would expand the search space, but is difficult for the optimizer itself
+   * to do it, because optimizer rules must preserve the number and type of
+   * fields. Hence, this transform that operates on the entire tree, similar
+   * to the {@link RelStructuredTypeFlattener type-flattening transform}.
+   *
+   * <p>Currently this functionality is disabled in farrago/luciddb; the
+   * default implementation of this method does nothing.
+   *
+   * @param ordered Whether the relational expression must produce results in
+   * a particular order (typically because it has an ORDER BY at top level)
+   * @param rootRel Relational expression that is at the root of the tree
+   * @return Trimmed relational expression
+   */
+  public RelNode trimUnusedFields(boolean ordered, RelNode rootRel) {
+    // Trim fields that are not used by their consumer.
+    if (isTrimUnusedFields()) {
+      final RelFieldTrimmer trimmer = newFieldTrimmer();
+      final List<RelCollation> collations =
+          rootRel.getTraitSet().getTraits(RelCollationTraitDef.INSTANCE);
+      rootRel = trimmer.trim(rootRel);
+      if (!ordered
+          && collations != null
+          && !collations.isEmpty()
+          && !collations.equals(ImmutableList.of(RelCollations.EMPTY))) {
+        final RelTraitSet traitSet = rootRel.getTraitSet()
+            .replace(RelCollationTraitDef.INSTANCE, collations);
+        rootRel = rootRel.copy(traitSet, rootRel.getInputs());
+      }
+      boolean dumpPlan = SQL2REL_LOGGER.isLoggable(Level.FINE);
+      if (dumpPlan) {
+        SQL2REL_LOGGER.fine(
+            RelOptUtil.dumpPlan(
+                "Plan after trimming unused fields",
+                rootRel,
+                false,
+                SqlExplainLevel.EXPPLAN_ATTRIBUTES));
+      }
+    }
+    return rootRel;
+  }
+
+  /**
+   * Creates a RelFieldTrimmer.
+   *
+   * @return Field trimmer
+   */
+  protected RelFieldTrimmer newFieldTrimmer() {
+    final RelBuilder relBuilder =
+        RelFactories.LOGICAL_BUILDER.create(cluster, null);
+    return new RelFieldTrimmer(validator, relBuilder);
+  }
+
+  /**
+   * Converts an unvalidated query's parse tree into a relational expression.
+   *
+   * @param query           Query to convert
+   * @param needsValidation Whether to validate the query before converting;
+   *                        <code>false</code> if the query has already been
+   *                        validated.
+   * @param top             Whether the query is top-level, say if its result
+   *                        will become a JDBC result set; <code>false</code> if
+   *                        the query will be part of a view.
+   */
+  public RelRoot convertQuery(
+      SqlNode query,
+      final boolean needsValidation,
+      final boolean top) {
+    if (needsValidation) {
+      query = validator.validate(query);
+    }
 
-    /**
-     * @return the RelOptCluster in use.
-     */
-    public RelOptCluster getCluster() {
-        return cluster;
+    RelNode result = convertQueryRecursive(query, top, null).rel;
+    if (top) {
+      if (isStream(query)) {
+        result = new LogicalDelta(cluster, result.getTraitSet(), result);
+      }
+    }
+    RelCollation collation = RelCollations.EMPTY;
+    if (!query.isA(SqlKind.DML)) {
+      if (isOrdered(query)) {
+        collation = requiredCollation(result);
+      }
+    }
+    checkConvertedType(query, result);
+
+    boolean dumpPlan = SQL2REL_LOGGER.isLoggable(Level.FINE);
+    if (dumpPlan) {
+      SQL2REL_LOGGER.fine(
+          RelOptUtil.dumpPlan(
+              "Plan after converting SqlNode to RelNode",
+              result,
+              false,
+              SqlExplainLevel.EXPPLAN_ATTRIBUTES));
     }
 
-    /**
-     * Returns the row-expression builder.
-     */
-    public RexBuilder getRexBuilder() {
-        return rexBuilder;
+    final RelDataType validatedRowType = validator.getValidatedNodeType(query);
+    return RelRoot.of(result, validatedRowType, query.getKind())
+        .withCollation(collation);
+  }
+
+  private static boolean isStream(SqlNode query) {
+    return query instanceof SqlSelect
+        && ((SqlSelect) query).isKeywordPresent(SqlSelectKeyword.STREAM);
+  }
+
+  public static boolean isOrdered(SqlNode query) {
+    switch (query.getKind()) {
+    case SELECT:
+      return ((SqlSelect) query).getOrderList() != null
+          && ((SqlSelect) query).getOrderList().size() > 0;
+    case WITH:
+      return isOrdered(((SqlWith) query).body);
+    case ORDER_BY:
+      return ((SqlOrderBy) query).orderList.size() > 0;
+    default:
+      return false;
     }
+  }
 
-    /**
-     * Returns the number of dynamic parameters encountered during translation;
-     * this must only be called after {@link #convertQuery}.
-     *
-     * @return number of dynamic parameters
-     */
-    public int getDynamicParamCount() {
-        return dynamicParamSqlNodes.size();
+  private RelCollation requiredCollation(RelNode r) {
+    if (r instanceof Sort) {
+      return ((Sort) r).collation;
+    }
+    if (r instanceof Project) {
+      return requiredCollation(((Project) r).getInput());
+    }
+    if (r instanceof Delta) {
+      return requiredCollation(((Delta) r).getInput());
+    }
+    throw new AssertionError();
+  }
+
+  /**
+   * Converts a SELECT statement's parse tree into a relational expression.
+   */
+  public RelNode convertSelect(SqlSelect select, boolean top) {
+    final SqlValidatorScope selectScope = validator.getWhereScope(select);
+    final Blackboard bb = createBlackboard(selectScope, null, top);
+    convertSelectImpl(bb, select);
+    return bb.root;
+  }
+
+  /**
+   * Factory method for creating translation workspace.
+   */
+  protected Blackboard createBlackboard(SqlValidatorScope scope,
+      Map<String, RexNode> nameToNodeMap, boolean top) {
+    return new Blackboard(scope, nameToNodeMap, top);
+  }
+
+  /**
+   * Implementation of {@link #convertSelect(SqlSelect, boolean)};
+   * derived class may override.
+   */
+  protected void convertSelectImpl(
+      final Blackboard bb,
+      SqlSelect select) {
+    convertFrom(
+        bb,
+        select.getFrom());
+    convertWhere(
+        bb,
+        select.getWhere());
+
+    final List<SqlNode> orderExprList = new ArrayList<>();
+    final List<RelFieldCollation> collationList = new ArrayList<>();
+    gatherOrderExprs(
+        bb,
+        select,
+        select.getOrderList(),
+        orderExprList,
+        collationList);
+    final RelCollation collation =
+        cluster.traitSet().canonize(RelCollations.of(collationList));
+
+    if (validator.isAggregate(select)) {
+      convertAgg(
+          bb,
+          select,
+          orderExprList);
+    } else {
+      convertSelectList(
+          bb,
+          select,
+          orderExprList);
     }
 
-    /**
-     * Returns the type inferred for a dynamic parameter.
-     *
-     * @param index 0-based index of dynamic parameter
-     * @return inferred type, never null
-     */
-    public RelDataType getDynamicParamType(int index) {
-        SqlNode sqlNode = dynamicParamSqlNodes.get(index);
-        if (sqlNode == null) {
-            throw Util.needToImplement("dynamic param type inference");
-        }
-        return validator.getValidatedNodeType(sqlNode);
+    if (select.isDistinct()) {
+      distinctify(bb, true);
+    }
+    convertOrder(
+        select, bb, collation, orderExprList, select.getOffset(),
+        select.getFetch());
+    bb.setRoot(bb.root, true);
+  }
+
+  /**
+   * Having translated 'SELECT ... FROM ... [GROUP BY ...] [HAVING ...]', adds
+   * a relational expression to make the results unique.
+   *
+   * <p>If the SELECT clause contains duplicate expressions, adds
+   * {@link org.apache.calcite.rel.logical.LogicalProject}s so that we are
+   * grouping on the minimal set of keys. The performance gain isn't huge, but
+   * it is difficult to detect these duplicate expressions later.
+   *
+   * @param bb               Blackboard
+   * @param checkForDupExprs Check for duplicate expressions
+   */
+  private void distinctify(
+      Blackboard bb,
+      boolean checkForDupExprs) {
+    // Look for duplicate expressions in the project.
+    // Say we have 'select x, y, x, z'.
+    // Then dups will be {[2, 0]}
+    // and oldToNew will be {[0, 0], [1, 1], [2, 0], [3, 2]}
+    RelNode rel = bb.root;
+    if (checkForDupExprs && (rel instanceof LogicalProject)) {
+      LogicalProject project = (LogicalProject) rel;
+      final List<RexNode> projectExprs = project.getProjects();
+      final List<Integer> origins = new ArrayList<>();
+      int dupCount = 0;
+      for (int i = 0; i < projectExprs.size(); i++) {
+        int x = findExpr(projectExprs.get(i), projectExprs, i);
+        if (x >= 0) {
+          origins.add(x);
+          ++dupCount;
+        } else {
+          origins.add(i);
+        }
+      }
+      if (dupCount == 0) {
+        distinctify(bb, false);
+        return;
+      }
+
+      final Map<Integer, Integer> squished = Maps.newHashMap();
+      final List<RelDataTypeField> fields = rel.getRowType().getFieldList();
+      final List<Pair<RexNode, String>> newProjects = Lists.newArrayList();
+      for (int i = 0; i < fields.size(); i++) {
+        if (origins.get(i) == i) {
+          squished.put(i, newProjects.size());
+          newProjects.add(RexInputRef.of2(i, fields));
+        }
+      }
+      rel =
+          LogicalProject.create(rel, Pair.left(newProjects),
+              Pair.right(newProjects));
+      bb.root = rel;
+      distinctify(bb, false);
+      rel = bb.root;
+
+      // Create the expressions to reverse the mapping.
+      // Project($0, $1, $0, $2).
+      final List<Pair<RexNode, String>> undoProjects = Lists.newArrayList();
+      for (int i = 0; i < fields.size(); i++) {
+        final int origin = origins.get(i);
+        RelDataTypeField field = fields.get(i);
+        undoProjects.add(
+            Pair.of(
+                (RexNode) new RexInputRef(
+                    squished.get(origin), field.getType()),
+                field.getName()));
+      }
+
+      rel =
+          LogicalProject.create(rel, Pair.left(undoProjects),
+              Pair.right(undoProjects));
+      bb.setRoot(
+          rel,
+          false);
+
+      return;
     }
 
-    /**
-     * Returns the current count of the number of dynamic parameters in an
-     * EXPLAIN PLAN statement.
-     *
-     * @param increment if true, increment the count
-     * @return the current count before the optional increment
-     */
-    public int getDynamicParamCountInExplain(boolean increment) {
-        int retVal = nDynamicParamsInExplain;
-        if (increment) {
-            ++nDynamicParamsInExplain;
-        }
-        return retVal;
+    // Usual case: all of the expressions in the SELECT clause are
+    // different.
+    final ImmutableBitSet groupSet =
+        ImmutableBitSet.range(rel.getRowType().getFieldCount());
+    rel =
+        createAggregate(bb, false, groupSet, ImmutableList.of(groupSet),
+            ImmutableList.<AggregateCall>of());
+
+    bb.setRoot(
+        rel,
+        false);
+  }
+
+  private int findExpr(RexNode seek, List<RexNode> exprs, int count) {
+    for (int i = 0; i < count; i++) {
+      RexNode expr = exprs.get(i);
+      if (expr.toString().equals(seek.toString())) {
+        return i;
+      }
+    }
+    return -1;
+  }
+
+  /**
+   * Converts a query's ORDER BY clause, if any.
+   *
+   * @param select        Query
+   * @param bb            Blackboard
+   * @param collation     Collation list
+   * @param orderExprList Method populates this list with orderBy expressions
+   *                      not present in selectList
+   * @param offset        Expression for number of rows to discard before
+   *                      returning first row
+   * @param fetch         Expression for number of rows to fetch
+   */
+  protected void convertOrder(
+      SqlSelect select,
+      Blackboard bb,
+      RelCollation collation,
+      List<SqlNode> orderExprList,
+      SqlNode offset,
+      SqlNode fetch) {
+    if (select.getOrderList() == null
+        || select.getOrderList().getList().isEmpty()) {
+      assert collation.getFieldCollations().isEmpty();
+      if ((offset == null
+            || ((SqlLiteral) offset).bigDecimalValue().equals(BigDecimal.ZERO))
+          && fetch == null) {
+        return;
+      }
     }
 
-    /**
-     * @return mapping of non-correlated subqueries that have been converted to
-     * the constants that they evaluate to
-     */
-    public Map<SqlNode, RexNode> getMapConvertedNonCorrSubqs() {
-        return mapConvertedNonCorrSubqs;
+    // Create a sorter using the previously constructed collations.
+    bb.setRoot(
+        LogicalSort.create(bb.root, collation,
+            offset == null ? null : convertExpression(offset),
+            fetch == null ? null : convertExpression(fetch)),
+        false);
+
+    // If extra expressions were added to the project list for sorting,
+    // add another project to remove them. But make the collation empty, because
+    // we can't represent the real collation.
+    //
+    // If it is the top node, use the real collation, but don't trim fields.
+    if (orderExprList.size() > 0 && !bb.top) {
+      final List<RexNode> exprs = new ArrayList<>();
+      final RelDataType rowType = bb.root.getRowType();
+      final int fieldCount =
+          rowType.getFieldCount() - orderExprList.size();
+      for (int i = 0; i < fieldCount; i++) {
+        exprs.add(rexBuilder.makeInputRef(bb.root, i));
+      }
+      bb.setRoot(
+          LogicalProject.create(bb.root, exprs,
+              rowType.getFieldNames().subList(0, fieldCount)),
+          false);
+    }
+  }
+
+  /**
+   * Returns whether a given node contains a {@link SqlInOperator}.
+   *
+   * @param node a RexNode tree
+   */
+  private static boolean containsInOperator(
+      SqlNode node) {
+    try {
+      SqlVisitor<Void> visitor =
+          new SqlBasicVisitor<Void>() {
+            public Void visit(SqlCall call) {
+              if (call.getOperator() instanceof SqlInOperator) {
+                throw new Util.FoundOne(call);
+              }
+              return super.visit(call);
+            }
+          };
+      node.accept(visitor);
+      return false;
+    } catch (Util.FoundOne e) {
+      Util.swallow(e, null);
+      return true;
+    }
+  }
+
+  /**
+   * Push down all the NOT logical operators into any IN/NOT IN operators.
+   *
+   * @param sqlNode the root node from which to look for NOT operators
+   * @return the transformed SqlNode representation with NOT pushed down.
+   */
+  private static SqlNode pushDownNotForIn(SqlNode sqlNode) {
+    if ((sqlNode instanceof SqlCall) && containsInOperator(sqlNode)) {
+      SqlCall sqlCall = (SqlCall) sqlNode;
+      if ((sqlCall.getOperator() == SqlStdOperatorTable.AND)
+          || (sqlCall.getOperator() == SqlStdOperatorTable.OR)) {
+        SqlNode[] sqlOperands = ((SqlBasicCall) sqlCall).operands;
+        for (int i = 0; i < sqlOperands.length; i++) {
+          sqlOperands[i] = pushDownNotForIn(sqlOperands[i]);
+        }
+        return sqlNode;
+      } else if (sqlCall.getOperator() == SqlStdOperatorTable.NOT) {
+        SqlNode childNode = sqlCall.operand(0);
+        assert childNode instanceof SqlCall;
+        SqlBasicCall childSqlCall = (SqlBasicCall) childNode;
+        if (childSqlCall.getOperator() == SqlStdOperatorTable.AND) {
+          SqlNode[] andOperands = childSqlCall.getOperands();
+          SqlNode[] orOperands = new SqlNode[andOperands.length];
+          for (int i = 0; i < orOperands.length; i++) {
+            orOperands[i] =
+                SqlStdOperatorTable.NOT.createCall(
+                    SqlParserPos.ZERO,
+                    andOperands[i]);
+          }
+          for (int i = 0; i < orOperands.length; i++) {
+            orOperands[i] = pushDownNotForIn(orOperands[i]);
+          }
+          return SqlStdOperatorTable.OR.createCall(SqlParserPos.ZERO,
+              orOperands[0], orOperands[1]);
+        } else if (childSqlCall.getOperator() == SqlStdOperatorTable.OR) {
+          SqlNode[] orOperands = childSqlCall.getOperands();
+          SqlNode[] andOperands = new SqlNode[orOperands.length];
+          for (int i = 0; i < andOperands.length; i++) {
+            andOperands[i] =
+                SqlStdOperatorTable.NOT.createCall(
+                    SqlParserPos.ZERO,
+                    orOperands[i]);
+          }
+          for (int i = 0; i < andOperands.length; i++) {
+            andOperands[i] = pushDownNotForIn(andOperands[i]);
+          }
+          return SqlStdOperatorTable.AND.createCall(SqlParserPos.ZERO,
+              andOperands[0], andOperands[1]);
+        } else if (childSqlCall.getOperator() == SqlStdOperatorTable.NOT) {
+          SqlNode[] notOperands = childSqlCall.getOperands();
+          assert notOperands.length == 1;
+          return pushDownNotForIn(notOperands[0]);
+        } else if (childSqlCall.getOperator() instanceof SqlInOperator) {
+          SqlNode[] inOperands = childSqlCall.getOperands();
+          SqlInOperator inOp =
+              (SqlInOperator) childSqlCall.getOperator();
+          if (inOp.isNotIn()) {
+            return SqlStdOperatorTable.IN.createCall(
+                SqlParserPos.ZERO,
+                inOperands[0],
+                inOperands[1]);
+          } else {
+            return SqlStdOperatorTable.NOT_IN.createCall(
+                SqlParserPos.ZERO,
+                inOperands[0],
+                inOperands[1]);
+          }
+        } else {
+          // childSqlCall is "leaf" node in a logical expression tree
+          // (only considering AND, OR, NOT)
+          return sqlNode;
+        }
+      } else {
+        // sqlNode is "leaf" node in a logical expression tree
+        // (only considering AND, OR, NOT)
+        return sqlNode;
+      }
+    } else {
+      // tree rooted at sqlNode does not contain inOperator
+      return sqlNode;
+    }
+  }
+
+  /**
+   * Converts a WHERE clause.
+   *
+   * @param bb    Blackboard
+   * @param where WHERE clause, may be null
+   */
+  private void convertWhere(
+      final Blackboard bb,
+      final SqlNode where) {
+    if (where == null) {
+      return;
     }
+    SqlNode newWhere = pushDownNotForIn(where);
+    replaceSubqueries(bb, newWhere, RelOptUtil.Logic.UNKNOWN_AS_FALSE);
+    final RexNode convertedWhere = bb.convertExpression(newWhere);
 
-    /**
-     * Adds to the current map of non-correlated converted subqueries the
-     * elements from another map that contains non-correlated subqueries that
-     * have been converted by another SqlToRelConverter.
-     *
-     * @param alreadyConvertedNonCorrSubqs the other map
-     */
-    public void addConvertedNonCorrSubqs(Map<SqlNode, RexNode> alreadyConvertedNonCorrSubqs) {
-        mapConvertedNonCorrSubqs.putAll(alreadyConvertedNonCorrSubqs);
+    // only allocate filter if the condition is not TRUE
+    if (convertedWhere.isAlwaysTrue()) {
+      return;
     }
 
-    /**
-     * Set a new DefaultValueFactory. To have any effect, this must be called
-     * before any convert method.
-     *
-     * @param factory new DefaultValueFactory
-     */
-    public void setDefaultValueFactory(DefaultValueFactory factory) {
-        defaultValueFactory = factory;
+    final RelNode filter = RelOptUtil.createFilter(bb.root, convertedWhere);
+    final RelNode r;
+    final CorrelationUse p = getCorrelationUse(bb, filter);
+    if (p != null) {
+      assert p.r instanceof Filter;
+      Filter f = (Filter) p.r;
+      r = LogicalFilter.create(f.getInput(), f.getCondition(),
+          ImmutableSet.of(p.id));
+    } else {
+      r = filter;
     }
 
-    /**
-     * Sets a new SubqueryConverter. To have any effect, this must be called
-     * before any convert method.
-     *
-     * @param converter new SubqueryConverter
-     */
-    public void setSubqueryConverter(SubqueryConverter converter) {
-        subqueryConverter = converter;
+    bb.setRoot(r, false);
+  }
+
+  private void replaceSubqueries(
+      final Blackboard bb,
+      final SqlNode expr,
+      RelOptUtil.Logic logic) {
+    findSubqueries(bb, expr, logic, false);
+    for (SubQuery node : bb.subqueryList) {
+      substituteSubquery(bb, node);
     }
+  }
 
-    /**
-     * Indicates that the current statement is part of an EXPLAIN PLAN statement
-     *
-     * @param nDynamicParams number of dynamic parameters in the statement
-     */
-    public void setIsExplain(int nDynamicParams) {
-        isExplain = true;
-        nDynamicParamsInExplain = nDynamicParams;
+  private void substituteSubquery(Blackboard bb, SubQuery subQuery) {
+    final RexNode expr = subQuery.expr;
+    if (expr != null) {
+      // Already done.
+      return;
     }
 
-    /**
-     * Controls whether table access references are converted to physical rels
-     * immediately. The optimizer doesn't like leaf rels to have
-     * {@link Convention#NONE}. However, if we are doing further conversion
-     * passes (e.g. {@link RelStructuredTypeFlattener}), then we may need to
-     * defer conversion. To have any effect, this must be called before any
-     * convert method.
-     *
-     * @param enabled true for immediate conversion (the default); false to
-     *                generate logical LogicalTableScan instances
-     */
-    public void enableTableAccessConversion(boolean enabled) {
-        shouldConvertTableAccess = enabled;
+    final SqlBasicCall call;
+    final RelNode rel;
+    final SqlNode query;
+    final Pair<RelNode, Boolean> converted;
+    switch (subQuery.node.getKind()) {
+    case CURSOR:
+      convertCursor(bb, subQuery);
+      return;
+
+    case MULTISET_QUERY_CONSTRUCTOR:
+    case MULTISET_VALUE_CONSTRUCTOR:
+    case ARRAY_QUERY_CONSTRUCTOR:
+      rel = convertMultisets(ImmutableList.of(subQuery.node), bb);
+      subQuery.expr = bb.register(rel, JoinRelType.INNER);
+      return;
+
+    case IN:
+      call = (SqlBasicCall) subQuery.node;
+      query = call.operand(1);
+      if (!expand && !(query instanceof SqlNodeList)) {
+        return;
+      }
+      final SqlNode leftKeyNode = call.operand(0);
+
+      final List<RexNode> leftKeys;
+      switch (leftKeyNode.getKind()) {
+      case ROW:
+        leftKeys = Lists.newArrayList();
+        for (SqlNode sqlExpr : ((SqlBasicCall) leftKeyNode).getOperandList()) {
+          leftKeys.add(bb.convertExpression(sqlExpr));
+        }
+        break;
+      default:
+        leftKeys = ImmutableList.of(bb.convertExpression(leftKeyNode));
+      }
+
+      final boolean isNotIn = ((SqlInOperator) call.getOperator()).isNotIn();
+      if (query instanceof SqlNodeList) {
+        SqlNodeList valueList = (SqlNodeList) query;
+        if (!containsNullLiteral(valueList)
+            && valueList.size() < getInSubqueryThreshold()) {
+          // We're under the threshold, so convert to OR.
+          subQuery.expr =
+              convertInToOr(
+                  bb,
+                  leftKeys,
+                  valueList,
+                  isNotIn);
+          return;
+        }
+
+        // Otherwise, let convertExists translate
+        // values list into an inline table for the
+        // reference to Q below.
+      }
+
+      // Project out the search columns from the left side
+
+      //  Q1:
+      // "select from emp where emp.deptno in (select col1 from T)"
+      //
+      // is converted to
+      //
+      // "select from
+      //   emp inner join (select distinct col1 from T)) q
+      //   on emp.deptno = q.col1
+      //
+      // Q2:
+      // "select from emp where emp.deptno not in (Q)"
+      //
+      // is converted to
+      //
+      // "select from
+      //   emp left outer join (select distinct col1, TRUE from T) q
+      //   on emp.deptno = q.col1
+      //   where emp.deptno <> null
+      //         and q.indicator <> TRUE"
+      //
+      final boolean outerJoin = bb.subqueryNeedsOuterJoin
+          || isNotIn
+          || subQuery.logic == RelOptUtil.Logic.TRUE_FALSE_UNKNOWN;
+      final RelDataType targetRowType =
+          SqlTypeUtil.promoteToRowType(typeFactory,
+              validator.getValidatedNodeType(leftKeyNode), null);
+      converted =
+          convertExists(query, RelOptUtil.SubqueryType.IN, subQuery.logic,
+              outerJoin, targetRowType);
+      if (converted.right) {
+        // Generate
+        //    emp CROSS JOIN (SELECT COUNT(*) AS c,
+        //                       COUNT(deptno) AS ck FROM dept)
+        final RelDataType longType =
+            typeFactory.createSqlType(SqlTypeName.BIGINT);
+        final RelNode seek = converted.left.getInput(0); // fragile
+        final int keyCount = leftKeys.size();
+        final List<Integer> args = ImmutableIntList.range(0, keyCount);
+        LogicalAggregate aggregate =
+            LogicalAggregate.create(seek, false, ImmutableBitSet.of(), null,
+                ImmutableList.of(
+                    AggregateCall.create(SqlStdOperatorTable.COUNT, false,
+                        ImmutableList.<Integer>of(), -1, longType, null),
+                    AggregateCall.create(SqlStdOperatorTable.COUNT, false,
+                        args, -1, longType, null)));
+        LogicalJoin join =
+            LogicalJoin.create(bb.root, aggregate, rexBuilder.makeLiteral(true),
+                ImmutableSet.<CorrelationId>of(), JoinRelType.INNER);
+        bb.setRoot(join, false);
+      }
+      RexNode rex =
+          bb.register(converted.left,
+              outerJoin ? JoinRelType.LEFT : JoinRelType.INNER, leftKeys);
+
+      subQuery.expr = translateIn(subQuery, bb.root, rex);
+      if (isNotIn) {
+        subQuery.expr =
+            rexBuilder.makeCall(SqlStdOperatorTable.NOT, subQuery.expr);
+      }
+      return;
+
+    case EXISTS:
+      // "select from emp where exists (select a from T)"
+      //
+      // is converted to the following if the subquery is correlated:
+      //
+      // "select from emp left outer join (select AGG_TRUE() as indicator
+      // from T group by corr_var) q where q.indicator is true"
+      //
+      // If there is no correlation, the expression is replaced with a
+      // boolean indicating whether the subquery returned 0 or >= 1 row.
+      call = (SqlBasicCall) subQuery.node;
+      query = call.operand(0);
+      if (!expand) {
+        return;
+      }
+      converted = convertExists(query, RelOptUtil.SubqueryType.EXISTS,
+          subQuery.logic, true, null);
+      assert !converted.right;
+      if (convertNonCorrelatedSubQuery(subQuery, bb, converted.left, true)) {
+        return;
+      }
+      subQuery.expr = bb.register(converted.left, JoinRelType.LEFT);
+      return;
+
+    case SCALAR_QUERY:
+      // Convert the subquery.  If it's non-correlated, convert it
+      // to a constant expression.
+      if (!expand) {
+        return;
+      }
+      call = (SqlBasicCall) subQuery.node;
+      query = call.operand(0);
+      converted = convertExists(query, RelOptUtil.SubqueryType.SCALAR,
+          subQuery.logic, true, null);
+      assert !converted.right;
+      if (convertNonCorrelatedSubQuery(subQuery, bb, converted.left, false)) {
+        return;
+      }
+      rel = convertToSingleValueSubq(query, converted.left);
+      subQuery.expr = bb.register(rel, JoinRelType.LEFT);
+      return;
+
+    case SELECT:
+      // This is used when converting multiset queries:
+      //
+      // select * from unnest(select multiset[deptno] from emps);
+      //
+      converted = convertExists(subQuery.node, RelOptUtil.SubqueryType.SCALAR,
+          subQuery.logic, true, null);
+      assert !converted.right;
+      subQuery.expr = bb.register(converted.left, JoinRelType.LEFT);
+      return;
+
+    default:
+      throw Util.newInternal("unexpected kind of subquery :" + subQuery.node);
     }
+  }
+
+  private RexNode translateIn(SubQuery subQuery, RelNode root,
+      final RexNode rex) {
+    switch (subQuery.logic) {
+    case TRUE:
+      return rexBuilder.makeLiteral(true);
+
+    case UNKNOWN_AS_FALSE:
+      assert rex instanceof RexRangeRef;
+      final int fieldCount = rex.getType().getFieldCount();
+      RexNode rexNode = rexBuilder.makeFieldAccess(rex, fieldCount - 1);
+      rexNode = rexBuilder.makeCall(SqlStdOperatorTable.IS_TRUE, rexNode);
+
+      // Then append the IS NOT NULL(leftKeysForIn).
+      //
+      // RexRangeRef contains the following fields:
+      //   leftKeysForIn,
+      //   rightKeysForIn (the original subquery select list),
+      //   nullIndicator
+      //
+      // The first two lists contain the same number of fields.
+      final int k = (fieldCount - 1) / 2;
+      for (int i = 0; i < k; i++) {
+        rexNode =
+            rexBuilder.makeCall(
+                SqlStdOperatorTable.AND,
+                rexNode,
+                rexBuilder.makeCall(
+                    SqlStdOperatorTable.IS_NOT_NULL,
+                    rexBuilder.makeFieldAccess(rex, i)));
+      }
+      return rexNode;
+
+    case TRUE_FALSE_UNKNOWN:
+    case UNKNOWN_AS_TRUE:
+      // select e.deptno,
+      //   case
+      //   when ct.c = 0 then false
+      //   when dt.i is not null then true
+      //   when e.deptno is null then null
+      //   when ct.ck < ct.c then null
+      //   else false
+      //   end
+      // from e
+      // cross join (select count(*) as c, count(deptno) as ck from v) as ct
+      // left join (select distinct deptno, true as i from v) as dt
+      //   on e.deptno = dt.deptno
+      final Join join = (Join) root;
+      final Project left = (Project) join.getLeft();
+      final RelNode leftLeft = ((Join) left.getInput()).getLeft();
+      final int leftLeftCount = leftLeft.getRowType().getFieldCount();
+      final RelDataType nullableBooleanType =
+          typeFactory.createTypeWithNullability(
+              typeFactory.createSqlType(SqlTypeName.BOOLEAN), true);
+      final RelDataType longType =
+          typeFactory.createSqlType(SqlTypeName.BIGINT);
+      final RexNode cRef = rexBuilder.makeInputRef(root, leftLeftCount);
+      final RexNode ckRef = rexBuilder.makeInputRef(root, leftLeftCount + 1);
+      final RexNode iRef =
+          rexBuilder.makeInputRef(root, root.getRowType().getFieldCount() - 1);
+
+      final RexLiteral zero =
+          rexBuilder.makeExactLiteral(BigDecimal.ZERO, longType);
+      final RexLiteral trueLiteral = rexBuilder.makeLiteral(true);
+      final RexLiteral falseLiteral = rexBuilder.makeLiteral(false);
+      final RexNode unknownLiteral =
+          rexBuilder.makeNullLiteral(SqlTypeName.BOOLEAN);
+
+      final ImmutableList.Builder<RexNode> args = ImmutableList.builder();
+      args.add(rexBuilder.makeCall(SqlStdOperatorTable.EQUALS, cRef, zero),
+          falseLiteral,
+          rexBuilder.makeCall(SqlStdOperatorTable.IS_NOT_NULL, iRef),
+          trueLiteral);
+      final JoinInfo joinInfo = join.analyzeCondition();
+      for (int leftKey : joinInfo.leftKeys) {
+        final RexNode kRef = rexBuilder.makeInputRef(root, leftKey);
+        args.add(rexBuilder.makeCall(SqlStdOperatorTable.IS_NULL, kRef),
+            unknownLiteral);
+      }
+      args.add(rexBuilder.makeCall(SqlStdOperatorTable.LESS_THAN, ckRef, cRef),
+          unknownLiteral,
+          falseLiteral);
+
+      return rexBuilder.makeCall(
+          nullableBooleanType,
+          SqlStdOperatorTable.CASE,
+          args.build());
+
+    default:
+      throw new AssertionError(subQuery.logic);
+    }
+  }
 
-    /**
-     * Controls whether instances of
-     * {@link org.apache.calcite.rel.logical.LogicalValues} are generated. These
-     * may not be supported by all physical implementations. To have any effect,
-     * this must be called before any convert method.
-     *
-     * @param enabled true to allow LogicalValues to be generated (the default);
-     *                false to force substitution of Project+OneRow instead
-     */
-    public void enableValuesRelCreation(boolean enabled) {
-        shouldCreateValuesRel = enabled;
-    }
-
-    private void checkConvertedType(SqlNode query, RelNode result) {
-        if (!query.isA(SqlKind.DML)) {
-            // Verify that conversion from SQL to relational algebra did
-            // not perturb any type information.  (We can't do this if the
-            // SQL statement is something like an INSERT which has no
-            // validator type information associated with its result,
-            // hence the namespace check above.)
-            RelDataType convertedRowType = result.getRowType();
-            if (!checkConvertedRowType(query, convertedRowType)) {
-                RelDataType validatedRowType = validator.getValidatedNodeType(query);
-                validatedRowType = uniquifyFields(validatedRowType);
-                throw Util.newInternal("Conversion to relational algebra failed to " + "preserve datatypes:\n" + "validated type:\n" + validatedRowType.getFullTypeString() + "\nconverted type:\n" + convertedRowType.getFullTypeString() + "\nrel:\n" + RelOptUtil.toString(result));
-            }
+  private static boolean containsNullLiteral(SqlNodeList valueList) {
+    for (SqlNode node : valueList.getList()) {
+      if (node instanceof SqlLiteral) {
+        SqlLiteral lit = (SqlLiteral) node;
+        if (lit.getValue() == null) {
+          return true;
         }
+      }
     }
-
-    public RelNode flattenTypes(RelNode rootRel, boolean restructure) {
-        RelStructuredTypeFlattener typeFlattener = new RelStructuredTypeFlattener(rexBuilder, createToRelContext());
-        return typeFlattener.rewrite(rootRel, restructure);
+    return false;
+  }
+
+  /**
+   * Determines if a subquery is non-correlated and if so, converts it to a
+   * constant.
+   *
+   * @param subQuery  the call that references the subquery
+   * @param bb        blackboard used to convert the subquery
+   * @param converted RelNode tree corresponding to the subquery
+   * @param isExists  true if the subquery is part of an EXISTS expression
+   * @return if the subquery can be converted to a constant
+   */
+  private boolean convertNonCorrelatedSubQuery(
+      SubQuery subQuery,
+      Blackboard bb,
+      RelNode converted,
+      boolean isExists) {
+    SqlCall call = (SqlBasicCall) subQuery.node;
+    if (subqueryConverter.canConvertSubquery()
+        && isSubQueryNonCorrelated(converted, bb)) {
+      // First check if the subquery has already been converted
+      // because it's a nested subquery.  If so, don't re-evaluate
+      // it again.
+      RexNode constExpr = mapConvertedNonCorrSubqs.get(call);
+      if (constExpr == null) {
+        constExpr =
+            subqueryConverter.convertSubquery(
+                call,
+                this,
+                isExists,
+                isExplain);
+      }
+      if (constExpr != null) {
+        subQuery.expr = constExpr;
+        mapConvertedNonCorrSubqs.put(call, constExpr);
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * Converts the RelNode tree for a select statement to a select that
+   * produces a single value.
+   *
+   * @param query the query
+   * @param plan   the original RelNode tree corresponding to the statement
+   * @return the converted RelNode tree
+   */
+  public RelNode convertToSingleValueSubq(
+      SqlNode query,
+      RelNode plan) {
+    // Check whether query is guaranteed to produce a single value.
+    if (query instanceof SqlSelect) {
+      SqlSelect select = (SqlSelect) query;
+      SqlNodeList selectList = select.getSelectList();
+      SqlNodeList groupList = select.getGroup();
+
+      if ((selectList.size() == 1)
+          && ((groupList == null) || (groupList.size() == 0))) {
+        SqlNode selectExpr = selectList.get(0);
+        if (selectExpr instanceof SqlCall) {
+          SqlCall selectExprCall = (SqlCall) selectExpr;
+          if (Util.isSingleValue(selectExprCall)) {
+            return plan;
+          }
+        }
+
+        // If there is a limit with 0 or 1,
+        // it is ensured to produce a single value
+        if (select.getFetch() != null
+            && select.getFetch() instanceof SqlNumericLiteral) {
+          SqlNumericLiteral limitNum = (SqlNumericLiteral) select.getFetch();
+          if (((BigDecimal) limitNum.getValue()).intValue() < 2) {
+            return plan;
+          }
+        }
+      }
+    } else if (query instanceof SqlCall) {
+      // If the query is (values ...),
+      // it is necessary to look into the operands to determine
+      // whether SingleValueAgg is necessary
+      SqlCall exprCall = (SqlCall) query;
+      if (exprCall.getOperator()
+          instanceof SqlValuesOperator
+              && Util.isSingleValue(exprCall)) {
+        return plan;
+      }
     }
 
-    /**
-     * If subquery is correlated and decorrelation is enabled, performs
-     * decorrelation.
-     *
-     * @param query   Query
-     * @param rootRel Root relational expression
-     * @return New root relational expression after decorrelation
-     */
-    public RelNode decorrelate(SqlNode query, RelNode rootRel) {
-        if (!enableDecorrelation()) {
-            return rootRel;
-        }
-        final RelNode result = decorrelateQuery(rootRel);
-        if (result != rootRel) {
-            checkConvertedType(query, result);
-        }
-        return result;
+    // If not, project SingleValueAgg
+    return RelOptUtil.createSingleValueAggRel(
+        cluster,
+        plan);
+  }
+
+  /**
+   * Converts "x IN (1, 2, ...)" to "x=1 OR x=2 OR ...".
+   *
+   * @param leftKeys   LHS
+   * @param valuesList RHS
+   * @param isNotIn    is this a NOT IN operator
+   * @return converted expression
+   */
+  private RexNode convertInToOr(
+      final Blackboard bb,
+      final List<RexNode> leftKeys,
+      SqlNodeList valuesList,
+      boolean isNotIn) {
+    final List<RexNode> comparisons = new ArrayList<>();
+    for (SqlNode rightVals : valuesList) {
+      RexNode rexComparison;
+      if (leftKeys.size() == 1) {
+        rexComparison =
+            rexBuilder.makeCall(
+                SqlStdOperatorTable.EQUALS,
+                leftKeys.get(0),
+                rexBuilder.ensureType(leftKeys.get(0).getType(),
+                    bb.convertExpression(rightVals), true));
+      } else {
+        assert rightVals instanceof SqlCall;
+        final SqlBasicCall call = (SqlBasicCall) rightVals;
+        assert (call.getOperator() instanceof SqlRowOperator)
+            && call.operandCount() == leftKeys.size();
+        rexComparison =
+            RexUtil.composeConjunction(
+                rexBuilder,
+                Iterables.transform(
+                    Pair.zip(leftKeys, call.getOperandList()),
+                    new Function<Pair<RexNode, SqlNode>, RexNode>() {
+                      public RexNode apply(Pair<RexNode, SqlNode> pair) {
+                        return rexBuilder.makeCall(SqlStdOperatorTable.EQUALS,
+                            pair.left,
+                            rexBuilder.ensureType(pair.left.getType(),
+                                bb.convertExpression(pair.right), true));
+                      }
+                    }),
+                false);
+      }
+      comparisons.add(rexComparison);
     }
 
-    /**
-     * Walks over a tree of relational expressions, replacing each
-     * {@link RelNode} with a 'slimmed down' relational expression that projects
-     * only the fields required by its consumer.
-     *
-     * <p>This may make things easier for the optimizer, by removing crud that
-     * would expand the search space, but is difficult for the optimizer itself
-     * to do it, because optimizer rules must preserve the number and type of
-     * fields. Hence, this transform that operates on the entire tree, similar
-     * to the {@link RelStructuredTypeFlattener type-flattening transform}.
-     *
-     * <p>Currently this functionality is disabled in farrago/luciddb; the
-     * default implementation of this method does nothing.
-     *
-     * @param rootRel Relational expression that is at the root of the tree
-     * @return Trimmed relational expression
-     */
-    public RelNode trimUnusedFields(RelNode rootRel) {
-        // Trim fields that are not used by their consumer.
-        if (isTrimUnusedFields()) {
-            final RelFieldTrimmer trimmer = newFieldTrimmer();
-            rootRel = trimmer.trim(rootRel);
-            boolean dumpPlan = SQL2REL_LOGGER.isLoggable(Level.FINE);
-            if (dumpPlan) {
-                SQL2REL_LOGGER.fine(RelOptUtil.dumpPlan("Plan after trimming unused fields", rootRel, false, SqlExplainLevel.EXPPLAN_ATTRIBUTES));
-            }
-        }
-        return rootRel;
+    RexNode result =
+        RexUtil.composeDisjunction(rexBuilder, comparisons, true);
+    assert result != null;
+
+    if (isNotIn) {
+      result =
+          rexBuilder.makeCall(
+              SqlStdOperatorTable.NOT,
+              result);
     }
 
-    /**
-     * Creates a RelFieldTrimmer.
-     *
-     * @return Field trimmer
-     */
-    protected RelFieldTrimmer newFieldTrimmer() {
-        return new RelFieldTrimmer(validator);
+    return result;
+  }
+
+  /**
+   * Gets the list size threshold under which {@link #convertInToOr} is used.
+   * Lists of this size or greater will instead be converted to use a join
+   * against an inline table
+   * ({@link org.apache.calcite.rel.logical.LogicalValues}) rather than a
+   * predicate. A threshold of 0 forces usage of an inline table in all cases; a
+   * threshold of Integer.MAX_VALUE forces usage of OR in all cases
+   *
+   * @return threshold, default {@link #IN_SUBQUERY_THRESHOLD}
+   */
+  protected int getInSubqueryThreshold() {
+    /* OVERRIDE POINT */
+    return Integer.MAX_VALUE;
+  }
+
+  /**
+   * Converts an EXISTS or IN predicate into a join. For EXISTS, the subquery
+   * produces an indicator variable, and the result is a relational expression
+   * which outer joins that indicator to the original query. After performing
+   * the outer join, the condition will be TRUE if the EXISTS condition holds,
+   * NULL otherwise.
+   *
+   * @param seek           A query, for example 'select * from emp' or
+   *                       'values (1,2,3)' or '('Foo', 34)'.
+   * @param subqueryType   Whether sub-query is IN, EXISTS or scalar
+   * @param logic Whether the answer needs to be in full 3-valued logic (TRUE,
+   *     FALSE, UNKNOWN) will be required, or whether we can accept an
+   *     approximation (say representing UNKNOWN as FALSE)
+   * @param needsOuterJoin Whether an outer join is needed
+   * @return join expression
+   * @pre extraExpr == null || extraName != null
+   */
+  private Pair<RelNode, Boolean> convertExists(
+      SqlNode seek,
+      RelOptUtil.SubqueryType subqueryType,
+      RelOptUtil.Logic logic,
+      boolean needsOuterJoin,
+      RelDataType targetDataType) {
+    final SqlValidatorScope seekScope =
+        (seek instanceof SqlSelect)
+            ? validator.getSelectScope((SqlSelect) seek)
+            : null;
+    final Blackboard seekBb = createBlackboard(seekScope, null, false);
+    RelNode seekRel = convertQueryOrInList(seekBb, seek, targetDataType);
+
+    return RelOptUtil.createExistsPlan(seekRel, subqueryType, logic,
+        needsOuterJoin);
+  }
+
+  private RelNode convertQueryOrInList(
+      Blackboard bb,
+      SqlNode seek,
+      RelDataType targetRowType) {
+    // NOTE: Once we start accepting single-row queries as row constructors,
+    // there will be an ambiguity here for a case like X IN ((SELECT Y FROM
+    // Z)).  The SQL standard resolves the ambiguity by saying that a lone
+    // select should be interpreted as a table expression, not a row
+    // expression.  The semantic difference is that a table expression can
+    // return multiple rows.
+    if (seek instanceof SqlNodeList) {
+      return convertRowValues(
+          bb,
+          seek,
+          ((SqlNodeList) seek).getList(),
+          false,
+          targetRowType);
+    } else {
+      return convertQueryRecursive(seek, false, null).project();
+    }
+  }
+
+  private RelNode convertRowValues(
+      Blackboard bb,
+      SqlNode rowList,
+      Collection<SqlNode> rows,
+      boolean allowLiteralsOnly,
+      RelDataType targetRowType) {
+    // NOTE jvs 30-Apr-2006: We combine all rows consisting entirely of
+    // literals into a single LogicalValues; this gives the optimizer a smaller
+    // input tree.  For everything else (computed expressions, row
+    // subqueries), we union each row in as a projection on top of a
+    // LogicalOneRow.
+
+    final ImmutableList.Builder<ImmutableList<RexLiteral>> tupleList =
+        ImmutableList.builder();
+    final RelDataType rowType;
+    if (targetRowType != null) {
+      rowType = targetRowType;
+    } else {
+      rowType =
+          SqlTypeUtil.promoteToRowType(
+              typeFactory,
+              validator.getValidatedNodeType(rowList),
+              null);
     }
 
-    /**
-     * Converts an unvalidated query's parse tree into a relational expression.
-     *
-     * @param query           Query to convert
-     * @param needsValidation Whether to validate the query before converting;
-     *                        <code>false</code> if the query has already been
-     *                        validated.
-     * @param top             Whether the query is top-level, say if its result
-     *                        will become a JDBC result set; <code>false</code> if
-     *                        the query will be part of a view.
-     */
-    public RelNode convertQuery(SqlNode query, final boolean needsValidation, final boolean top) {
-        if (needsValidation) {
-            query = validator.validate(query);
+    final List<RelNode> unionInputs = new ArrayList<>();
+    for (SqlNode node : rows) {
+      SqlBasicCall call;
+      if (isRowConstructor(node)) {
+        call = (SqlBasicCall) node;
+        ImmutableList.Builder<RexLiteral> tuple = ImmutableList.builder();
+        for (Ord<SqlNode> operand : Ord.zip(call.operands)) {
+          RexLiteral rexLiteral =
+              convertLiteralInValuesList(
+                  operand.e,
+                  bb,
+                  rowType,
+                  operand.i);
+          if ((rexLiteral == null) && allowLiteralsOnly) {
+            return null;
+          }
+          if ((rexLiteral == null) || !shouldCreateValuesRel) {
+            // fallback to convertRowConstructor
+            tuple = null;
+            break;
+          }
+          tuple.add(rexLiteral);
+        }
+        if (tuple != null) {
+          tupleList.add(tuple.build());
+          continue;
+        }
+      } else {
+        RexLiteral rexLiteral =
+            convertLiteralInValuesList(
+                node,
+                bb,
+                rowType,
+                0);
+        if ((rexLiteral != null) && shouldCreateValuesRel) {
+          tupleList.add(ImmutableList.of(rexLiteral));
+          continue;
+        } else {
+          if ((rexLiteral == null) && allowLiteralsOnly) {
+            return null;
+          }
         }
 
-        RelNode result = convertQueryRecursive(query, top, null);
-        if (top && isStream(query)) {
-            result = new LogicalDelta(cluster, result.getTraitSet(), result);
-        }
-        checkConvertedType(query, result);
+        // convert "1" to "row(1)"
+        call =
+            (SqlBasicCall) SqlStdOperatorTable.ROW.createCall(
+                SqlParserPos.ZERO,
+                node);
+      }
+      unionInputs.add(convertRowConstructor(bb, call));
+    }
+    LogicalValues values =
+        LogicalValues.create(cluster, rowType, tupleList.build());
+    RelNode resultRel;
+    if (unionInputs.isEmpty()) {
+      resultRel = values;
+    } else {
+      if (!values.getTuples().isEmpty()) {
+        unionInputs.add(values);
+      }
+      resultRel = LogicalUnion.create(unionInputs, true);
+    }
+    leaves.add(resultRel);
+    return resultRel;
+  }
+
+  private RexLiteral convertLiteralInValuesList(
+      SqlNode sqlNode,
+      Blackboard bb,
+      RelDataType rowType,
+      int iField) {
+    if (!(sqlNode instanceof SqlLiteral)) {
+      return null;
+    }
+    RelDataTypeField field = rowType.getFieldList().get(iField);
+    RelDataType type = field.getType();
+    if (type.isStruct()) {
+      // null literals for weird stuff like UDT's need
+      // special handling during type flattening, so
+      // don't use LogicalValues for those
+      return null;
+    }
 
-        boolean dumpPlan = SQL2REL_LOGGER.isLoggable(Level.FINE);
-        if (dumpPlan) {
-            SQL2REL_LOGGER.fine(RelOptUtil.dumpPlan("Plan after converting SqlNode to RelNode", result, false, SqlExplainLevel.EXPPLAN_ATTRIBUTES));
-        }
+    RexNode literalExpr =
+        exprConverter.convertLiteral(
+            bb,
+            (SqlLiteral) sqlNode);
 
-        return result;
+    if (!(literalExpr instanceof RexLiteral)) {
+      assert literalExpr.isA(SqlKind.CAST);
+      RexNode child = ((RexCall) literalExpr).getOperands().get(0);
+      assert RexLiteral.isNullLiteral(child);
+
+      // NOTE jvs 22-Nov-2006:  we preserve type info
+      // in LogicalValues digest, so it's OK to lose it here
+      return (RexLiteral) child;
     }
 
-    private static boolean isStream(SqlNode query) {
-        return query instanceof SqlSelect && ((SqlSelect) query).isKeywordPresent(SqlSelectKeyword.STREAM);
+    RexLiteral literal = (RexLiteral) literalExpr;
+
+    Comparable value = literal.getValue();
+
+    if (SqlTypeUtil.isExactNumeric(type) && SqlTypeUtil.hasScale(type)) {
+      BigDecimal roundedValue =
+          NumberUtil.rescaleBigDecimal(
+              (BigDecimal) value,
+              type.getScale());
+      return rexBuilder.makeExactLiteral(
+          roundedValue,
+          type);
     }
 
-    protected boolean checkConvertedRowType(SqlNode query, RelDataType convertedRowType) {
-        RelDataType validatedRowType = validator.getValidatedNodeType(query);
-        validatedRowType = uniquifyFields(validatedRowType);
+    if ((value instanceof NlsString)
+        && (type.getSqlTypeName() == SqlTypeName.CHAR)) {
+      // pad fixed character type
+      NlsString unpadded = (NlsString) value;
+      return rexBuilder.makeCharLiteral(
+          new NlsString(
+              Spaces.padRight(unpadded.getValue(), type.getPrecision()),
+              unpadded.getCharsetName(),
+              unpadded.getCollation()));
+    }
+    return literal;
+  }
 
-        return RelOptUtil.equal("validated row type", validatedRowType, "converted row type", convertedRowType, false);
+  private boolean isRowConstructor(SqlNode node) {
+    if (!(node.getKind() == SqlKind.ROW)) {
+      return false;
+    }
+    SqlCall call = (SqlCall) node;
+    return call.getOperator().getName().equalsIgnoreCase("row");
+  }
+
+  /**
+   * Builds a list of all <code>IN</code> or <code>EXISTS</code> operators
+   * inside SQL parse tree. Does not traverse inside queries.
+   *
+   * @param bb                           blackboard
+   * @param node                         the SQL parse tree
+   * @param logic Whether the answer needs to be in full 3-valued logic (TRUE,
+   *              FALSE, UNKNOWN) will be required, or whether we can accept
+   *              an approximation (say representing UNKNOWN as FALSE)
+   * @param registerOnlyScalarSubqueries if set to true and the parse tree
+   *                                     corresponds to a variation of a select
+   *                                     node, only register it if it's a scalar
+   *                                     subquery
+   */
+  private void findSubqueries(
+      Blackboard bb,
+      SqlNode node,
+      RelOptUtil.Logic logic,
+      boolean registerOnlyScalarSubqueries) {
+    final SqlKind kind = node.getKind();
+    switch (kind) {
+    case EXISTS:
+    case SELECT:
+    case MULTISET_QUERY_CONSTRUCTOR:
+    case MULTISET_VALUE_CONSTRUCTOR:
+    case ARRAY_QUERY_CONSTRUCTOR:
+    case CURSOR:
+    case SCALAR_QUERY:
+      if (!registerOnlyScalarSubqueries
+          || (kind == SqlKind.SCALAR_QUERY)) {
+        bb.registerSubquery(node, RelOptUtil.Logic.TRUE_FALSE);
+      }
+      return;
+    case IN:
+      if (((SqlCall) node).getOperator() == SqlStdOperatorTable.NOT_IN) {
+        logic = logic.negate();
+      }
+      break;
+    case NOT:
+      logic = logic.negate();
+      break;
+    }
+    if (node instanceof SqlCall) {
+      if (kind == SqlKind.OR
+          || kind == SqlKind.NOT) {
+        // It's always correct to outer join subquery with
+        // containing query; however, when predicates involve Or
+        // or NOT, outer join might be necessary.
+        bb.subqueryNeedsOuterJoin = true;
+      }
+      for (SqlNode operand : ((SqlCall) node).getOperandList()) {
+        if (operand != null) {
+          // In the case of an IN expression, locate scalar
+          // subqueries so we can convert them to constants
+          findSubqueries(
+              bb,
+              operand,
+              logic,
+              kind == SqlKind.IN || registerOnlyScalarSubqueries);
+        }
+      }
+    } else if (node instanceof SqlNodeList) {
+      for (SqlNode child : (SqlNodeList) node) {
+        findSubqueries(
+            bb,
+            child,
+            logic,
+            kind == SqlKind.IN || registerOnlyScalarSubqueries);
+      }
     }
 
-    protected RelDataType uniquifyFields(RelDataType rowType) {
-        return validator.getTypeFactory().createStructType(RelOptUtil.getFieldTypeList(rowType), SqlValidatorUtil.uniquify(rowType.getFieldNames()));
+    // Now that we've located any scalar subqueries inside the IN
+    // expression, register the IN expression itself.  We need to
+    // register the scalar subqueries first so they can be converted
+    // before the IN expression is converted.
+    if (kind == SqlKind.IN) {
+      if (logic == RelOptUtil.Logic.TRUE_FALSE_UNKNOWN
+          && !validator.getValidatedNodeType(node).isNullable()) {
+        logic = RelOptUtil.Logic.UNKNOWN_AS_FALSE;
+      }
+      // TODO: This conversion is only valid in the WHERE clause
+      if (logic == RelOptUtil.Logic.UNKNOWN_AS_FALSE
+          && !bb.subqueryNeedsOuterJoin) {
+        logic = RelOptUtil.Logic.TRUE;
+      }
+      bb.registerSubquery(node, logic);
+    }
+  }
+
+  /**
+   * Converts an expression from {@link SqlNode} to {@link RexNode} format.
+   *
+   * @param node Expression to translate
+   * @return Converted expression
+   */
+  public RexNode convertExpression(
+      SqlNode node) {
+    Map<String, RelDataType> nameToTypeMap = Collections.emptyMap();
+    final ParameterScope scope =
+        new ParameterScope((SqlValidatorImpl) validator, nameToTypeMap);
+    final Blackboard bb = createBlackboard(scope, null, false);
+    return bb.convertExpression(node);
+  }
+
+  /**
+   * Converts an expression from {@link SqlNode} to {@link RexNode} format,
+   * mapping identifier references to predefined expressions.
+   *
+   * @param node          Expression to translate
+   * @param nameToNodeMap map from String to {@link RexNode}; when an
+   *                      {@link SqlIdentifier} is encountered, it is used as a
+   *                      key and translated to the corresponding value from
+   *                      this map
+   * @return Converted expression
+   */
+  public RexNode convertExpression(
+      SqlNode node,
+      Map<String, RexNode> nameToNodeMap) {
+    final Map<String, RelDataType> nameToTypeMap = new HashMap<>();
+    for (Map.Entry<String, RexNode> entry : nameToNodeMap.entrySet()) {
+      nameToTypeMap.put(entry.getKey(), entry.getValue().getType());
+    }
+    final ParameterScope scope =
+        new ParameterScope((SqlValidatorImpl) validator, nameToTypeMap);
+    final Blackboard bb = createBlackboard(scope, nameToNodeMap, false);
+    return bb.convertExpression(node);
+  }
+
+  /**
+   * Converts a non-standard expression.
+   *
+   * <p>This method is an extension-point that derived classes can override. If
+   * this method returns a null result, the normal expression translation
+   * process will proceed. The default implementation always returns null.
+   *
+   * @param node Expression
+   * @param bb   Blackboard
+   * @return null to proceed with the usual expression translation process
+   */
+  protected RexNode convertExtendedExpression(
+      SqlNode node,
+      Blackboard bb) {
+    return null;
+  }
+
+  private RexNode convertOver(Blackboard bb, SqlNode node) {
+    SqlCall call = (SqlCall) node;
+    SqlCall aggCall = call.operand(0);
+    SqlNode windowOrRef = call.operand(1);
+    final SqlWindow window =
+        validator.resolveWindow(windowOrRef, bb.scope, true);
+    // ROW_NUMBER() expects specific kind of framing.
+    if (aggCall.getOperator() == SqlStdOperatorTable.ROW_NUMBER) {
+      window.setLowerBound(SqlWindow.createUnboundedPreceding(SqlParserPos.ZERO));
+      window.setUpperBound(SqlWindow.createCurrentRow(SqlParserPos.ZERO));
+      window.setRows(SqlLiteral.createBoolean(true, SqlParserPos.ZERO));
+    }
+    final SqlNodeList partitionList = window.getPartitionList();
+    final ImmutableList.Builder<RexNode> partitionKeys =
+        ImmutableList.builder();
+    for (SqlNode partition : partitionList) {
+      partitionKeys.add(bb.convertExpression(partition));
+    }
+    RexNode lowerBound = bb.convertExpression(window.getLowerBound());
+    RexNode upperBound = bb.convertExpression(window.getUpperBound());
+    SqlNodeList orderList = window.getOrderList();
+    if ((orderList.size() == 0) && !window.isRows()) {
+      // A logical range requires an ORDER BY clause. Use the implicit
+      // ordering of this relation. There must be one, otherwise it would
+      // have failed validation.
+      orderList = bb.scope.getOrderList();
+      if (orderList == null) {
+        throw new AssertionError(
+            "Relation should have sort key for implicit ORDER BY");
+      }
+    }
+    final ImmutableList.Builder<RexFieldCollation> orderKeys =
+        ImmutableList.builder();
+    final Set<SqlKind> flags = EnumSet.noneOf(SqlKind.class);
+    for (SqlNode order : orderList) {
+      flags.clear();
+      RexNode e = bb.convertSortExpression(order, flags);
+      orderKeys.add(new RexFieldCollation(e, flags));
+    }
+    try {
+      Util.permAssert(bb.window == null, "already in window agg mode");
+      bb.window = window;
+      RexNode rexAgg = exprConverter.convertCall(bb, aggCall);
+      rexAgg =
+          rexBuilder.ensureType(
+              validator.getValidatedNodeType(call), rexAgg, false);
+
+      // Walk over the tree and apply 'over' to all agg functions. This is
+      // necessary because the returned expression is not necessarily a call
+      // to an agg function. For example, AVG(x) becomes SUM(x) / COUNT(x).
+      final RexShuttle visitor =
+          new HistogramShuttle(
+              partitionKeys.build(), orderKeys.build(),
+              RexWindowBound.create(window.getLowerBound(), lowerBound),
+              RexWindowBound.create(window.getUpperBound(), upperBound),
+              window);
+      return rexAgg.accept(visitor);
+    } finally {
+      bb.window = null;
+    }
+  }
+
+  /**
+   * Converts a FROM clause into a relational expression.
+   *
+   * @param bb   Scope within which to resolve identifiers
+   * @param from FROM clause of a query. Examples include:
+   *
+   *             <ul>
+   *             <li>a single table ("SALES.EMP"),
+   *             <li>an aliased table ("EMP AS E"),
+   *             <li>a list of tables ("EMP, DEPT"),
+   *             <li>an ANSI Join expression ("EMP JOIN DEPT ON EMP.DEPTNO =
+   *             DEPT.DEPTNO"),
+   *             <li>a VALUES clause ("VALUES ('Fred', 20)"),
+   *             <li>a query ("(SELECT * FROM EMP WHERE GENDER = 'F')"),
+   *             <li>or any combination of the above.
+   *             </ul>
+   */
+  protected void convertFrom(
+      Blackboard bb,
+      SqlNode from) {
+    final SqlCall call;
+    final SqlNode[] operands;
+    switch (from.getKind()) {
+    case AS:
+      convertFrom(bb, ((SqlCall) from).operand(0));
+      return;
+
+    case WITH_ITEM:
+      convertFrom(bb, ((SqlWithItem) from).query);
+      return;
+
+    case WITH:
+      convertFrom(bb, ((SqlWith) from).body);
+      return;
+
+    case TABLESAMPLE:
+      operands = ((SqlBasicCall) from).getOperands();
+      SqlSampleSpec sampleSpec = SqlLiteral.sampleValue(operands[1]);
+      if (sampleSpec instanceof SqlSampleSpec.SqlSubstitutionSampleSpec) {
+        String sampleName =
+            ((SqlSampleSpec.SqlSubstitutionSampleSpec) sampleSpec)
+                .getName();
+        datasetStack.push(sampleName);
+        convertFrom(bb, operands[0]);
+        datasetStack.pop();
+      } else if (sampleSpec instanceof SqlSampleSpec.SqlTableSampleSpec) {
+        SqlSampleSpec.SqlTableSampleSpec tableSampleSpec =
+            (SqlSampleSpec.SqlTableSampleSpec) sampleSpec;
+        convertFrom(bb, operands[0]);
+        RelOptSamplingParameters params =
+            new RelOptSamplingParameters(
+                tableSampleSpec.isBernoulli(),
+                tableSampleSpec.getSamplePercentage(),
+                tableSampleSpec.isRepeatable(),
+                tableSampleSpec.getRepeatableSeed());
+        bb.setRoot(new Sample(cluster, bb.root, params), false);
+      } else {
+        throw Util.newInternal(
+            "unknown TABLESAMPLE type: " + sampleSpec);
+      }
+      return;
+
+    case IDENTIFIER:
+      final SqlValidatorNamespace fromNamespace =
+          validator.getNamespace(from).resolve();
+      if (fromNamespace.getNode() != null) {
+        convertFrom(bb, fromNamespace.getNode());
+        return;
+      }
+      final String datasetName =
+          datasetStack.isEmpty() ? null : datasetStack.peek();
+      boolean[] usedDataset = {false};
+      RelOptTable table =
+          SqlValidatorUtil.getRelOptTable(
+              fromNamespace,
+              catalogReader,
+              datasetName,
+              usedDataset);
+      final RelNode tableRel;
+      if (shouldConvertTableAccess) {
+        tableRel = toRel(table);
+      } else {
+        tableRel = LogicalTableScan.create(cluster, table);
+      }
+      bb.setRoot(tableRel, true);
+      if (usedDataset[0]) {
+        bb.setDataset(datasetName);
+      }
+      return;
+
+    case JOIN:
+      final SqlJoin join = (SqlJoin) from;
+      final SqlValidatorScope scope = validator.getJoinScope(from);
+      final Blackboard fromBlackboard = createBlackboard(scope, null, false);
+      SqlNode left = join.getLeft();
+      SqlNode right = join.getRight();
+      final boolean isNatural = join.isNatural();
+      final JoinType joinType = join.getJoinType();
+      final SqlValidatorScope leftScope =
+          Util.first(validator.getJoinScope(left),
+              ((DelegatingScope) bb.scope).getParent());
+      final Blackboard leftBlackboard =
+          createBlackboard(leftScope, null, false);
+      final SqlValidatorScope rightScope =
+          Util.first(validator.getJoinScope(right),
+              ((DelegatingScope) bb.scope).getParent());
+      final Blackboard rightBlackboard =
+          createBlackboard(rightScope, null, false);
+      convertFrom(leftBlackboard, left);
+      RelNode leftRel = leftBlackboard.root;
+      convertFrom(rightBlackboard, right);
+      RelNode rightRel = rightBlackboard.root;
+      JoinRelType convertedJoinType = convertJoinType(joinType);
+      RexNode conditionExp;
+      final SqlValidatorNamespace leftNamespace = validator.getNamespace(left);
+      final SqlValidatorNamespace rightNamespace = validator.getNamespace(right);
+      if (isNatural) {
+        final RelDataType leftRowType = leftNamespace.getRowType();
+        final RelDataType rightRowType = rightNamespace.getRowType();
+        final List<String> columnList =
+            SqlValidatorUtil.deriveNaturalJoinColumnList(leftRowType,
+                rightRowType);
+        conditionExp = convertUsing(leftNamespace, rightNamespace,
+            columnList);
+      } else {
+        conditionExp =
+            convertJoinCondition(
+                fromBlackboard,
+                leftNamespace,
+                rightNamespace,
+                join.getCondition(),
+                join.getConditionType(),
+                leftRel,
+                rightRel);
+      }
+
+      final RelNode joinRel =
+          createJoin(
+              fromBlackboard,
+              leftRel,
+              rightRel,
+              conditionExp,
+              convertedJoinType);
+      bb.setRoot(joinRel, false);
+      return;
+
+    case SELECT:
+    case INTERSECT:
+    case EXCEPT:
+    case UNION:
+      final RelNode rel = convertQueryRecursive(from, false, null).project();
+      bb.setRoot(rel, true);
+      return;
+
+    case VALUES:
+      convertValuesImpl(bb, (SqlCall) from, null);
+      return;
+
+    case UNNEST:
+      call = (SqlCall) from;
+      final SqlNode node = call.operand(0);
+      final SqlUnnestOperator operator = (SqlUnnestOperator) call.getOperator();
+      replaceSubqueries(bb, node, RelOptUtil.Logic.TRUE_FALSE_UNKNOWN);
+      final RelNode childRel =
+          RelOptUtil.createProject(
+              (null != bb.root) ? bb.root : LogicalValues.createOneRow(cluster),
+              Collections.singletonList(bb.convertExpression(node)),
+              Collections.singletonList(validator.deriveAlias(node, 0)),
+              true);
+
+      Uncollect uncollect =
+          new Uncollect(cluster, cluster.traitSetOf(Convention.NONE),
+              childRel, operator.withOrdinality);
+      bb.setRoot(uncollect, true);
+      return;
+
+    case COLLECTION_TABLE:
+      call = (SqlCall) from;
+
+      // Dig out real call; TABLE() wrapper is just syntactic.
+      assert call.getOperandList().size() == 1;
+      final SqlCall call2 = call.operand(0);
+      convertCollectionTable(bb, call2);
+      return;
+
+    default:
+      throw Util.newInternal("not a join operator " + from);
+    }
+  }
+
+  protected void convertCollectionTable(
+      Blackboard bb,
+      SqlCall call) {
+    final SqlOperator operator = call.getOperator();
+    if (operator == SqlStdOperatorTable.TABLESAMPLE) {
+      final String sampleName =
+          SqlLiteral.stringValue(call.operand(0));
+      datasetStack.push(sampleName);
+      SqlCall cursorCall = call.operand(1);
+      SqlNode query = cursorCall.operand(0);
+      RelNode converted = convertQuery(query, false, false).rel;
+      bb.setRoot(converted, f

<TRUNCATED>

[37/52] [abbrv] kylin git commit: KYLIN-1416 remove website from coding branch

Posted by li...@apache.org.
http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/assets/css/bootstrap.min.css
----------------------------------------------------------------------
diff --git a/website/assets/css/bootstrap.min.css b/website/assets/css/bootstrap.min.css
deleted file mode 100644
index eed8a30..0000000
--- a/website/assets/css/bootstrap.min.css
+++ /dev/null
@@ -1,5083 +0,0 @@
-/*!
- * Bootstrap v3.0.0
- *
- * Copyright 2013 Twitter, Inc
- * Licensed under the Apache License v2.0
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Designed and built with all the love in the world by @mdo and @fat.
- *//*! normalize.css v2.1.0 | MIT License | git.io/normalize */article, aside, details, figcaption, figure, footer, header, hgroup, main, nav, section, summary {
-	display: block
-}
-audio, canvas, video {
-	display: inline-block
-}
-audio:not([controls]) {
-	display: none;
-	height: 0
-}
-[hidden] {
-display:none
-}
-html {
-	font-family: sans-serif;
-	-webkit-text-size-adjust: 100%;
-	-ms-text-size-adjust: 100%
-}
-body {
-	margin: 0
-}
-a:focus {
-	outline: thin dotted
-}
-a:active, a:hover {
-	outline: 0
-}
-h1 {
-	margin: .67em 0;
-	font-size: 2em
-}
-abbr[title] {
-	border-bottom: 1px dotted
-}
-b, strong {
-	font-weight: bold
-}
-dfn {
-	font-style: italic
-}
-hr {
-	height: 0;
-	-moz-box-sizing: content-box;
-	box-sizing: content-box
-}
-mark {
-	color: #000;
-	background: #ff0
-}
-code, kbd, pre, samp {
-	font-family: monospace, serif;
-	font-size: 1em
-}
-pre {
-	white-space: pre-wrap
-}
-q {
-	quotes: "\201C" "\201D" "\2018" "\2019"
-}
-small {
-	font-size: 80%
-}
-sub, sup {
-	position: relative;
-	font-size: 75%;
-	line-height: 0;
-	vertical-align: baseline
-}
-sup {
-	top: -0.5em
-}
-sub {
-	bottom: -0.25em
-}
-img {
-	border: 0
-}
-svg:not(:root) {
-	overflow: hidden
-}
-figure {
-	margin: 0
-}
-fieldset {
-	padding: .35em .625em .75em;
-	margin: 0 2px;
-	border: 1px solid #c0c0c0
-}
-legend {
-	padding: 0;
-	border: 0
-}
-button, input, select, textarea {
-	margin: 0;
-	font-family: inherit;
-	font-size: 100%
-}
-button, input {
-	line-height: normal
-}
-button, select {
-	text-transform: none
-}
-button, html input[type="button"], input[type="reset"], input[type="submit"] {
-	cursor: pointer;
-	-webkit-appearance: button
-}
-button[disabled], html input[disabled] {
-	cursor: default
-}
-input[type="checkbox"], input[type="radio"] {
-	padding: 0;
-	box-sizing: border-box
-}
-input[type="search"] {
-	-webkit-box-sizing: content-box;
-	-moz-box-sizing: content-box;
-	box-sizing: content-box;
-	-webkit-appearance: textfield
-}
-input[type="search"]::-webkit-search-cancel-button, input[type="search"]::-webkit-search-decoration {
--webkit-appearance:none
-}
-button::-moz-focus-inner, input::-moz-focus-inner {
-padding:0;
-border:0
-}
-textarea {
-	overflow: auto;
-	vertical-align: top
-}
-table {
-	border-collapse: collapse;
-	border-spacing: 0
-}
-@media print {
-* {
-	color: #000!important;
-	text-shadow: none!important;
-	background: transparent!important;
-	box-shadow: none!important
-}
-a, a:visited {
-	text-decoration: underline
-}
-a[href]:after {
-	content: " (" attr(href) ")"
-}
-abbr[title]:after {
-	content: " (" attr(title) ")"
-}
-.ir a:after, a[href^="javascript:"]:after, a[href^="#"]:after {
-	content: ""
-}
-pre, blockquote {
-	border: 1px solid #999;
-	page-break-inside: avoid
-}
-thead {
-	display: table-header-group
-}
-tr, img {
-	page-break-inside: avoid
-}
-img {
-	max-width: 100%!important
-}
-@page {
-margin:2cm .5cm
-}
-p, h2, h3 {
-	orphans: 3;
-	widows: 3
-}
-h2, h3 {
-	page-break-after: avoid
-}
-.navbar {
-	display: none
-}
-.table td, .table th {
-	background-color: #fff!important
-}
-.btn>.caret, .dropup>.btn>.caret {
-	border-top-color: #000!important
-}
-.label {
-	border: 1px solid #000
-}
-.table {
-	border-collapse: collapse!important
-}
-.table-bordered th, .table-bordered td {
-	border: 1px solid #ddd!important
-}
-}
-*, *:before, *:after {
-	-webkit-box-sizing: border-box;
-	-moz-box-sizing: border-box;
-	box-sizing: border-box
-}
-html {
-	font-size: 62.5%;
-	-webkit-tap-highlight-color: rgba(0,0,0,0)
-}
-body {
-	font-family: "Helvetica Neue", Helvetica, Arial, sans-serif;
-	font-size: 14px;
-	line-height: 1.428571429;
-	color: #333;
-	background-color: #fff
-}
-input, button, select, textarea {
-	font-family: inherit;
-	font-size: inherit;
-	line-height: inherit
-}
-button, input, select[multiple], textarea {
-	background-image: none
-}
-a {
-	color: #428bca;
-	text-decoration: none
-}
-a:hover, a:focus {
-	color: #2a6496;
-	text-decoration: underline
-}
-a:focus {
-	outline: thin dotted #333;
-	outline: 5px auto -webkit-focus-ring-color;
-	outline-offset: -2px
-}
-img {
-	vertical-align: middle
-}
-.img-responsive {
-	display: block;
-	height: auto;
-	max-width: 100%
-}
-.img-rounded {
-	border-radius: 6px
-}
-.img-thumbnail {
-	display: inline-block;
-	height: auto;
-	max-width: 100%;
-	padding: 4px;
-	line-height: 1.428571429;
-	background-color: #fff;
-	border: 1px solid #ddd;
-	border-radius: 4px;
-	-webkit-transition: all .2s ease-in-out;
-	transition: all .2s ease-in-out
-}
-.img-circle {
-	border-radius: 50%
-}
-hr {
-	margin-top: 20px;
-	margin-bottom: 20px;
-	border: 0;
-	border-top: 1px solid #eee
-}
-.sr-only {
-	position: absolute;
-	width: 1px;
-	height: 1px;
-	padding: 0;
-	margin: -1px;
-	overflow: hidden;
-	clip: rect(0 0 0 0);
-	border: 0
-}
-p {
-	margin: 0 0 10px
-}
-.lead {
-	margin-bottom: 20px;
-	font-size: 16.099999999999998px;
-	font-weight: 200;
-	line-height: 1.4
-}
-@media(min-width:768px) {
-.lead {
-	font-size: 21px
-}
-}
-small {
-	font-size: 85%
-}
-cite {
-	font-style: normal
-}
-.text-muted {
-	color: #999
-}
-.text-primary {
-	color: #428bca
-}
-.text-warning {
-	color: #c09853
-}
-.text-danger {
-	color: #b94a48
-}
-.text-success {
-	color: #468847
-}
-.text-info {
-	color: #3a87ad
-}
-.text-left {
-	text-align: left
-}
-.text-right {
-	text-align: right
-}
-.text-center {
-	text-align: center
-}
-h1, h2, h3, h4, h5, h6, .h1, .h2, .h3, .h4, .h5, .h6 {
-	font-family: "Helvetica Neue", Helvetica, Arial, sans-serif;
-	font-weight: 500;
-	line-height: 1.1
-}
-h1 small, h2 small, h3 small, h4 small, h5 small, h6 small, .h1 small, .h2 small, .h3 small, .h4 small, .h5 small, .h6 small {
-	font-weight: normal;
-	line-height: 1;
-	color: #999
-}
-h1, h2, h3 {
-	margin-top: 20px;
-	margin-bottom: 10px
-}
-h4, h5, h6 {
-	margin-top: 10px;
-	margin-bottom: 10px
-}
-h1, .h1 {
-	font-size: 36px
-}
-h2, .h2 {
-	font-size: 30px
-}
-h3, .h3 {
-	font-size: 24px
-}
-h4, .h4 {
-	font-size: 18px
-}
-h5, .h5 {
-	font-size: 14px
-}
-h6, .h6 {
-	font-size: 12px
-}
-h1 small, .h1 small {
-	font-size: 24px
-}
-h2 small, .h2 small {
-	font-size: 18px
-}
-h3 small, .h3 small, h4 small, .h4 small {
-	font-size: 14px
-}
-.page-header {
-	padding-bottom: 9px;
-	margin: 40px 0 20px;
-	border-bottom: 1px solid #eee
-}
-ul, ol {
-	margin-top: 0;
-	margin-bottom: 10px
-}
-ul ul, ol ul, ul ol, ol ol {
-	margin-bottom: 0
-}
-.list-unstyled {
-	padding-left: 0;
-	list-style: none
-}
-.list-inline {
-	padding-left: 0;
-	list-style: none
-}
-.list-inline>li {
-	display: inline-block;
-	padding-right: 5px;
-	padding-left: 5px
-}
-dl {
-	margin-bottom: 20px
-}
-dt, dd {
-	line-height: 1.428571429
-}
-dt {
-	font-weight: bold
-}
-dd {
-	margin-left: 0
-}
-@media(min-width:768px) {
-.dl-horizontal dt {
-	float: left;
-	width: 160px;
-	overflow: hidden;
-	clear: left;
-	text-align: right;
-	text-overflow: ellipsis;
-	white-space: nowrap
-}
-.dl-horizontal dd {
-	margin-left: 180px
-}
-.dl-horizontal dd:before, .dl-horizontal dd:after {
-	display: table;
-	content: " "
-}
-.dl-horizontal dd:after {
-	clear: both
-}
-.dl-horizontal dd:before, .dl-horizontal dd:after {
-	display: table;
-	content: " "
-}
-.dl-horizontal dd:after {
-	clear: both
-}
-}
-abbr[title], abbr[data-original-title] {
-	cursor: help;
-	border-bottom: 1px dotted #999
-}
-abbr.initialism {
-	font-size: 90%;
-	text-transform: uppercase
-}
-blockquote {
-	padding: 10px 20px;
-	margin: 0 0 20px;
-	border-left: 5px solid #eee
-}
-blockquote p {
-	font-size: 17.5px;
-	font-weight: 300;
-	line-height: 1.25
-}
-blockquote p:last-child {
-	margin-bottom: 0
-}
-blockquote small {
-	display: block;
-	line-height: 1.428571429;
-	color: #999
-}
-blockquote small:before {
-	content: '\2014 \00A0'
-}
-blockquote.pull-right {
-	padding-right: 15px;
-	padding-left: 0;
-	border-right: 5px solid #eee;
-	border-left: 0
-}
-blockquote.pull-right p, blockquote.pull-right small {
-	text-align: right
-}
-blockquote.pull-right small:before {
-	content: ''
-}
-blockquote.pull-right small:after {
-	content: '\00A0 \2014'
-}
-q:before, q:after, blockquote:before, blockquote:after {
-	content: ""
-}
-address {
-	display: block;
-	margin-bottom: 20px;
-	font-style: normal;
-	line-height: 1.428571429
-}
-code, pre {
-	font-family: Monaco, Menlo, Consolas, "Courier New", monospace
-}
-code {
-	padding: 2px 4px;
-	font-size: 90%;
-	color: #c7254e;
-	white-space: nowrap;
-	background-color: #f9f2f4;
-	border-radius: 4px
-}
-pre {
-	display: block;
-	padding: 9.5px;
-	margin: 0 0 10px;
-	font-size: 13px;
-	line-height: 1.428571429;
-	color: #333;
-	word-break: break-all;
-	word-wrap: break-word;
-	background-color: #f5f5f5;
-	border: 1px solid #ccc;
-	border-radius: 4px
-}
-pre.prettyprint {
-	margin-bottom: 20px
-}
-pre code {
-	padding: 0;
-	font-size: inherit;
-	color: inherit;
-	white-space: pre-wrap;
-	background-color: transparent;
-	border: 0
-}
-.pre-scrollable {
-	max-height: 340px;
-	overflow-y: scroll
-}
-.container {
-	padding-right: 15px;
-	padding-left: 15px;
-	margin-right: auto;
-	margin-left: auto
-}
-.container:before, .container:after {
-	display: table;
-	content: " "
-}
-.container:after {
-	clear: both
-}
-.container:before, .container:after {
-	display: table;
-	content: " "
-}
-.container:after {
-	clear: both
-}
-.row {
-	margin-right: -15px;
-	margin-left: -15px
-}
-.row:before, .row:after {
-	display: table;
-	content: " "
-}
-.row:after {
-	clear: both
-}
-.row:before, .row:after {
-	display: table;
-	content: " "
-}
-.row:after {
-	clear: both
-}
-.col-xs-1, .col-xs-2, .col-xs-3, .col-xs-4, .col-xs-5, .col-xs-6, .col-xs-7, .col-xs-8, .col-xs-9, .col-xs-10, .col-xs-11, .col-xs-12, .col-sm-1, .col-sm-2, .col-sm-3, .col-sm-4, .col-sm-5, .col-sm-6, .col-sm-7, .col-sm-8, .col-sm-9, .col-sm-10, .col-sm-11, .col-sm-12, .col-md-1, .col-md-2, .col-md-3, .col-md-4, .col-md-5, .col-md-6, .col-md-7, .col-md-8, .col-md-9, .col-md-10, .col-md-11, .col-md-12, .col-lg-1, .col-lg-2, .col-lg-3, .col-lg-4, .col-lg-5, .col-lg-6, .col-lg-7, .col-lg-8, .col-lg-9, .col-lg-10, .col-lg-11, .col-lg-12 {
-	position: relative;
-	min-height: 1px;
-	padding-right: 15px;
-	padding-left: 15px
-}
-.col-xs-1, .col-xs-2, .col-xs-3, .col-xs-4, .col-xs-5, .col-xs-6, .col-xs-7, .col-xs-8, .col-xs-9, .col-xs-10, .col-xs-11 {
-	float: left
-}
-.col-xs-1 {
-	width: 8.333333333333332%
-}
-.col-xs-2 {
-	width: 16.666666666666664%
-}
-.col-xs-3 {
-	width: 25%
-}
-.col-xs-4 {
-	width: 33.33333333333333%
-}
-.col-xs-5 {
-	width: 41.66666666666667%
-}
-.col-xs-6 {
-	width: 50%
-}
-.col-xs-7 {
-	width: 58.333333333333336%
-}
-.col-xs-8 {
-	width: 66.66666666666666%
-}
-.col-xs-9 {
-	width: 75%
-}
-.col-xs-10 {
-	width: 83.33333333333334%
-}
-.col-xs-11 {
-	width: 91.66666666666666%
-}
-.col-xs-12 {
-	width: 100%
-}
-@media(min-width:768px) {
-.container {
-	max-width: 750px
-}
-.col-sm-1, .col-sm-2, .col-sm-3, .col-sm-4, .col-sm-5, .col-sm-6, .col-sm-7, .col-sm-8, .col-sm-9, .col-sm-10, .col-sm-11 {
-	float: left
-}
-.col-sm-1 {
-	width: 8.333333333333332%
-}
-.col-sm-2 {
-	width: 16.666666666666664%
-}
-.col-sm-3 {
-	width: 25%
-}
-.col-sm-4 {
-	width: 33.33333333333333%
-}
-.col-sm-5 {
-	width: 41.66666666666667%
-}
-.col-sm-6 {
-	width: 50%
-}
-.col-sm-7 {
-	width: 58.333333333333336%
-}
-.col-sm-8 {
-	width: 66.66666666666666%
-}
-.col-sm-9 {
-	width: 75%
-}
-.col-sm-10 {
-	width: 83.33333333333334%
-}
-.col-sm-11 {
-	width: 91.66666666666666%
-}
-.col-sm-12 {
-	width: 100%
-}
-.col-sm-push-1 {
-	left: 8.333333333333332%
-}
-.col-sm-push-2 {
-	left: 16.666666666666664%
-}
-.col-sm-push-3 {
-	left: 25%
-}
-.col-sm-push-4 {
-	left: 33.33333333333333%
-}
-.col-sm-push-5 {
-	left: 41.66666666666667%
-}
-.col-sm-push-6 {
-	left: 50%
-}
-.col-sm-push-7 {
-	left: 58.333333333333336%
-}
-.col-sm-push-8 {
-	left: 66.66666666666666%
-}
-.col-sm-push-9 {
-	left: 75%
-}
-.col-sm-push-10 {
-	left: 83.33333333333334%
-}
-.col-sm-push-11 {
-	left: 91.66666666666666%
-}
-.col-sm-pull-1 {
-	right: 8.333333333333332%
-}
-.col-sm-pull-2 {
-	right: 16.666666666666664%
-}
-.col-sm-pull-3 {
-	right: 25%
-}
-.col-sm-pull-4 {
-	right: 33.33333333333333%
-}
-.col-sm-pull-5 {
-	right: 41.66666666666667%
-}
-.col-sm-pull-6 {
-	right: 50%
-}
-.col-sm-pull-7 {
-	right: 58.333333333333336%
-}
-.col-sm-pull-8 {
-	right: 66.66666666666666%
-}
-.col-sm-pull-9 {
-	right: 75%
-}
-.col-sm-pull-10 {
-	right: 83.33333333333334%
-}
-.col-sm-pull-11 {
-	right: 91.66666666666666%
-}
-.col-sm-offset-1 {
-	margin-left: 8.333333333333332%
-}
-.col-sm-offset-2 {
-	margin-left: 16.666666666666664%
-}
-.col-sm-offset-3 {
-	margin-left: 25%
-}
-.col-sm-offset-4 {
-	margin-left: 33.33333333333333%
-}
-.col-sm-offset-5 {
-	margin-left: 41.66666666666667%
-}
-.col-sm-offset-6 {
-	margin-left: 50%
-}
-.col-sm-offset-7 {
-	margin-left: 58.333333333333336%
-}
-.col-sm-offset-8 {
-	margin-left: 66.66666666666666%
-}
-.col-sm-offset-9 {
-	margin-left: 75%
-}
-.col-sm-offset-10 {
-	margin-left: 83.33333333333334%
-}
-.col-sm-offset-11 {
-	margin-left: 91.66666666666666%
-}
-}
-@media(min-width:992px) {
-.container {
-	max-width: 970px
-}
-.col-md-1, .col-md-2, .col-md-3, .col-md-4, .col-md-5, .col-md-6, .col-md-7, .col-md-8, .col-md-9, .col-md-10, .col-md-11 {
-	float: left
-}
-.col-md-1 {
-	width: 8.333333333333332%
-}
-.col-md-2 {
-	width: 16.666666666666664%
-}
-.col-md-3 {
-	width: 25%
-}
-.col-md-4 {
-	width: 33.33333333333333%
-}
-.col-md-5 {
-	width: 41.66666666666667%
-}
-.col-md-6 {
-	width: 50%
-}
-.col-md-7 {
-	width: 58.333333333333336%
-}
-.col-md-8 {
-	width: 66.66666666666666%
-}
-.col-md-9 {
-	width: 75%
-}
-.col-md-10 {
-	width: 83.33333333333334%
-}
-.col-md-11 {
-	width: 91.66666666666666%
-}
-.col-md-12 {
-	width: 100%
-}
-.col-md-push-0 {
-	left: auto
-}
-.col-md-push-1 {
-	left: 8.333333333333332%
-}
-.col-md-push-2 {
-	left: 16.666666666666664%
-}
-.col-md-push-3 {
-	left: 25%
-}
-.col-md-push-4 {
-	left: 33.33333333333333%
-}
-.col-md-push-5 {
-	left: 41.66666666666667%
-}
-.col-md-push-6 {
-	left: 50%
-}
-.col-md-push-7 {
-	left: 58.333333333333336%
-}
-.col-md-push-8 {
-	left: 66.66666666666666%
-}
-.col-md-push-9 {
-	left: 75%
-}
-.col-md-push-10 {
-	left: 83.33333333333334%
-}
-.col-md-push-11 {
-	left: 91.66666666666666%
-}
-.col-md-pull-0 {
-	right: auto
-}
-.col-md-pull-1 {
-	right: 8.333333333333332%
-}
-.col-md-pull-2 {
-	right: 16.666666666666664%
-}
-.col-md-pull-3 {
-	right: 25%
-}
-.col-md-pull-4 {
-	right: 33.33333333333333%
-}
-.col-md-pull-5 {
-	right: 41.66666666666667%
-}
-.col-md-pull-6 {
-	right: 50%
-}
-.col-md-pull-7 {
-	right: 58.333333333333336%
-}
-.col-md-pull-8 {
-	right: 66.66666666666666%
-}
-.col-md-pull-9 {
-	right: 75%
-}
-.col-md-pull-10 {
-	right: 83.33333333333334%
-}
-.col-md-pull-11 {
-	right: 91.66666666666666%
-}
-.col-md-offset-0 {
-	margin-left: 0
-}
-.col-md-offset-1 {
-	margin-left: 8.333333333333332%
-}
-.col-md-offset-2 {
-	margin-left: 16.666666666666664%
-}
-.col-md-offset-3 {
-	margin-left: 25%
-}
-.col-md-offset-4 {
-	margin-left: 33.33333333333333%
-}
-.col-md-offset-5 {
-	margin-left: 41.66666666666667%
-}
-.col-md-offset-6 {
-	margin-left: 50%
-}
-.col-md-offset-7 {
-	margin-left: 58.333333333333336%
-}
-.col-md-offset-8 {
-	margin-left: 66.66666666666666%
-}
-.col-md-offset-9 {
-	margin-left: 75%
-}
-.col-md-offset-10 {
-	margin-left: 83.33333333333334%
-}
-.col-md-offset-11 {
-	margin-left: 91.66666666666666%
-}
-}
-@media(min-width:1200px) {
-.container {
-	max-width: 1170px
-}
-.col-lg-1, .col-lg-2, .col-lg-3, .col-lg-4, .col-lg-5, .col-lg-6, .col-lg-7, .col-lg-8, .col-lg-9, .col-lg-10, .col-lg-11 {
-	float: left
-}
-.col-lg-1 {
-	width: 8.333333333333332%
-}
-.col-lg-2 {
-	width: 16.666666666666664%
-}
-.col-lg-3 {
-	width: 25%
-}
-.col-lg-4 {
-	width: 33.33333333333333%
-}
-.col-lg-5 {
-	width: 41.66666666666667%
-}
-.col-lg-6 {
-	width: 50%
-}
-.col-lg-7 {
-	width: 58.333333333333336%
-}
-.col-lg-8 {
-	width: 66.66666666666666%
-}
-.col-lg-9 {
-	width: 75%
-}
-.col-lg-10 {
-	width: 83.33333333333334%
-}
-.col-lg-11 {
-	width: 91.66666666666666%
-}
-.col-lg-12 {
-	width: 100%
-}
-.col-lg-push-0 {
-	left: auto
-}
-.col-lg-push-1 {
-	left: 8.333333333333332%
-}
-.col-lg-push-2 {
-	left: 16.666666666666664%
-}
-.col-lg-push-3 {
-	left: 25%
-}
-.col-lg-push-4 {
-	left: 33.33333333333333%
-}
-.col-lg-push-5 {
-	left: 41.66666666666667%
-}
-.col-lg-push-6 {
-	left: 50%
-}
-.col-lg-push-7 {
-	left: 58.333333333333336%
-}
-.col-lg-push-8 {
-	left: 66.66666666666666%
-}
-.col-lg-push-9 {
-	left: 75%
-}
-.col-lg-push-10 {
-	left: 83.33333333333334%
-}
-.col-lg-push-11 {
-	left: 91.66666666666666%
-}
-.col-lg-pull-0 {
-	right: auto
-}
-.col-lg-pull-1 {
-	right: 8.333333333333332%
-}
-.col-lg-pull-2 {
-	right: 16.666666666666664%
-}
-.col-lg-pull-3 {
-	right: 25%
-}
-.col-lg-pull-4 {
-	right: 33.33333333333333%
-}
-.col-lg-pull-5 {
-	right: 41.66666666666667%
-}
-.col-lg-pull-6 {
-	right: 50%
-}
-.col-lg-pull-7 {
-	right: 58.333333333333336%
-}
-.col-lg-pull-8 {
-	right: 66.66666666666666%
-}
-.col-lg-pull-9 {
-	right: 75%
-}
-.col-lg-pull-10 {
-	right: 83.33333333333334%
-}
-.col-lg-pull-11 {
-	right: 91.66666666666666%
-}
-.col-lg-offset-0 {
-	margin-left: 0
-}
-.col-lg-offset-1 {
-	margin-left: 8.333333333333332%
-}
-.col-lg-offset-2 {
-	margin-left: 16.666666666666664%
-}
-.col-lg-offset-3 {
-	margin-left: 25%
-}
-.col-lg-offset-4 {
-	margin-left: 33.33333333333333%
-}
-.col-lg-offset-5 {
-	margin-left: 41.66666666666667%
-}
-.col-lg-offset-6 {
-	margin-left: 50%
-}
-.col-lg-offset-7 {
-	margin-left: 58.333333333333336%
-}
-.col-lg-offset-8 {
-	margin-left: 66.66666666666666%
-}
-.col-lg-offset-9 {
-	margin-left: 75%
-}
-.col-lg-offset-10 {
-	margin-left: 83.33333333333334%
-}
-.col-lg-offset-11 {
-	margin-left: 91.66666666666666%
-}
-}
-table {
-	max-width: 100%;
-	background-color: transparent
-}
-th {
-	text-align: left
-}
-.table {
-	width: 100%;
-	margin-bottom: 20px
-}
-.table thead>tr>th, .table tbody>tr>th, .table tfoot>tr>th, .table thead>tr>td, .table tbody>tr>td, .table tfoot>tr>td {
-	padding: 8px;
-	line-height: 1.428571429;
-	vertical-align: top;
-	border-top: 1px solid #ddd
-}
-.table thead>tr>th {
-	vertical-align: bottom;
-	border-bottom: 2px solid #ddd
-}
-.table caption+thead tr:first-child th, .table colgroup+thead tr:first-child th, .table thead:first-child tr:first-child th, .table caption+thead tr:first-child td, .table colgroup+thead tr:first-child td, .table thead:first-child tr:first-child td {
-	border-top: 0
-}
-.table tbody+tbody {
-	border-top: 2px solid #ddd
-}
-.table .table {
-	background-color: #fff
-}
-.table-condensed thead>tr>th, .table-condensed tbody>tr>th, .table-condensed tfoot>tr>th, .table-condensed thead>tr>td, .table-condensed tbody>tr>td, .table-condensed tfoot>tr>td {
-	padding: 5px
-}
-.table-bordered {
-	border: 1px solid #ddd
-}
-.table-bordered>thead>tr>th, .table-bordered>tbody>tr>th, .table-bordered>tfoot>tr>th, .table-bordered>thead>tr>td, .table-bordered>tbody>tr>td, .table-bordered>tfoot>tr>td {
-	border: 1px solid #ddd
-}
-.table-bordered>thead>tr>th, .table-bordered>thead>tr>td {
-	border-bottom-width: 2px
-}
-.table-striped>tbody>tr:nth-child(odd)>td, .table-striped>tbody>tr:nth-child(odd)>th {
-	background-color: #f9f9f9
-}
-.table-hover>tbody>tr:hover>td, .table-hover>tbody>tr:hover>th {
-	background-color: #f5f5f5
-}
-table col[class*="col-"] {
-	display: table-column;
-	float: none
-}
-table td[class*="col-"], table th[class*="col-"] {
-	display: table-cell;
-	float: none
-}
-.table>thead>tr>td.active, .table>tbody>tr>td.active, .table>tfoot>tr>td.active, .table>thead>tr>th.active, .table>tbody>tr>th.active, .table>tfoot>tr>th.active, .table>thead>tr.active>td, .table>tbody>tr.active>td, .table>tfoot>tr.active>td, .table>thead>tr.active>th, .table>tbody>tr.active>th, .table>tfoot>tr.active>th {
-	background-color: #f5f5f5
-}
-.table>thead>tr>td.success, .table>tbody>tr>td.success, .table>tfoot>tr>td.success, .table>thead>tr>th.success, .table>tbody>tr>th.success, .table>tfoot>tr>th.success, .table>thead>tr.success>td, .table>tbody>tr.success>td, .table>tfoot>tr.success>td, .table>thead>tr.success>th, .table>tbody>tr.success>th, .table>tfoot>tr.success>th {
-	background-color: #dff0d8;
-	border-color: #d6e9c6
-}
-.table-hover>tbody>tr>td.success:hover, .table-hover>tbody>tr>th.success:hover, .table-hover>tbody>tr.success:hover>td {
-	background-color: #d0e9c6;
-	border-color: #c9e2b3
-}
-.table>thead>tr>td.danger, .table>tbody>tr>td.danger, .table>tfoot>tr>td.danger, .table>thead>tr>th.danger, .table>tbody>tr>th.danger, .table>tfoot>tr>th.danger, .table>thead>tr.danger>td, .table>tbody>tr.danger>td, .table>tfoot>tr.danger>td, .table>thead>tr.danger>th, .table>tbody>tr.danger>th, .table>tfoot>tr.danger>th {
-	background-color: #f2dede;
-	border-color: #eed3d7
-}
-.table-hover>tbody>tr>td.danger:hover, .table-hover>tbody>tr>th.danger:hover, .table-hover>tbody>tr.danger:hover>td {
-	background-color: #ebcccc;
-	border-color: #e6c1c7
-}
-.table>thead>tr>td.warning, .table>tbody>tr>td.warning, .table>tfoot>tr>td.warning, .table>thead>tr>th.warning, .table>tbody>tr>th.warning, .table>tfoot>tr>th.warning, .table>thead>tr.warning>td, .table>tbody>tr.warning>td, .table>tfoot>tr.warning>td, .table>thead>tr.warning>th, .table>tbody>tr.warning>th, .table>tfoot>tr.warning>th {
-	background-color: #fcf8e3;
-	border-color: #fbeed5
-}
-.table-hover>tbody>tr>td.warning:hover, .table-hover>tbody>tr>th.warning:hover, .table-hover>tbody>tr.warning:hover>td {
-	background-color: #faf2cc;
-	border-color: #f8e5be
-}
-@media(max-width:768px) {
-.table-responsive {
-	width: 100%;
-	margin-bottom: 15px;
-	overflow-x: scroll;
-	overflow-y: hidden;
-	border: 1px solid #ddd
-}
-.table-responsive>.table {
-	margin-bottom: 0;
-	background-color: #fff
-}
-.table-responsive>.table>thead>tr>th, .table-responsive>.table>tbody>tr>th, .table-responsive>.table>tfoot>tr>th, .table-responsive>.table>thead>tr>td, .table-responsive>.table>tbody>tr>td, .table-responsive>.table>tfoot>tr>td {
-	white-space: nowrap
-}
-.table-responsive>.table-bordered {
-	border: 0
-}
-.table-responsive>.table-bordered>thead>tr>th:first-child, .table-responsive>.table-bordered>tbody>tr>th:first-child, .table-responsive>.table-bordered>tfoot>tr>th:first-child, .table-responsive>.table-bordered>thead>tr>td:first-child, .table-responsive>.table-bordered>tbody>tr>td:first-child, .table-responsive>.table-bordered>tfoot>tr>td:first-child {
-	border-left: 0
-}
-.table-responsive>.table-bordered>thead>tr>th:last-child, .table-responsive>.table-bordered>tbody>tr>th:last-child, .table-responsive>.table-bordered>tfoot>tr>th:last-child, .table-responsive>.table-bordered>thead>tr>td:last-child, .table-responsive>.table-bordered>tbody>tr>td:last-child, .table-responsive>.table-bordered>tfoot>tr>td:last-child {
-	border-right: 0
-}
-.table-responsive>.table-bordered>thead>tr:last-child>th, .table-responsive>.table-bordered>tbody>tr:last-child>th, .table-responsive>.table-bordered>tfoot>tr:last-child>th, .table-responsive>.table-bordered>thead>tr:last-child>td, .table-responsive>.table-bordered>tbody>tr:last-child>td, .table-responsive>.table-bordered>tfoot>tr:last-child>td {
-	border-bottom: 0
-}
-}
-fieldset {
-	padding: 0;
-	margin: 0;
-	border: 0
-}
-legend {
-	display: block;
-	width: 100%;
-	padding: 0;
-	margin-bottom: 20px;
-	font-size: 21px;
-	line-height: inherit;
-	color: #333;
-	border: 0;
-	border-bottom: 1px solid #e5e5e5
-}
-label {
-	display: inline-block;
-	margin-bottom: 5px;
-	font-weight: bold
-}
-input[type="search"] {
-	-webkit-box-sizing: border-box;
-	-moz-box-sizing: border-box;
-	box-sizing: border-box
-}
-input[type="radio"], input[type="checkbox"] {
-	margin: 4px 0 0;
-	margin-top: 1px \9;
-	line-height: normal
-}
-input[type="file"] {
-	display: block
-}
-select[multiple], select[size] {
-	height: auto
-}
-select optgroup {
-	font-family: inherit;
-	font-size: inherit;
-	font-style: inherit
-}
-input[type="file"]:focus, input[type="radio"]:focus, input[type="checkbox"]:focus {
-	outline: thin dotted #333;
-	outline: 5px auto -webkit-focus-ring-color;
-	outline-offset: -2px
-}
-input[type="number"]::-webkit-outer-spin-button, input[type="number"]::-webkit-inner-spin-button {
-height:auto
-}
-.form-control:-moz-placeholder {
-color:#999
-}
-.form-control::-moz-placeholder {
-color:#999
-}
-.form-control:-ms-input-placeholder {
-color:#999
-}
-.form-control::-webkit-input-placeholder {
-color:#999
-}
-.form-control {
-	display: block;
-	width: 100%;
-	height: 34px;
-	padding: 6px 12px;
-	font-size: 14px;
-	line-height: 1.428571429;
-	color: #555;
-	vertical-align: middle;
-	background-color: #fff;
-	border: 1px solid #ccc;
-	border-radius: 4px;
-	-webkit-box-shadow: inset 0 1px 1px rgba(0,0,0,0.075);
-	box-shadow: inset 0 1px 1px rgba(0,0,0,0.075);
-	-webkit-transition: border-color ease-in-out .15s, box-shadow ease-in-out .15s;
-	transition: border-color ease-in-out .15s, box-shadow ease-in-out .15s
-}
-.form-control:focus {
-	border-color: #66afe9;
-	outline: 0;
-	-webkit-box-shadow: inset 0 1px 1px rgba(0,0,0,0.075), 0 0 8px rgba(102,175,233,0.6);
-	box-shadow: inset 0 1px 1px rgba(0,0,0,0.075), 0 0 8px rgba(102,175,233,0.6)
-}
-.form-control[disabled], .form-control[readonly], fieldset[disabled] .form-control {
-	cursor: not-allowed;
-	background-color: #eee
-}
-textarea.form-control {
-	height: auto
-}
-.form-group {
-	margin-bottom: 15px
-}
-.radio, .checkbox {
-	display: block;
-	min-height: 20px;
-	padding-left: 20px;
-	margin-top: 10px;
-	margin-bottom: 10px;
-	vertical-align: middle
-}
-.radio label, .checkbox label {
-	display: inline;
-	margin-bottom: 0;
-	font-weight: normal;
-	cursor: pointer
-}
-.radio input[type="radio"], .radio-inline input[type="radio"], .checkbox input[type="checkbox"], .checkbox-inline input[type="checkbox"] {
-	float: left;
-	margin-left: -20px
-}
-.radio+.radio, .checkbox+.checkbox {
-	margin-top: -5px
-}
-.radio-inline, .checkbox-inline {
-	display: inline-block;
-	padding-left: 20px;
-	margin-bottom: 0;
-	font-weight: normal;
-	vertical-align: middle;
-	cursor: pointer
-}
-.radio-inline+.radio-inline, .checkbox-inline+.checkbox-inline {
-	margin-top: 0;
-	margin-left: 10px
-}
-input[type="radio"][disabled], input[type="checkbox"][disabled], .radio[disabled], .radio-inline[disabled], .checkbox[disabled], .checkbox-inline[disabled], fieldset[disabled] input[type="radio"], fieldset[disabled] input[type="checkbox"], fieldset[disabled] .radio, fieldset[disabled] .radio-inline, fieldset[disabled] .checkbox, fieldset[disabled] .checkbox-inline {
-	cursor: not-allowed
-}
-.input-sm {
-	height: 30px;
-	padding: 5px 10px;
-	font-size: 12px;
-	line-height: 1.5;
-	border-radius: 3px
-}
-select.input-sm {
-	height: 30px;
-	line-height: 30px
-}
-textarea.input-sm {
-	height: auto
-}
-.input-lg {
-	height: 45px;
-	padding: 10px 16px;
-	font-size: 18px;
-	line-height: 1.33;
-	border-radius: 6px
-}
-select.input-lg {
-	height: 45px;
-	line-height: 45px
-}
-textarea.input-lg {
-	height: auto
-}
-.has-warning .help-block, .has-warning .control-label {
-	color: #c09853
-}
-.has-warning .form-control {
-	border-color: #c09853;
-	-webkit-box-shadow: inset 0 1px 1px rgba(0,0,0,0.075);
-	box-shadow: inset 0 1px 1px rgba(0,0,0,0.075)
-}
-.has-warning .form-control:focus {
-	border-color: #a47e3c;
-	-webkit-box-shadow: inset 0 1px 1px rgba(0,0,0,0.075), 0 0 6px #dbc59e;
-	box-shadow: inset 0 1px 1px rgba(0,0,0,0.075), 0 0 6px #dbc59e
-}
-.has-warning .input-group-addon {
-	color: #c09853;
-	background-color: #fcf8e3;
-	border-color: #c09853
-}
-.has-error .help-block, .has-error .control-label {
-	color: #b94a48
-}
-.has-error .form-control {
-	border-color: #b94a48;
-	-webkit-box-shadow: inset 0 1px 1px rgba(0,0,0,0.075);
-	box-shadow: inset 0 1px 1px rgba(0,0,0,0.075)
-}
-.has-error .form-control:focus {
-	border-color: #953b39;
-	-webkit-box-shadow: inset 0 1px 1px rgba(0,0,0,0.075), 0 0 6px #d59392;
-	box-shadow: inset 0 1px 1px rgba(0,0,0,0.075), 0 0 6px #d59392
-}
-.has-error .input-group-addon {
-	color: #b94a48;
-	background-color: #f2dede;
-	border-color: #b94a48
-}
-.has-success .help-block, .has-success .control-label {
-	color: #468847
-}
-.has-success .form-control {
-	border-color: #468847;
-	-webkit-box-shadow: inset 0 1px 1px rgba(0,0,0,0.075);
-	box-shadow: inset 0 1px 1px rgba(0,0,0,0.075)
-}
-.has-success .form-control:focus {
-	border-color: #356635;
-	-webkit-box-shadow: inset 0 1px 1px rgba(0,0,0,0.075), 0 0 6px #7aba7b;
-	box-shadow: inset 0 1px 1px rgba(0,0,0,0.075), 0 0 6px #7aba7b
-}
-.has-success .input-group-addon {
-	color: #468847;
-	background-color: #dff0d8;
-	border-color: #468847
-}
-.form-control-static {
-	padding-top: 7px;
-	margin-bottom: 0
-}
-.help-block {
-	display: block;
-	margin-top: 5px;
-	margin-bottom: 10px;
-	color: #737373
-}
-@media(min-width:768px) {
-.form-inline .form-group {
-	display: inline-block;
-	margin-bottom: 0;
-	vertical-align: middle
-}
-.form-inline .form-control {
-	display: inline-block
-}
-.form-inline .radio, .form-inline .checkbox {
-	display: inline-block;
-	padding-left: 0;
-	margin-top: 0;
-	margin-bottom: 0
-}
-.form-inline .radio input[type="radio"], .form-inline .checkbox input[type="checkbox"] {
-	float: none;
-	margin-left: 0
-}
-}
-.form-horizontal .control-label, .form-horizontal .radio, .form-horizontal .checkbox, .form-horizontal .radio-inline, .form-horizontal .checkbox-inline {
-	padding-top: 7px;
-	margin-top: 0;
-	margin-bottom: 0
-}
-.form-horizontal .form-group {
-	margin-right: -15px;
-	margin-left: -15px
-}
-.form-horizontal .form-group:before, .form-horizontal .form-group:after {
-	display: table;
-	content: " "
-}
-.form-horizontal .form-group:after {
-	clear: both
-}
-.form-horizontal .form-group:before, .form-horizontal .form-group:after {
-	display: table;
-	content: " "
-}
-.form-horizontal .form-group:after {
-	clear: both
-}
-@media(min-width:768px) {
-.form-horizontal .control-label {
-	text-align: right
-}
-}
-.btn {
-	display: inline-block;
-	padding: 6px 12px;
-	margin-bottom: 0;
-	font-size: 14px;
-	font-weight: normal;
-	line-height: 1.428571429;
-	text-align: center;
-	white-space: nowrap;
-	vertical-align: middle;
-	cursor: pointer;
-	border: 1px solid transparent;
-	border-radius: 4px;
-	-webkit-user-select: none;
-	-moz-user-select: none;
-	-ms-user-select: none;
-	-o-user-select: none;
-	user-select: none
-}
-.btn:focus {
-	outline: thin dotted #333;
-	outline: 5px auto -webkit-focus-ring-color;
-	outline-offset: -2px
-}
-.btn:hover, .btn:focus {
-	color: #333;
-	text-decoration: none
-}
-.btn:active, .btn.active {
-	background-image: none;
-	outline: 0;
-	-webkit-box-shadow: inset 0 3px 5px rgba(0,0,0,0.125);
-	box-shadow: inset 0 3px 5px rgba(0,0,0,0.125)
-}
-.btn.disabled, .btn[disabled], fieldset[disabled] .btn {
-	pointer-events: none;
-	cursor: not-allowed;
-	opacity: .65;
-	filter: alpha(opacity=65);
-	-webkit-box-shadow: none;
-	box-shadow: none
-}
-.btn-default {
-	color: #333;
-	background-color: #fff;
-	border-color: #ccc
-}
-.btn-default:hover, .btn-default:focus, .btn-default:active, .btn-default.active, .open .dropdown-toggle.btn-default {
-	color: #333;
-	background-color: #ebebeb;
-	border-color: #adadad
-}
-.btn-default:active, .btn-default.active, .open .dropdown-toggle.btn-default {
-	background-image: none
-}
-.btn-default.disabled, .btn-default[disabled], fieldset[disabled] .btn-default, .btn-default.disabled:hover, .btn-default[disabled]:hover, fieldset[disabled] .btn-default:hover, .btn-default.disabled:focus, .btn-default[disabled]:focus, fieldset[disabled] .btn-default:focus, .btn-default.disabled:active, .btn-default[disabled]:active, fieldset[disabled] .btn-default:active, .btn-default.disabled.active, .btn-default[disabled].active, fieldset[disabled] .btn-default.active {
-	background-color: #fff;
-	border-color: #ccc
-}
-.btn-primary {
-	color: #fff;
-	background-color: #428bca;
-	border-color: #357ebd
-}
-.btn-primary:hover, .btn-primary:focus, .btn-primary:active, .btn-primary.active, .open .dropdown-toggle.btn-primary {
-	color: #fff;
-	background-color: #3276b1;
-	border-color: #285e8e
-}
-.btn-primary:active, .btn-primary.active, .open .dropdown-toggle.btn-primary {
-	background-image: none
-}
-.btn-primary.disabled, .btn-primary[disabled], fieldset[disabled] .btn-primary, .btn-primary.disabled:hover, .btn-primary[disabled]:hover, fieldset[disabled] .btn-primary:hover, .btn-primary.disabled:focus, .btn-primary[disabled]:focus, fieldset[disabled] .btn-primary:focus, .btn-primary.disabled:active, .btn-primary[disabled]:active, fieldset[disabled] .btn-primary:active, .btn-primary.disabled.active, .btn-primary[disabled].active, fieldset[disabled] .btn-primary.active {
-	background-color: #428bca;
-	border-color: #357ebd
-}
-.btn-warning {
-	color: #fff;
-	background-color: #f0ad4e;
-	border-color: #eea236
-}
-.btn-warning:hover, .btn-warning:focus, .btn-warning:active, .btn-warning.active, .open .dropdown-toggle.btn-warning {
-	color: #fff;
-	background-color: #ed9c28;
-	border-color: #d58512
-}
-.btn-warning:active, .btn-warning.active, .open .dropdown-toggle.btn-warning {
-	background-image: none
-}
-.btn-warning.disabled, .btn-warning[disabled], fieldset[disabled] .btn-warning, .btn-warning.disabled:hover, .btn-warning[disabled]:hover, fieldset[disabled] .btn-warning:hover, .btn-warning.disabled:focus, .btn-warning[disabled]:focus, fieldset[disabled] .btn-warning:focus, .btn-warning.disabled:active, .btn-warning[disabled]:active, fieldset[disabled] .btn-warning:active, .btn-warning.disabled.active, .btn-warning[disabled].active, fieldset[disabled] .btn-warning.active {
-	background-color: #f0ad4e;
-	border-color: #eea236
-}
-.btn-danger {
-	color: #fff;
-	background-color: #d9534f;
-	border-color: #d43f3a
-}
-.btn-danger:hover, .btn-danger:focus, .btn-danger:active, .btn-danger.active, .open .dropdown-toggle.btn-danger {
-	color: #fff;
-	background-color: #d2322d;
-	border-color: #ac2925
-}
-.btn-danger:active, .btn-danger.active, .open .dropdown-toggle.btn-danger {
-	background-image: none
-}
-.btn-danger.disabled, .btn-danger[disabled], fieldset[disabled] .btn-danger, .btn-danger.disabled:hover, .btn-danger[disabled]:hover, fieldset[disabled] .btn-danger:hover, .btn-danger.disabled:focus, .btn-danger[disabled]:focus, fieldset[disabled] .btn-danger:focus, .btn-danger.disabled:active, .btn-danger[disabled]:active, fieldset[disabled] .btn-danger:active, .btn-danger.disabled.active, .btn-danger[disabled].active, fieldset[disabled] .btn-danger.active {
-	background-color: #d9534f;
-	border-color: #d43f3a
-}
-.btn-success {
-	color: #fff;
-	background-color: #5cb85c;
-	border-color: #4cae4c
-}
-.btn-success:hover, .btn-success:focus, .btn-success:active, .btn-success.active, .open .dropdown-toggle.btn-success {
-	color: #fff;
-	background-color: #47a447;
-	border-color: #398439
-}
-.btn-success:active, .btn-success.active, .open .dropdown-toggle.btn-success {
-	background-image: none
-}
-.btn-success.disabled, .btn-success[disabled], fieldset[disabled] .btn-success, .btn-success.disabled:hover, .btn-success[disabled]:hover, fieldset[disabled] .btn-success:hover, .btn-success.disabled:focus, .btn-success[disabled]:focus, fieldset[disabled] .btn-success:focus, .btn-success.disabled:active, .btn-success[disabled]:active, fieldset[disabled] .btn-success:active, .btn-success.disabled.active, .btn-success[disabled].active, fieldset[disabled] .btn-success.active {
-	background-color: #5cb85c;
-	border-color: #4cae4c
-}
-.btn-info {
-	color: #fff;
-	background-color: #5bc0de;
-	border-color: #46b8da
-}
-.btn-info:hover, .btn-info:focus, .btn-info:active, .btn-info.active, .open .dropdown-toggle.btn-info {
-	color: #fff;
-	background-color: #39b3d7;
-	border-color: #269abc
-}
-.btn-info:active, .btn-info.active, .open .dropdown-toggle.btn-info {
-	background-image: none
-}
-.btn-info.disabled, .btn-info[disabled], fieldset[disabled] .btn-info, .btn-info.disabled:hover, .btn-info[disabled]:hover, fieldset[disabled] .btn-info:hover, .btn-info.disabled:focus, .btn-info[disabled]:focus, fieldset[disabled] .btn-info:focus, .btn-info.disabled:active, .btn-info[disabled]:active, fieldset[disabled] .btn-info:active, .btn-info.disabled.active, .btn-info[disabled].active, fieldset[disabled] .btn-info.active {
-	background-color: #5bc0de;
-	border-color: #46b8da
-}
-.btn-link {
-	font-weight: normal;
-	color: #428bca;
-	cursor: pointer;
-	border-radius: 0
-}
-.btn-link, .btn-link:active, .btn-link[disabled], fieldset[disabled] .btn-link {
-	background-color: transparent;
-	-webkit-box-shadow: none;
-	box-shadow: none
-}
-.btn-link, .btn-link:hover, .btn-link:focus, .btn-link:active {
-	border-color: transparent
-}
-.btn-link:hover, .btn-link:focus {
-	color: #2a6496;
-	text-decoration: underline;
-	background-color: transparent
-}
-.btn-link[disabled]:hover, fieldset[disabled] .btn-link:hover, .btn-link[disabled]:focus, fieldset[disabled] .btn-link:focus {
-	color: #999;
-	text-decoration: none
-}
-.btn-lg {
-	padding: 10px 16px;
-	font-size: 18px;
-	line-height: 1.33;
-	border-radius: 6px
-}
-.btn-sm, .btn-xs {
-	padding: 5px 10px;
-	font-size: 12px;
-	line-height: 1.5;
-	border-radius: 3px
-}
-.btn-xs {
-	padding: 1px 5px
-}
-.btn-block {
-	display: block;
-	width: 100%;
-	padding-right: 0;
-	padding-left: 0
-}
-.btn-block+.btn-block {
-	margin-top: 5px
-}
-input[type="submit"].btn-block, input[type="reset"].btn-block, input[type="button"].btn-block {
-	width: 100%
-}
-.fade {
-	opacity: 0;
-	-webkit-transition: opacity .15s linear;
-	transition: opacity .15s linear
-}
-.fade.in {
-	opacity: 1
-}
-.collapse {
-	display: none
-}
-.collapse.in {
-	display: block
-}
-.collapsing {
-	position: relative;
-	height: 0;
-	overflow: hidden;
-	-webkit-transition: height .35s ease;
-	transition: height .35s ease
-}
-@font-face {
-	font-family: 'Glyphicons Halflings';
-	src: url('../fonts/glyphicons-halflings-regular.eot');
-	src: url('../fonts/glyphicons-halflings-regular.eot?#iefix') format('embedded-opentype'), url('../fonts/glyphicons-halflings-regular.woff') format('woff'), url('../fonts/glyphicons-halflings-regular.ttf') format('truetype'), url('../fonts/glyphicons-halflings-regular.svg#glyphicons-halflingsregular') format('svg')
-}
-.glyphicon {
-	position: relative;
-	top: 1px;
-	display: inline-block;
-	font-family: 'Glyphicons Halflings';
-	-webkit-font-smoothing: antialiased;
-	font-style: normal;
-	font-weight: normal;
-	line-height: 1
-}
-.glyphicon-asterisk:before {
-	content: "\2a"
-}
-.glyphicon-plus:before {
-	content: "\2b"
-}
-.glyphicon-euro:before {
-	content: "\20ac"
-}
-.glyphicon-minus:before {
-	content: "\2212"
-}
-.glyphicon-cloud:before {
-	content: "\2601"
-}
-.glyphicon-envelope:before {
-	content: "\2709"
-}
-.glyphicon-pencil:before {
-	content: "\270f"
-}
-.glyphicon-glass:before {
-	content: "\e001"
-}
-.glyphicon-music:before {
-	content: "\e002"
-}
-.glyphicon-search:before {
-	content: "\e003"
-}
-.glyphicon-heart:before {
-	content: "\e005"
-}
-.glyphicon-star:before {
-	content: "\e006"
-}
-.glyphicon-star-empty:before {
-	content: "\e007"
-}
-.glyphicon-user:before {
-	content: "\e008"
-}
-.glyphicon-film:before {
-	content: "\e009"
-}
-.glyphicon-th-large:before {
-	content: "\e010"
-}
-.glyphicon-th:before {
-	content: "\e011"
-}
-.glyphicon-th-list:before {
-	content: "\e012"
-}
-.glyphicon-ok:before {
-	content: "\e013"
-}
-.glyphicon-remove:before {
-	content: "\e014"
-}
-.glyphicon-zoom-in:before {
-	content: "\e015"
-}
-.glyphicon-zoom-out:before {
-	content: "\e016"
-}
-.glyphicon-off:before {
-	content: "\e017"
-}
-.glyphicon-signal:before {
-	content: "\e018"
-}
-.glyphicon-cog:before {
-	content: "\e019"
-}
-.glyphicon-trash:before {
-	content: "\e020"
-}
-.glyphicon-home:before {
-	content: "\e021"
-}
-.glyphicon-file:before {
-	content: "\e022"
-}
-.glyphicon-time:before {
-	content: "\e023"
-}
-.glyphicon-road:before {
-	content: "\e024"
-}
-.glyphicon-download-alt:before {
-	content: "\e025"
-}
-.glyphicon-download:before {
-	content: "\e026"
-}
-.glyphicon-upload:before {
-	content: "\e027"
-}
-.glyphicon-inbox:before {
-	content: "\e028"
-}
-.glyphicon-play-circle:before {
-	content: "\e029"
-}
-.glyphicon-repeat:before {
-	content: "\e030"
-}
-.glyphicon-refresh:before {
-	content: "\e031"
-}
-.glyphicon-list-alt:before {
-	content: "\e032"
-}
-.glyphicon-flag:before {
-	content: "\e034"
-}
-.glyphicon-headphones:before {
-	content: "\e035"
-}
-.glyphicon-volume-off:before {
-	content: "\e036"
-}
-.glyphicon-volume-down:before {
-	content: "\e037"
-}
-.glyphicon-volume-up:before {
-	content: "\e038"
-}
-.glyphicon-qrcode:before {
-	content: "\e039"
-}
-.glyphicon-barcode:before {
-	content: "\e040"
-}
-.glyphicon-tag:before {
-	content: "\e041"
-}
-.glyphicon-tags:before {
-	content: "\e042"
-}
-.glyphicon-book:before {
-	content: "\e043"
-}
-.glyphicon-print:before {
-	content: "\e045"
-}
-.glyphicon-font:before {
-	content: "\e047"
-}
-.glyphicon-bold:before {
-	content: "\e048"
-}
-.glyphicon-italic:before {
-	content: "\e049"
-}
-.glyphicon-text-height:before {
-	content: "\e050"
-}
-.glyphicon-text-width:before {
-	content: "\e051"
-}
-.glyphicon-align-left:before {
-	content: "\e052"
-}
-.glyphicon-align-center:before {
-	content: "\e053"
-}
-.glyphicon-align-right:before {
-	content: "\e054"
-}
-.glyphicon-align-justify:before {
-	content: "\e055"
-}
-.glyphicon-list:before {
-	content: "\e056"
-}
-.glyphicon-indent-left:before {
-	content: "\e057"
-}
-.glyphicon-indent-right:before {
-	content: "\e058"
-}
-.glyphicon-facetime-video:before {
-	content: "\e059"
-}
-.glyphicon-picture:before {
-	content: "\e060"
-}
-.glyphicon-map-marker:before {
-	content: "\e062"
-}
-.glyphicon-adjust:before {
-	content: "\e063"
-}
-.glyphicon-tint:before {
-	content: "\e064"
-}
-.glyphicon-edit:before {
-	content: "\e065"
-}
-.glyphicon-share:before {
-	content: "\e066"
-}
-.glyphicon-check:before {
-	content: "\e067"
-}
-.glyphicon-move:before {
-	content: "\e068"
-}
-.glyphicon-step-backward:before {
-	content: "\e069"
-}
-.glyphicon-fast-backward:before {
-	content: "\e070"
-}
-.glyphicon-backward:before {
-	content: "\e071"
-}
-.glyphicon-play:before {
-	content: "\e072"
-}
-.glyphicon-pause:before {
-	content: "\e073"
-}
-.glyphicon-stop:before {
-	content: "\e074"
-}
-.glyphicon-forward:before {
-	content: "\e075"
-}
-.glyphicon-fast-forward:before {
-	content: "\e076"
-}
-.glyphicon-step-forward:before {
-	content: "\e077"
-}
-.glyphicon-eject:before {
-	content: "\e078"
-}
-.glyphicon-chevron-left:before {
-	content: "\e079"
-}
-.glyphicon-chevron-right:before {
-	content: "\e080"
-}
-.glyphicon-plus-sign:before {
-	content: "\e081"
-}
-.glyphicon-minus-sign:before {
-	content: "\e082"
-}
-.glyphicon-remove-sign:before {
-	content: "\e083"
-}
-.glyphicon-ok-sign:before {
-	content: "\e084"
-}
-.glyphicon-question-sign:before {
-	content: "\e085"
-}
-.glyphicon-info-sign:before {
-	content: "\e086"
-}
-.glyphicon-screenshot:before {
-	content: "\e087"
-}
-.glyphicon-remove-circle:before {
-	content: "\e088"
-}
-.glyphicon-ok-circle:before {
-	content: "\e089"
-}
-.glyphicon-ban-circle:before {
-	content: "\e090"
-}
-.glyphicon-arrow-left:before {
-	content: "\e091"
-}
-.glyphicon-arrow-right:before {
-	content: "\e092"
-}
-.glyphicon-arrow-up:before {
-	content: "\e093"
-}
-.glyphicon-arrow-down:before {
-	content: "\e094"
-}
-.glyphicon-share-alt:before {
-	content: "\e095"
-}
-.glyphicon-resize-full:before {
-	content: "\e096"
-}
-.glyphicon-resize-small:before {
-	content: "\e097"
-}
-.glyphicon-exclamation-sign:before {
-	content: "\e101"
-}
-.glyphicon-gift:before {
-	content: "\e102"
-}
-.glyphicon-leaf:before {
-	content: "\e103"
-}
-.glyphicon-eye-open:before {
-	content: "\e105"
-}
-.glyphicon-eye-close:before {
-	content: "\e106"
-}
-.glyphicon-warning-sign:before {
-	content: "\e107"
-}
-.glyphicon-plane:before {
-	content: "\e108"
-}
-.glyphicon-random:before {
-	content: "\e110"
-}
-.glyphicon-comment:before {
-	content: "\e111"
-}
-.glyphicon-magnet:before {
-	content: "\e112"
-}
-.glyphicon-chevron-up:before {
-	content: "\e113"
-}
-.glyphicon-chevron-down:before {
-	content: "\e114"
-}
-.glyphicon-retweet:before {
-	content: "\e115"
-}
-.glyphicon-shopping-cart:before {
-	content: "\e116"
-}
-.glyphicon-folder-close:before {
-	content: "\e117"
-}
-.glyphicon-folder-open:before {
-	content: "\e118"
-}
-.glyphicon-resize-vertical:before {
-	content: "\e119"
-}
-.glyphicon-resize-horizontal:before {
-	content: "\e120"
-}
-.glyphicon-hdd:before {
-	content: "\e121"
-}
-.glyphicon-bullhorn:before {
-	content: "\e122"
-}
-.glyphicon-certificate:before {
-	content: "\e124"
-}
-.glyphicon-thumbs-up:before {
-	content: "\e125"
-}
-.glyphicon-thumbs-down:before {
-	content: "\e126"
-}
-.glyphicon-hand-right:before {
-	content: "\e127"
-}
-.glyphicon-hand-left:before {
-	content: "\e128"
-}
-.glyphicon-hand-up:before {
-	content: "\e129"
-}
-.glyphicon-hand-down:before {
-	content: "\e130"
-}
-.glyphicon-circle-arrow-right:before {
-	content: "\e131"
-}
-.glyphicon-circle-arrow-left:before {
-	content: "\e132"
-}
-.glyphicon-circle-arrow-up:before {
-	content: "\e133"
-}
-.glyphicon-circle-arrow-down:before {
-	content: "\e134"
-}
-.glyphicon-globe:before {
-	content: "\e135"
-}
-.glyphicon-tasks:before {
-	content: "\e137"
-}
-.glyphicon-filter:before {
-	content: "\e138"
-}
-.glyphicon-fullscreen:before {
-	content: "\e140"
-}
-.glyphicon-dashboard:before {
-	content: "\e141"
-}
-.glyphicon-heart-empty:before {
-	content: "\e143"
-}
-.glyphicon-link:before {
-	content: "\e144"
-}
-.glyphicon-phone:before {
-	content: "\e145"
-}
-.glyphicon-usd:before {
-	content: "\e148"
-}
-.glyphicon-gbp:before {
-	content: "\e149"
-}
-.glyphicon-sort:before {
-	content: "\e150"
-}
-.glyphicon-sort-by-alphabet:before {
-	content: "\e151"
-}
-.glyphicon-sort-by-alphabet-alt:before {
-	content: "\e152"
-}
-.glyphicon-sort-by-order:before {
-	content: "\e153"
-}
-.glyphicon-sort-by-order-alt:before {
-	content: "\e154"
-}
-.glyphicon-sort-by-attributes:before {
-	content: "\e155"
-}
-.glyphicon-sort-by-attributes-alt:before {
-	content: "\e156"
-}
-.glyphicon-unchecked:before {
-	content: "\e157"
-}
-.glyphicon-expand:before {
-	content: "\e158"
-}
-.glyphicon-collapse-down:before {
-	content: "\e159"
-}
-.glyphicon-collapse-up:before {
-	content: "\e160"
-}
-.glyphicon-log-in:before {
-	content: "\e161"
-}
-.glyphicon-flash:before {
-	content: "\e162"
-}
-.glyphicon-log-out:before {
-	content: "\e163"
-}
-.glyphicon-new-window:before {
-	content: "\e164"
-}
-.glyphicon-record:before {
-	content: "\e165"
-}
-.glyphicon-save:before {
-	content: "\e166"
-}
-.glyphicon-open:before {
-	content: "\e167"
-}
-.glyphicon-saved:before {
-	content: "\e168"
-}
-.glyphicon-import:before {
-	content: "\e169"
-}
-.glyphicon-export:before {
-	content: "\e170"
-}
-.glyphicon-send:before {
-	content: "\e171"
-}
-.glyphicon-floppy-disk:before {
-	content: "\e172"
-}
-.glyphicon-floppy-saved:before {
-	content: "\e173"
-}
-.glyphicon-floppy-remove:before {
-	content: "\e174"
-}
-.glyphicon-floppy-save:before {
-	content: "\e175"
-}
-.glyphicon-floppy-open:before {
-	content: "\e176"
-}
-.glyphicon-credit-card:before {
-	content: "\e177"
-}
-.glyphicon-transfer:before {
-	content: "\e178"
-}
-.glyphicon-cutlery:before {
-	content: "\e179"
-}
-.glyphicon-header:before {
-	content: "\e180"
-}
-.glyphicon-compressed:before {
-	content: "\e181"
-}
-.glyphicon-earphone:before {
-	content: "\e182"
-}
-.glyphicon-phone-alt:before {
-	content: "\e183"
-}
-.glyphicon-tower:before {
-	content: "\e184"
-}
-.glyphicon-stats:before {
-	content: "\e185"
-}
-.glyphicon-sd-video:before {
-	content: "\e186"
-}
-.glyphicon-hd-video:before {
-	content: "\e187"
-}
-.glyphicon-subtitles:before {
-	content: "\e188"
-}
-.glyphicon-sound-stereo:before {
-	content: "\e189"
-}
-.glyphicon-sound-dolby:before {
-	content: "\e190"
-}
-.glyphicon-sound-5-1:before {
-	content: "\e191"
-}
-.glyphicon-sound-6-1:before {
-	content: "\e192"
-}
-.glyphicon-sound-7-1:before {
-	content: "\e193"
-}
-.glyphicon-copyright-mark:before {
-	content: "\e194"
-}
-.glyphicon-registration-mark:before {
-	content: "\e195"
-}
-.glyphicon-cloud-download:before {
-	content: "\e197"
-}
-.glyphicon-cloud-upload:before {
-	content: "\e198"
-}
-.glyphicon-tree-conifer:before {
-	content: "\e199"
-}
-.glyphicon-tree-deciduous:before {
-	content: "\e200"
-}
-.glyphicon-briefcase:before {
-	content: "\1f4bc"
-}
-.glyphicon-calendar:before {
-	content: "\1f4c5"
-}
-.glyphicon-pushpin:before {
-	content: "\1f4cc"
-}
-.glyphicon-paperclip:before {
-	content: "\1f4ce"
-}
-.glyphicon-camera:before {
-	content: "\1f4f7"
-}
-.glyphicon-lock:before {
-	content: "\1f512"
-}
-.glyphicon-bell:before {
-	content: "\1f514"
-}
-.glyphicon-bookmark:before {
-	content: "\1f516"
-}
-.glyphicon-fire:before {
-	content: "\1f525"
-}
-.glyphicon-wrench:before {
-	content: "\1f527"
-}
-.caret {
-	display: inline-block;
-	width: 0;
-	height: 0;
-	margin-left: 2px;
-	vertical-align: middle;
-	border-top: 4px solid #000;
-	border-right: 4px solid transparent;
-	border-bottom: 0 dotted;
-	border-left: 4px solid transparent;
-	content: ""
-}
-.dropdown {
-	position: relative
-}
-.dropdown-toggle:focus {
-	outline: 0
-}
-.dropdown-menu {
-	position: absolute;
-	top: 100%;
-	left: 0;
-	z-index: 1000;
-	display: none;
-	float: left;
-	min-width: 160px;
-	padding: 5px 0;
-	margin: 2px 0 0;
-	font-size: 14px;
-	list-style: none;
-	background-color: #fff;
-	border: 1px solid #ccc;
-	border: 1px solid rgba(0,0,0,0.15);
-	border-radius: 4px;
-	-webkit-box-shadow: 0 6px 12px rgba(0,0,0,0.175);
-	box-shadow: 0 6px 12px rgba(0,0,0,0.175);
-	background-clip: padding-box
-}
-.dropdown-menu.pull-right {
-	right: 0;
-	left: auto
-}
-.dropdown-menu .divider {
-	height: 1px;
-	margin: 9px 0;
-	overflow: hidden;
-	background-color: #e5e5e5
-}
-.dropdown-menu>li>a {
-	display: block;
-	padding: 3px 20px;
-	clear: both;
-	font-weight: normal;
-	line-height: 1.428571429;
-	color: #333;
-	white-space: nowrap
-}
-.dropdown-menu>li>a:hover, .dropdown-menu>li>a:focus {
-	color: #fff;
-	text-decoration: none;
-	background-color: #428bca
-}
-.dropdown-menu>.active>a, .dropdown-menu>.active>a:hover, .dropdown-menu>.active>a:focus {
-	color: #fff;
-	text-decoration: none;
-	background-color: #428bca;
-	outline: 0
-}
-.dropdown-menu>.disabled>a, .dropdown-menu>.disabled>a:hover, .dropdown-menu>.disabled>a:focus {
-	color: #999
-}
-.dropdown-menu>.disabled>a:hover, .dropdown-menu>.disabled>a:focus {
-	text-decoration: none;
-	cursor: not-allowed;
-	background-color: transparent;
-	background-image: none;
-filter:progid:DXImageTransform.Microsoft.gradient(enabled=false)
-}
-.open>.dropdown-menu {
-	display: block
-}
-.open>a {
-	outline: 0
-}
-.dropdown-header {
-	display: block;
-	padding: 3px 20px;
-	font-size: 12px;
-	line-height: 1.428571429;
-	color: #999
-}
-.dropdown-backdrop {
-	position: fixed;
-	top: 0;
-	right: 0;
-	bottom: 0;
-	left: 0;
-	z-index: 990
-}
-.pull-right>.dropdown-menu {
-	right: 0;
-	left: auto
-}
-.dropup .caret, .navbar-fixed-bottom .dropdown .caret {
-	border-top: 0 dotted;
-	border-bottom: 4px solid #000;
-	content: ""
-}
-.dropup .dropdown-menu, .navbar-fixed-bottom .dropdown .dropdown-menu {
-	top: auto;
-	bottom: 100%;
-	margin-bottom: 1px
-}
-@media(min-width:768px) {
-.navbar-right .dropdown-menu {
-	right: 0;
-	left: auto
-}
-}
-.btn-default .caret {
-	border-top-color: #333
-}
-.btn-primary .caret, .btn-success .caret, .btn-warning .caret, .btn-danger .caret, .btn-info .caret {
-	border-top-color: #fff
-}
-.dropup .btn-default .caret {
-	border-bottom-color: #333
-}
-.dropup .btn-primary .caret, .dropup .btn-success .caret, .dropup .btn-warning .caret, .dropup .btn-danger .caret, .dropup .btn-info .caret {
-	border-bottom-color: #fff
-}
-.btn-group, .btn-group-vertical {
-	position: relative;
-	display: inline-block;
-	vertical-align: middle
-}
-.btn-group>.btn, .btn-group-vertical>.btn {
-	position: relative;
-	float: left
-}
-.btn-group>.btn:hover, .btn-group-vertical>.btn:hover, .btn-group>.btn:focus, .btn-group-vertical>.btn:focus, .btn-group>.btn:active, .btn-group-vertical>.btn:active, .btn-group>.btn.active, .btn-group-vertical>.btn.active {
-	z-index: 2
-}
-.btn-group>.btn:focus, .btn-group-vertical>.btn:focus {
-	outline: 0
-}
-.btn-group .btn+.btn, .btn-group .btn+.btn-group, .btn-group .btn-group+.btn, .btn-group .btn-group+.btn-group {
-	margin-left: -1px
-}
-.btn-toolbar:before, .btn-toolbar:after {
-	display: table;
-	content: " "
-}
-.btn-toolbar:after {
-	clear: both
-}
-.btn-toolbar:before, .btn-toolbar:after {
-	display: table;
-	content: " "
-}
-.btn-toolbar:after {
-	clear: both
-}
-.btn-toolbar .btn-group {
-	float: left
-}
-.btn-toolbar>.btn+.btn, .btn-toolbar>.btn-group+.btn, .btn-toolbar>.btn+.btn-group, .btn-toolbar>.btn-group+.btn-group {
-	margin-left: 5px
-}
-.btn-group>.btn:not(:first-child):not(:last-child):not(.dropdown-toggle) {
-	border-radius: 0
-}
-.btn-group>.btn:first-child {
-	margin-left: 0
-}
-.btn-group>.btn:first-child:not(:last-child):not(.dropdown-toggle) {
-	border-top-right-radius: 0;
-	border-bottom-right-radius: 0
-}
-.btn-group>.btn:last-child:not(:first-child), .btn-group>.dropdown-toggle:not(:first-child) {
-	border-bottom-left-radius: 0;
-	border-top-left-radius: 0
-}
-.btn-group>.btn-group {
-	float: left
-}
-.btn-group>.btn-group:not(:first-child):not(:last-child)>.btn {
-	border-radius: 0
-}
-.btn-group>.btn-group:first-child>.btn:last-child, .btn-group>.btn-group:first-child>.dropdown-toggle {
-	border-top-right-radius: 0;
-	border-bottom-right-radius: 0
-}
-.btn-group>.btn-group:last-child>.btn:first-child {
-	border-bottom-left-radius: 0;
-	border-top-left-radius: 0
-}
-.btn-group .dropdown-toggle:active, .btn-group.open .dropdown-toggle {
-	outline: 0
-}
-.btn-group-xs>.btn {
-	padding: 5px 10px;
-	padding: 1px 5px;
-	font-size: 12px;
-	line-height: 1.5;
-	border-radius: 3px
-}
-.btn-group-sm>.btn {
-	padding: 5px 10px;
-	font-size: 12px;
-	line-height: 1.5;
-	border-radius: 3px
-}
-.btn-group-lg>.btn {
-	padding: 10px 16px;
-	font-size: 18px;
-	line-height: 1.33;
-	border-radius: 6px
-}
-.btn-group>.btn+.dropdown-toggle {
-	padding-right: 8px;
-	padding-left: 8px
-}
-.btn-group>.btn-lg+.dropdown-toggle {
-	padding-right: 12px;
-	padding-left: 12px
-}
-.btn-group.open .dropdown-toggle {
-	-webkit-box-shadow: inset 0 3px 5px rgba(0,0,0,0.125);
-	box-shadow: inset 0 3px 5px rgba(0,0,0,0.125)
-}
-.btn .caret {
-	margin-left: 0
-}
-.btn-lg .caret {
-	border-width: 5px 5px 0;
-	border-bottom-width: 0
-}
-.dropup .btn-lg .caret {
-	border-width: 0 5px 5px
-}
-.btn-group-vertical>.btn, .btn-group-vertical>.btn-group {
-	display: block;
-	float: none;
-	width: 100%;
-	max-width: 100%
-}
-.btn-group-vertical>.btn-group:before, .btn-group-vertical>.btn-group:after {
-	display: table;
-	content: " "
-}
-.btn-group-vertical>.btn-group:after {
-	clear: both
-}
-.btn-group-vertical>.btn-group:before, .btn-group-vertical>.btn-group:after {
-	display: table;
-	content: " "
-}
-.btn-group-vertical>.btn-group:after {
-	clear: both
-}
-.btn-group-vertical>.btn-group>.btn {
-	float: none
-}
-.btn-group-vertical>.btn+.btn, .btn-group-vertical>.btn+.btn-group, .btn-group-vertical>.btn-group+.btn, .btn-group-vertical>.btn-group+.btn-group {
-	margin-top: -1px;
-	margin-left: 0
-}
-.btn-group-vertical>.btn:not(:first-child):not(:last-child) {
-	border-radius: 0
-}
-.btn-group-vertical>.btn:first-child:not(:last-child) {
-	border-top-right-radius: 4px;
-	border-bottom-right-radius: 0;
-	border-bottom-left-radius: 0
-}
-.btn-group-vertical>.btn:last-child:not(:first-child) {
-	border-top-right-radius: 0;
-	border-bottom-left-radius: 4px;
-	border-top-left-radius: 0
-}
-.btn-group-vertical>.btn-group:not(:first-child):not(:last-child)>.btn {
-	border-radius: 0
-}
-.btn-group-vertical>.btn-group:first-child>.btn:last-child, .btn-group-vertical>.btn-group:first-child>.dropdown-toggle {
-	border-bottom-right-radius: 0;
-	border-bottom-left-radius: 0
-}
-.btn-group-vertical>.btn-group:last-child>.btn:first-child {
-	border-top-right-radius: 0;
-	border-top-left-radius: 0
-}
-.btn-group-justified {
-	display: table;
-	width: 100%;
-	border-collapse: separate;
-	table-layout: fixed
-}
-.btn-group-justified .btn {
-	display: table-cell;
-	float: none;
-	width: 1%
-}
-[data-toggle="buttons"]>.btn>input[type="radio"], [data-toggle="buttons"]>.btn>input[type="checkbox"] {
-display:none
-}
-.input-group {
-	position: relative;
-	display: table;
-	border-collapse: separate
-}
-.input-group.col {
-	float: none;
-	padding-right: 0;
-	padding-left: 0
-}
-.input-group .form-control {
-	width: 100%;
-	margin-bottom: 0
-}
-.input-group-lg>.form-control, .input-group-lg>.input-group-addon, .input-group-lg>.input-group-btn>.btn {
-	height: 45px;
-	padding: 10px 16px;
-	font-size: 18px;
-	line-height: 1.33;
-	border-radius: 6px
-}
-select.input-group-lg>.form-control, select.input-group-lg>.input-group-addon, select.input-group-lg>.input-group-btn>.btn {
-	height: 45px;
-	line-height: 45px
-}
-textarea.input-group-lg>.form-control, textarea.input-group-lg>.input-group-addon, textarea.input-group-lg>.input-group-btn>.btn {
-	height: auto
-}
-.input-group-sm>.form-control, .input-group-sm>.input-group-addon, .input-group-sm>.input-group-btn>.btn {
-	height: 30px;
-	padding: 5px 10px;
-	font-size: 12px;
-	line-height: 1.5;
-	border-radius: 3px
-}
-select.input-group-sm>.form-control, select.input-group-sm>.input-group-addon, select.input-group-sm>.input-group-btn>.btn {
-	height: 30px;
-	line-height: 30px
-}
-textarea.input-group-sm>.form-control, textarea.input-group-sm>.input-group-addon, textarea.input-group-sm>.input-group-btn>.btn {
-	height: auto
-}
-.input-group-addon, .input-group-btn, .input-group .form-control {
-	display: table-cell
-}
-.input-group-addon:not(:first-child):not(:last-child), .input-group-btn:not(:first-child):not(:last-child), .input-group .form-control:not(:first-child):not(:last-child) {
-	border-radius: 0
-}
-.input-group-addon, .input-group-btn {
-	width: 1%;
-	white-space: nowrap;
-	vertical-align: middle
-}
-.input-group-addon {
-	padding: 6px 12px;
-	font-size: 14px;
-	font-weight: normal;
-	line-height: 1;
-	text-align: center;
-	background-color: #eee;
-	border: 1px solid #ccc;
-	border-radius: 4px
-}
-.input-group-addon.input-sm {
-	padding: 5px 10px;
-	font-size: 12px;
-	border-radius: 3px
-}
-.input-group-addon.input-lg {
-	padding: 10px 16px;
-	font-size: 18px;
-	border-radius: 6px
-}
-.input-group-addon input[type="radio"], .input-group-addon input[type="checkbox"] {
-	margin-top: 0
-}
-.input-group .form-control:first-child, .input-group-addon:first-child, .input-group-btn:first-child>.btn, .input-group-btn:first-child>.dropdown-toggle, .input-group-btn:last-child>.btn:not(:last-child):not(.dropdown-toggle) {
-	border-top-right-radius: 0;
-	border-bottom-right-radius: 0
-}
-.input-group-addon:first-child {
-	border-right: 0
-}
-.input-group .form-control:last-child, .input-group-addon:last-child, .input-group-btn:last-child>.btn, .input-group-btn:last-child>.dropdown-toggle, .input-group-btn:first-child>.btn:not(:first-child) {
-	border-bottom-left-radius: 0;
-	border-top-left-radius: 0
-}
-.input-group-addon:last-child {
-	border-left: 0
-}
-.input-group-btn {
-	position: relative;
-	white-space: nowrap
-}
-.input-group-btn>.btn {
-	position: relative
-}
-.input-group-btn>.btn+.btn {
-	margin-left: -4px
-}
-.input-group-btn>.btn:hover, .input-group-btn>.btn:active {
-	z-index: 2
-}
-.nav {
-	padding-left: 0;
-	margin-bottom: 0;
-	list-style: none
-}
-.nav:before, .nav:after {
-	display: table;
-	content: " "
-}
-.nav:after {
-	clear: both
-}
-.nav:before, .nav:after {
-	display: table;
-	content: " "
-}
-.nav:after {
-	clear: both
-}
-.nav>li {
-	position: relative;
-	display: block
-}
-.nav>li>a {
-	position: relative;
-	display: block;
-	padding: 10px 15px
-}
-.nav>li>a:hover, .nav>li>a:focus {
-	text-decoration: none;
-	background-color: #eee
-}
-.nav>li.disabled>a {
-	color: #999
-}
-.nav>li.disabled>a:hover, .nav>li.disabled>a:focus {
-	color: #999;
-	text-decoration: none;
-	cursor: not-allowed;
-	background-color: transparent
-}
-.nav .open>a, .nav .open>a:hover, .nav .open>a:focus {
-	background-color: #eee;
-	border-color: #428bca
-}
-.nav .nav-divider {
-	height: 1px;
-	margin: 9px 0;
-	overflow: hidden;
-	background-color: #e5e5e5
-}
-.nav>li>a>img {
-	max-width: none
-}
-.nav-tabs {
-	border-bottom: 1px solid #ddd
-}
-.nav-tabs>li {
-	float: left;
-	margin-bottom: -1px
-}
-
-.nav-tabs>li>a {
-	margin-right: 2px;
-	line-height: 1.428571429;
-	border: 1px solid transparent;
-	border-radius: 4px 4px 0 0
-}
-
-
-
-.nav-tabs>li>a:hover {
-	border-color: #eee #eee #ddd
-}
-.nav-tabs>li.active>a, .nav-tabs>li.active>a:hover, .nav-tabs>li.active>a:focus {
-	color: #555;
-	cursor: default;
-	background-color: #fff;
-	border: 1px solid #ddd;
-	border-bottom-color: transparent
-}
-.nav-tabs.nav-justified {
-	width: 100%;
-	border-bottom: 0
-}
-.nav-tabs.nav-justified>li {
-	float: none
-}
-.nav-tabs.nav-justified>li>a {
-	text-align: center
-}
-@media(min-width:768px) {
-.nav-tabs.nav-justified>li {
-	display: table-cell;
-	width: 1%
-}
-}
-.nav-tabs.nav-justified>li>a {
-	margin-right: 0;
-	border-bottom: 1px solid #ddd
-}
-.nav-tabs.nav-justified>.active>a {
-	border-bottom-color: #fff
-}
-.nav-pills>li {
-	float: left
-}
-.nav-pills>li>a {
-	border-radius: 5px
-}
-.nav-pills>li+li {
-	margin-left: 2px
-}
-.nav-pills>li.active>a, .nav-pills>li.active>a:hover, .nav-pills>li.active>a:focus {
-	color: #fff;
-	background-color: #428bca
-}
-.nav-stacked>li {
-	float: none
-}
-.nav-stacked>li+li {
-	margin-top: 2px;
-	margin-left: 0
-}
-.nav-justified {
-	width: 100%
-}
-.nav-justified>li {
-	float: none
-}
-.nav-justified>li>a {
-	text-align: center
-}
-@media(min-width:768px) {
-.nav-justified>li {
-	display: table-cell;
-	width: 1%
-}
-}
-.nav-tabs-justified {
-	border-bottom: 0
-}
-.nav-tabs-justified>li>a {
-	margin-right: 0;
-	border-bottom: 1px solid #ddd
-}
-.nav-tabs-justified>.active>a {
-	border-bottom-color: #fff
-}
-.tabbable:before, .tabbable:after {
-	display: table;
-	content: " "
-}
-.tabbable:after {
-	clear: both
-}
-.tabbable:before, .tabbable:after {
-	display: table;
-	content: " "
-}
-.tabbable:after {
-	clear: both
-}
-.tab-content>.tab-pane, .pill-content>.pill-pane {
-	display: none
-}
-.tab-content>.active, .pill-content>.active {
-	display: block
-}
-.nav .caret {
-	border-top-color: #428bca;
-	border-bottom-color: #428bca
-}
-.nav a:hover .caret {
-	border-top-color: #2a6496;
-	border-bottom-color: #2a6496
-}
-.nav-tabs .dropdown-menu {
-	margin-top: -1px;
-	border-top-right-radius: 0;
-	border-top-left-radius: 0
-}
-.navbar {
-	position: relative;
-	z-index: 1000;
-	min-height: 50px;
-	margin-bottom: 20px;
-	border: 1px solid transparent
-}
-.navbar:before, .navbar:after {
-	display: table;
-	content: " "
-}
-.navbar:after {
-	clear: both
-}
-.navbar:before, .navbar:after {
-	display: table;
-	content: " "
-}
-.navbar:after {
-	clear: both
-}
-@media(min-width:768px) {
-.navbar {
-	border-radius: 4px
-}
-}
-.navbar-header:before, .navbar-header:after {
-	display: table;
-	content: " "
-}
-.navbar-header:after {
-	clear: both
-}
-.navbar-header:before, .navbar-header:after {
-	display: table;
-	content: " "
-}
-.navbar-header:after {
-	clear: both
-}
-@media(min-width:768px) {
-.navbar-header {
-	float: left
-}
-}
-.navbar-collapse {
-	max-height: 340px;
-	padding-right: 15px;
-	padding-left: 15px;
-	overflow-x: visible;
-	border-top: 1px solid transparent;
-	box-shadow: inset 0 1px 0 rgba(255,255,255,0.1);
-	-webkit-overflow-scrolling: touch
-}
-.navbar-collapse:before, .navbar-collapse:after {
-	display: table;
-	content: " "
-}
-.navbar-collapse:after {
-	clear: both
-}
-.navbar-collapse:before, .navbar-collapse:after {
-	display: table;
-	content: " "
-}
-.navbar-collapse:after {
-	clear: both
-}
-.navbar-collapse.in {
-	overflow-y: auto
-}
-@media(min-width:768px) {
-.navbar-collapse {
-	width: auto;
-	border-top: 0;
-	box-shadow: none
-}
-.navbar-collapse.collapse {
-	display: block!important;
-	height: auto!important;
-	padding-bottom: 0;
-	overflow: visible!important
-}
-.navbar-collapse.in {
-	overflow-y: visible
-}
-.navbar-collapse .navbar-nav.navbar-left:first-child {
-	margin-left: -15px
-}
-.navbar-collapse .navbar-nav.navbar-right:last-child {
-	margin-right: -15px
-}
-.navbar-collapse .navbar-text:last-child {
-	margin-right: 0
-}
-}
-.container>.navbar-header, .container>.navbar-collapse {
-	margin-right: -15px;
-	margin-left: -15px
-}
-@media(min-width:768px) {
-.container>.navbar-header, .container>.navbar-collapse {
-	margin-right: 0;
-	margin-left: 0
-}
-}
-.navbar-static-top {
-	border-width: 0 0 1px
-}
-@media(min-width:768px) {
-.navbar-static-top {
-	border-radius: 0
-}
-}
-.navbar-fixed-top, .navbar-fixed-bottom {
-	position: fixed;
-	right: 0;
-	left: 0;
-	border-width: 0 0 1px
-}
-@media(min-width:768px) {
-.navbar-fixed-top, .navbar-fixed-bottom {
-	border-radius: 0
-}
-}
-.navbar-fixed-top {
-	top: 0;
-	z-index: 1030
-}
-.navbar-fixed-bottom {
-	bottom: 0;
-	margin-bottom: 0
-}
-.navbar-brand {
-	float: left;
-	padding: 15px 15px;
-	font-size: 18px;
-	line-height: 20px
-}
-.navbar-brand:hover, .navbar-brand:focus {
-	text-decoration: none
-}
-@media(min-width:768px) {
-.navbar>.container .navbar-brand {
-	margin-left: -15px
-}
-}
-.navbar-toggle {
-	position: relative;
-	float: right;
-	padding: 9px 10px;
-	margin-top: 8px;
-	margin-right: 15px;
-	margin-bottom: 8px;
-	background-color: transparent;
-	border: 1px solid transparent;
-	border-radius: 4px
-}
-.navbar-toggle .icon-bar {
-	display: block;
-	width: 22px;
-	height: 2px;
-	border-radius: 1px
-}
-.navbar-toggle .icon-bar+.icon-bar {
-	margin-top: 4px
-}
-@media(min-width:768px) {
-.navbar-toggle {
-	display: none
-}
-}
-.navbar-nav {
-	margin: 7.5px -15px
-}
-.navbar-nav>li>a {
-	padding-top: 10px;
-	padding-bottom: 10px;
-	line-height: 20px
-}
-@media(max-width:767px) {
-.navbar-nav .open .dropdown-menu {
-	position: static;
-	float: none;
-	width: auto;
-	margin-top: 0;
-	background-color: transparent;
-	border: 0;
-	box-shadow: none
-}
-.navbar-nav .open .dropdown-menu>li>a, .navbar-nav .open .dropdown-menu .dropdown-header {
-	padding: 5px 15px 5px 25px
-}
-.navbar-nav .open .dropdown-menu>li>a {
-	line-height: 20px
-}
-.navbar-nav .open .dropdown-menu>li>a:hover, .navbar-nav .open .dropdown-menu>li>a:focus {
-	background-image: none
-}
-}
-@media(min-width:768px) {
-.navbar-nav {
-	float: left;
-	margin: 0
-}
-.navbar-nav>li {
-	float: left
-}
-.navbar-nav>li>a {
-	padding-top: 15px;
-	padding-bottom: 15px
-}
-}
-@media(min-width:768px) {
-.navbar-left {
-	float: left!important
-}
-.navbar-right {
-	float: right!important
-}
-}
-.navbar-form {
-	padding: 10px 15px;
-	margin-top: 8px;
-	margin-right: -15px;
-	margin-bottom: 8px;
-	margin-left: -15px;
-	border-top: 1px solid transparent;
-	border-bottom: 1px solid transparent;
-	-webkit-box-shadow: inset 0 1px 0 rgba(255,255,255,0.1), 0 1px 0 rgba(255,255,255,0.1);
-	box-shadow: inset 0 1px 0 rgba(255,255,255,0.1), 0 1px 0 rgba(255,255,255,0.1)
-}
-@media(min-width:768px) {
-.navbar-form .form-group {
-	display: inline-block;
-	margin-bottom: 0;
-	vertical-align: middle
-}
-.navbar-form .form-control {
-	display: inline-block
-}
-.navbar-form .radio, .navbar-form .checkbox {
-	display: inline-block;
-	padding-left: 0;
-	margin-top: 0;
-	margin-bottom: 0
-}
-.navbar-form .radio input[type="radio"], .navbar-form .checkbox input[type="checkbox"] {
-	float: none;
-	margin-left: 0
-}
-}
-@media(max-width:767px) {
-.navbar-form .form-group {
-	margin-bottom: 5px
-}
-}
-@media(min-width:768px) {
-.navbar-form {
-	width: auto;
-	padding-top: 0;
-	padding-bottom: 0;
-	margin-right: 0;
-	margin-left: 0;
-	border: 0;
-	-webkit-box-shadow: none;
-	box-shadow: none
-}
-}
-.navbar-nav>li>.dropdown-menu {
-	margin-top: 0;
-	border-top-right-radius: 0;
-	border-top-left-radius: 0
-}
-.navbar-fixed-bottom .navbar-nav>li>.dropdown-menu {
-	border-bottom-right-radius: 0;
-	border-bottom-left-radius: 0
-}
-.navbar-nav.pull-right>li>.dropdown-menu, .navbar-nav>li>.dropdown-menu.pull-right {
-	right: 0;
-	left: auto
-}
-.navbar-btn {
-	margin-top: 8px;
-	margin-bottom: 8px
-}
-.navbar-text {
-	float: left;
-	margin-top: 15px;
-	margin-bottom: 15px
-}
-@media(min-width:768px) {
-.navbar-text {
-	margin-right: 15px;
-	margin-left: 15px
-}
-}
-.navbar-default {
-	background-color: #f8f8f8;
-	border-color: #e7e7e7
-}
-.navbar-default .navbar-brand {
-	color: #777
-}
-.navbar-default .navbar-brand:hover, .navbar-default .navbar-brand:focus {
-	color: #5e5e5e;
-	background-color: transparent
-}
-.navbar-default .navbar-text {
-	color: #777
-}
-.navbar-default .navbar-nav>li>a {
-	color: #777
-}
-.navbar-default .navbar-nav>li>a:hover, .navbar-default .navbar-nav>li>a:focus {
-	color: #333;
-	background-color: transparent
-}
-.navbar-default .navbar-nav>.active>a, .navbar-default .navbar-nav>.active>a:hover, .navbar-default .navbar-nav>.active>a:focus {
-	color: #555;
-	background-color: #e7e7e7
-}
-.navbar-default .navbar-nav>.disabled>a, .navbar-default .navbar-nav>.disabled>a:hover, .navbar-default .navbar-nav>.disabled>a:focus {
-	color: #ccc;
-	background-color: transparent
-}
-.navbar-default .navbar-toggle {
-	border-color: #ddd
-}
-.navbar-default .navbar-toggle:hover, .navbar-default .navbar-toggle:focus {
-	background-color: #ddd
-}
-.navbar-default .navbar-toggle .icon-bar {
-	background-color: #ccc
-}
-.navbar-default .navbar-collapse, .navbar-default .navbar-form {
-	border-color: #e6e6e6
-}
-.navbar-default .navbar-nav>.dropdown>a:hover .caret, .navbar-default .navbar-nav>.dropdown>a:focus .caret {
-	border-top-color: #333;
-	border-bottom-color: #333
-}
-.navbar-default .navbar-nav>.open>a, .navbar-default .navbar-nav>.open>a:hover, .navbar-default .navbar-nav>.open>a:focus {
-	color: #555;
-	background-color: #e7e7e7
-}
-.navbar-default .navbar-nav>.open>a .caret, .navbar-default .navbar-nav>.open>a:hover .caret, .navbar-default .navbar-nav>.open>a:focus .caret {
-	border-top-color: #555;
-	border-bottom-color: #555
-}
-.navbar-default .navbar-nav>.dropdown>a .caret {
-	border-top-color: #777;
-	border-bottom-color: #777
-}
-@media(max-width:767px) {
-.navbar-default .navbar-nav .open .dropdown-menu>li>a {
-	color: #777
-}
-.navbar-default .navbar-nav .open .dropdown-menu>li>a:hover, .navbar-default .navbar-nav .open .dropdown-menu>li>a:focus {
-	color: #333;
-	background-color: transparent
-}
-.navbar-default .navbar-nav .open .dropdown-menu>.active>a, .navbar-default .navbar-nav .open .dropdown-menu>.active>a:hover, .navbar-default .navbar-nav .open .dropdown-menu>.active>a:focus {
-	color: #555;
-	background-color: #e7e7e7
-}
-.navbar-default .navbar-nav .open .dropdown-menu>.disabled>a, .navbar-default .navbar-nav .open .dropdown-menu>.disabled>a:hover, .navbar-default .navbar-nav .open .dropdown-menu>.disabled>a:focus {
-	color: #ccc;
-	background-color: transparent
-}
-}
-.navbar-default .navbar-link {
-	color: #777
-}
-.navbar-default .navbar-link:hover {
-	color: #333
-}
-.navbar-inverse {
-	background-color: #222;
-	border-color: #080808
-}
-.navbar-inverse .navbar-brand {
-	color: #999
-}
-.navbar-inverse .navbar-brand:hover, .navbar-inverse .navbar-brand:focus {
-	color: #fff;
-	background-color: transparent
-}
-.navbar-inverse .navbar-text {
-	color: #999
-}
-.navbar-inverse .navbar-nav>li>a {
-	color: #999
-}
-.navbar-inverse .navbar-nav>li>a:hover, .navbar-inverse .navbar-nav>li>a:focus {
-	color: #fff;
-	background-color: transparent
-}
-.navbar-inverse .navbar-nav>.active>a, .navbar-inverse .navbar-nav>.active>a:hover, .navbar-inverse .navbar-nav>.active>a:focus {
-	color: #fff;
-	background-color: #080808
-}
-.navbar-inverse .navbar-nav>.disabled>a, .navbar-inverse .navbar-nav>.disabled>a:hover, .navbar-inverse .navbar-nav>.disabled>a:focus {
-	color: #444;
-	background-color: transparent
-}
-.navbar-inverse .navbar-toggle {
-	border-color: #333
-}
-.navbar-inverse .navbar-toggle:hover, .navbar-inverse .navbar-toggle:focus {
-	background-color: #333
-}
-.navbar-inverse .navbar-toggle .icon-bar {
-	background-color: #fff
-}
-.navbar-inverse .navbar-collapse, .navbar-inverse .navbar-form {
-	border-color: #101010
-}
-.navbar-inverse .navbar-nav>.open>a, .navbar-inverse .navbar-nav>.open>a:hover, .navbar-inverse .navbar-nav>.open>a:focus {
-	color: #fff;
-	background-color: #080808
-}
-.navbar-inverse .navbar-nav>.dropdown>a:hover .caret {
-	border-top-color: #fff;
-	border-bottom-color: #fff
-}
-.navbar-inverse .navbar-nav>.dropdown>a .caret {
-	border-top-color: #999;
-	border-bottom-color: #999
-}
-.navbar-inverse .navbar-nav>.open>a .caret, .navbar-inverse .navbar-nav>.open>a:hover .caret, .navbar-inverse .navbar-nav>.open>a:focus .caret {
-	border-top-color: #fff;
-	border-bottom-color: #fff
-}
-@media(max-width:767px) {
-.navbar-inverse .navbar-nav .open .dropdown-menu>.dropdown-header {
-	border-color: #080808
-}
-.navbar-inverse .navbar-nav .open .dropdown-menu>li>a {
-	color: #999
-}
-.navbar-inverse .navbar-nav .open .dropdown-menu>li>a:hover, .navbar-inverse .navbar-nav .open .dropdown-menu>li>a:focus {
-	color: #fff;
-	background-color: transparent
-}
-.navbar-inverse .navbar-nav .open .dropdown-menu>.active>a, .navbar-inverse .navbar-nav .open .dropdown-menu>.active>a:hover, .navbar-inverse .navbar-nav .open .dropdown-menu>.active>a:focus {
-	color: #fff;
-	background-color: #080808
-}
-.navbar-inverse .navbar-nav .open .dropdown-menu>.disabled>a, .navbar-inverse .navbar-nav .open .dropdown-menu>.disabled>a:hover, .navbar-inverse .navbar-nav .open .dropdown-menu>.disabled>a:focus {
-	color: #444;
-	background-color: transparent
-}
-}
-.navbar-inverse .navbar-link {
-	color: #999
-}
-.navbar-inverse .navbar-link:hover {
-	color: #fff
-}
-.breadcrumb {
-	padding: 8px 15px;
-	margin-bottom: 20px;
-	list-style: none;
-	background-color: #f5f5f5;
-	border-radius: 4px
-}
-.breadcrumb>li {
-	display: inline-block
-}
-.breadcrumb>li+li:before {
-	padding: 0 5px;
-	color: #ccc;
-	content: "/\00a0"
-}
-.breadcrumb>.active {
-	color: #999
-}
-.pagination {
-	display: inline-block;
-	padding-left: 0;
-	margin: 20px 0;
-	border-radius: 4px
-}
-.pagination>li {
-	display: inline
-}
-.pagination>li>a, .pagination>li>span {
-	position: relative;
-	float: left;
-	padding: 6px 12px;
-	margin-left: -1px;
-	line-height: 1.428571429;
-	text-decoration: none;
-	background-color: #fff;
-	border: 1px solid #ddd
-}
-.pagination>li:first-child>a, .pagination>li:first-child>span {
-	margin-left: 0;
-	border-bottom-left-radius: 4px;
-	border-top-left-radius: 4px
-}
-.pagination>li:last-child>a, .pagination>li:last-child>span {
-	border-top-right-radius: 4px;
-	border-bottom-right-radius: 4px
-}
-.pagination>li>a:hover, .pagination>li>span:hover, .pagination>li>a:focus, .pagination>li>span:focus {
-	background-color: #eee
-}
-.pagination>.active>a, .pagination>.active>span, .pagination>.active>a:hover, .pagination>.active>span:hover, .pagination>.active>a:focus, .pagination>.active>span:focus {
-	z-index: 2;
-	color: #fff;
-	cursor: default;
-	background-color: #428bca;
-	border-color: #428bca
-}
-.pagination>.disabled>span, .pagination>.disabled>a, .pagination>.disabled>a:hover, .pagination>.disabled>a:focus {
-	color: #999;
-	cursor: not-allowed;
-	background-color: #fff;
-	border-color: #ddd
-}
-.pagination-lg>li>a, .pagination-lg>li>span {
-	padding: 10px 16px;
-	font-size: 18px
-}
-.pagination-lg>li:first-child>a, .pagination-lg>li:first-child>span {
-	border-bottom-left-radius: 6px;
-	border-top-left-radius: 6px
-}
-.pagination-lg>li:last-child>a, .pagination-lg>li:last-child>span {
-	border-top-right-radius: 6px;
-	border-bottom-right-radius: 6px
-}
-.pagination-sm>li>a, .pagination-sm>li>span {
-	padding: 5px 10px;
-	font-size: 12px
-}
-.pagination-sm>li:first-child>a, .pagination-sm>li:first-child>span {
-	border-bottom-left-radius: 3px;
-	border-top-left-radius: 3px
-}
-.pagination-sm>li:last-child>a, .pagination-sm>li:last-child>span {
-	border-top-right-radius: 3px;
-	border-bottom-right-radius: 3px
-}
-.pager {
-	padding-left: 0;
-	margin: 20px 0;
-	text-align: center;
-	list-style: none
-}
-.pager:before, .pager:after {
-	display: table;
-	content: " "
-}
-.pager:after {
-	clear: both
-}
-.pager:before, .pager:after {
-	display: table;
-	content: " "
-}
-.pager:after {
-	clear: both
-}
-.pager li {
-	display: inline
-}
-.pager li>a, .pager li>span {
-	display: inline-block;
-	padding: 5px 14px;
-	background-color: #fff;
-	border: 1px solid #ddd;
-	border-radius: 15px
-}
-.pager li>a:hover, .pager li>a:focus {
-	text-decoration: none;
-	background-color: #eee
-}
-.pager .next>a, .pager .next>span {
-	float: right
-}
-.pager .previous>a, .pager .previous>span {
-	float: left
-}
-.pager .disabled>a, .pager .disabled>a:hover, .pager .disabled>a:focus, .pager .disabled>span {
-	color: #999;
-	cursor: not-allowed;
-	background-color: #fff
-}
-.label {
-	display: inline;
-	padding: .2em .6em .3em;
-	font-size: 75%;
-	font-weight: bold;
-	line-height: 1;
-	color: #fff;
-	text-align: center;
-	white-space: nowrap;
-	vertical-align: baseline;
-	border-radius: .25em
-}
-.label[href]:hover, .label[href]:focus {
-	color: #fff;
-	text-decoration: none;
-	cursor: pointer
-}
-.label:empty {
-	display: none
-}
-.label-default {
-	background-color: #999
-}
-.label-default[href]:hover, .label-default[href]:focus {
-	background-color: #808080
-}
-.label-primary {
-	background-color: #428bca
-}
-.label-primary[href]:hover, .label-primary[href]:focus {
-	background-color: #3071a9
-}
-.label-success {
-	background-color: #5cb85c
-}
-.label-success[href]:hover, .label-success[href]:focus {
-	background-color: #449d44
-}
-.label-info {
-	background-color: #5bc0de
-}
-.label-info[href]:hover, .label-info[href]:focus {
-	background-color: #31b0d5
-}
-.label-warning {
-	background-color: #f0ad4e
-}
-.label-warning[href]:hover, .label-warning[href]:focus {
-	background-color: #ec971f
-}
-.label-danger {
-	background-color: #d9534f
-}
-.label-danger[href]:hover, .label-danger[href]:focus {
-	background-color: #c9302c
-}
-.badge {
-	display: inline-block;
-	min-width: 10px;
-	padding: 3px 7px;
-	font-size: 12px;
-	font-weight: bold;
-	line-height: 1;
-	color: #fff;
-	text-align: center;
-	white-space: nowrap;
-	vertical-align: baseline;
-	background-color: #999;
-	border-radius: 10px
-}
-.badge:empty {
-	display: none
-}
-a.badge:hover, a.badge:focus {
-	color: #fff;
-	text-decoration: none;
-	cursor: pointer
-}
-.btn .badge {
-	position: relative;
-	top: -1px
-}
-a.list-group-item.active>.badge, .nav-pills>.active>a>.badge {
-	color: #428bca;
-	background-color: #fff
-}
-.nav-pills>li>a>.badge {
-	margin-left: 3px
-}
-.jumbotron {
-	padding: 30px;
-	margin-bottom: 30px;
-	font-size: 21px;
-	font-weight: 200;
-	line-height: 2.1428571435;
-	color: inherit;
-	background-color: #eee
-}
-.jumbotron h1 {
-	line-height: 1;
-	color: inherit
-}
-.jumbotron p {
-	line-height: 1.4
-}
-.container .jumbotron {
-	border-radius: 6px
-}
-@media screen and (min-width:768px) {
-.jumbotron {
-	padding-top: 48px;
-	padding-bottom: 48px
-}
-.container .jumbotron {
-	padding-right: 60px;
-	padding-left: 60px
-}
-.jumbotron h1 {
-	font-size: 63px
-}
-}
-.thumbnail {
-	display: inline-block;
-	display: block;
-	height: auto;
-	max-width: 100%;
-	padding: 4px;
-	line-height: 1.428571429;
-	background-color: #fff;
-	border: 1px solid #ddd;
-	border-radius: 4px;
-	-webkit-transition: all .2s ease-in-out;
-	transition: all .2s ease-in-out
-}
-.thumbnail>img {
-	display: block;
-	height: auto;
-	max-width: 100%
-}
-a.thumbnail:hover, a.thumbnail:focus {
-	border-color: #428bca
-}
-.thumbnail>img {
-	margin-right: auto;
-	margin-left: auto
-}
-.thumbnail .caption {
-	padding: 9px;
-	color: #333
-}
-.alert {
-	padding: 15px;
-	margin-bottom: 20px;
-	border: 1px solid transparent;
-	border-radius: 4px
-}
-.alert h4 {
-	margin-top: 0;
-	color: inherit
-}
-.alert .alert-link {
-	font-weight: bold
-}
-.alert>p, .alert>ul {
-	margin-bottom: 0
-}
-.alert>p+p {
-	margin-top: 5px
-}
-.alert-dismissable {
-	padding-right: 35px
-}
-.alert-dismissable .close {
-	position: relative;
-	top: -2px;
-	right: -21px;
-	color: inherit
-}
-.alert-success {
-	color: #468847;
-	background-color: #dff0d8;
-	border-color: #d6e9c6
-}
-.alert-success hr {
-	border-top-color: #c9e2b3
-}
-.alert-success .alert-link {
-	color: #356635
-}
-.alert-info {
-	color: #3a87ad;
-	background-color: #d9edf7;
-	border-color: #bce8f1
-}
-.alert-info hr {
-	border-top-color: #a6e1ec
-}
-.alert-info .alert-link {
-	color: #2d6987
-}
-.alert-warning {
-	color: #c09853;
-	background-color: #fcf8e3;
-	border-color: #fbeed5
-}
-.alert-warning hr {
-	border-top-color: #f8e5be
-}
-.alert-warning .alert-link {
-	color: #a47e3c
-}
-.alert-danger {
-	color: #b94a48;
-	background-color: #f2dede;
-	border-color: #eed3d7
-}
-.alert-danger hr {
-	border-top-color: #e6c1c7
-}
-.alert-danger .alert-link {
-	color: #953b39
-}
-@-webkit-keyframes progress-bar-stripes {
-from {
-background-position:40px 0
-}
-to {
-	background-position: 0 0
-}
-}
-@-moz-keyframes progress-bar-stripes {
-from {
-background-position:40px 0
-}
-to {
-	background-position: 0 0
-}
-}
-@-o-keyframes progress-bar-stripes {
-from {
-background-position:0 0
-}
-to {
-	background-position: 40px 0
-}
-}
-@keyframes progress-bar-stripes {
-from {
-background-position:40px 0
-}
-to {
-	background-position: 0 0
-}
-}
-.progress {
-	height: 20px;
-	margin-bottom: 20px;
-	overflow: hidden;
-	background-color: #f5f5f5;
-	border-radius: 4px;
-	-webkit-box-shadow: inset 0 1px 2px rgba(0,0,0,0.1);
-	box-shadow: inset 0 1px 2px rgba(0,0,0,0.1)
-}
-.progress-bar {
-	float: left;
-	width: 0;
-	height: 100%;
-	font-size: 12px;
-	color: #fff;
-	text-align: center;
-	background-color: #428bca;
-	-webkit-box-shadow: inset 0 -1px 0 rgba(0,0,0,0.15);
-	box-shadow: inset 0 -1px 0 rgba(0,0,0,0.15);
-	-webkit-transition: width .6s ease;
-	transition: width .6s ease
-}
-.progress-striped .progress-bar {
-	background-image: -webkit-gradient(linear, 0 100%, 100% 0, color-stop(0.25, rgba(255,255,255,0.15)), color-stop(0.25, transparent), color-stop(0.5, transparent), color-stop(0.5, rgba(255,255,255,0.15)), color-stop(0.75, rgba(255,255,255,0.15)), color-stop(0.75, transparent), to(transparent));
-	background-image: -webkit-linear-gradient(45deg, rgba(255,255,255,0.15) 25%, transparent 25%, transparent 50%, rgba(255,255,255,0.15) 50%, rgba(255,255,255,0.15) 75%, transparent 75%, transparent);
-	background-image: -moz-linear-gradient(45deg, rgba(255,255,255,0.15) 25%, transparent 25%, transparent 50%, rgba(255,255,255,0.15) 50%, rgba(255,255,255,0.15) 75%, transparent 75%, transparent);
-	background-image: linear-gradient(45deg, rgba(255,255,255,0.15) 25%, transparent 25%, transparent 50%, rgba(255,255,255,0.15) 50%, rgba(255,255,255,0.15) 75%, transparent 75%, transparent);
-	background-size: 40px 40px
-}
-.progress.active .progress-bar {
-	-webkit-animation: progress-bar-stripes 2s linear infinite;
-	-moz-animation: progress-bar-stripes 2s linear infinite;
-	-ms-animation: progress-bar-stripes 2s linear infinite;
-	-o-animation: progress-bar-stripes 2s linear infinite;
-	animation: progress-bar-stripes 2s linear infinite
-}
-.progress-bar-success {
-	background-color: #5cb85c
-}
-.progress-striped .progress-bar-success {
-	background-image: -webkit-gradient(linear, 0 100%, 100% 0, color-stop(0.25, rgba(255,255,255,0.15)), color-stop(0.25, transparent), color-stop(0.5, transparent), color-stop(0.5, rgba(255,255,255,0.15)), color-stop(0.75, rgba(255,255,255,0.15)), color-stop(0.75, transparent), to(transparent));
-	background-image: -webkit-linear-gradient(45deg, rgba(255,255,255,0.15) 25%, transparent 25%, transparent 50%, rgba(255,255,255,0.15) 50%, rgba(255,255,255,0.15) 75%, transparent 75%, transparent);
-	background-image: -moz-linear-gradient(45deg, rgba(255,255,255,0.15) 25%, transparent 25%, transparent 50%, rgba(255,255,255,0.15) 50%, rgba(255,255,255,0.15) 75%, transparent 75%, transparent);
-	background-image: linear-gradient(45deg, rgba(255,255,255,0.15) 25%, transparent 25%, transparent 50%, rgba(255,255,255,0.15) 50%, rgba(255,255,255,0.15) 75%, transparent 75%, transparent)
-}
-.progress-bar-info {
-	background-color: #5bc0de
-}
-.progress-striped .progress-bar-info {
-	background-image: -webkit-gradient(linear, 0 100%, 100% 0, color-stop(0.25, rgba(255,255,255,0.15)), color-stop(0.25, transparent), color-stop(0.5, transparent), color-stop(0.5, rgba(255,255,255,0.15)), color-stop(0.75, rgba(255,255,255,0.15)), color-stop(0.75, transparent), to(transparent));
-	background-image: -webkit-linear-gradient(45deg, rgba(255,255,255,0.15) 25%, transparent 25%, transparent 50%, rgba(255,255,255,0.15) 50%, rgba(255,255,255,0.15) 75%, transparent 75%, transparent);
-	background-image: -moz-linear-gradient(45deg, rgba(255,255,255,0.15) 25%, transparent 25%, transparent 50%, rgba(255,255,255,0.15) 50%, rgba(255,255,255,0.15) 75%, transparent 75%, transparent);
-	background-image: linear-gradient(45deg, rgba(255,255,255,0.15) 25%, transparent 25%, transparent 50%, rgba(255,255,255,0.15) 50%, rgba(255,255,255,0.15) 75%, transparent 75%, transparent)
-}
-.progress-bar-warning {
-	background-color: #f0ad4e
-}
-.progress-striped .progress-bar-warning {
-	background-image: -webkit-gradient(linear, 0 100%, 100% 0, color-stop(0.25, rgba(255,255,255,0.15)), color-stop(0.25, transparent), color-stop(0.5, transparent), color-stop(0.5, rgba(255,255,255,0.15)), color-stop(0.75, rgba(255,255,255,0.15)), color-stop(0.75, transparent), to(transparent));
-	background-image: -webkit-linear-gradient(45deg, rgba(255,255,255,0.15) 25%, transparent 25%, transparent 50%, rgba(255,255,255,0.15) 50%, rgba(255,255,255,0.15) 75%, transparent 75%, transparent);
-	background-image: -moz-linear-gradient(45deg, rgba(255,255,255,0.15) 25%, transparent 25%, transparent 50%, rgba(255,255,255,0.15) 50%, rgba(255,255,255,0.15) 75%, transparent 75%, transparent);
-	background-image: linear-gradient(45deg, rgba(255,255,255,0.15) 25%, transparent 25%, transparent 50%, rgba(255,255,255,0.15) 50%, rgba(255,255,255,0.15) 75%, transparent 75%, transparent)
-}
-.progress-bar-danger {
-	background-color: #d9534f;
-}
-.progress-striped .progress-bar-danger {
-	background-image: -webkit-gradient(linear, 0 100%, 100% 0, color-stop(0.25, rgba(255,255,255,0.15)), color-stop(0.25, transparent), color-stop(0.5, transparent), color-stop(0.5, rgba(255,255,255,0.15)), color-stop(0.75, rgba(255,255,255,0.15)), color-stop(0.75, transparent), to(transparent));
-	background-image: -webkit-linear-gradient(45deg, rgba(255,255,255,0.15) 25%, transparent 25%, transparent 50%, rgba(255,255,255,0.15) 50%, rgba(255,255,255,0.15) 75%, transparent 75%, transparent);
-	background-image: -moz-linear-gradient(45deg, rgba(255,255,255,0.15) 25%, transparent 25%, transparent 50%, rgba(255,255,255,0.15) 50%, rgba(255,255,255,0.15) 75%, transparent 75%, transparent);
-	background-image: linear-gradient(45deg, rgba(255,255,255,0.15) 25%, transparent 25%, transparent 50%, rgba(255,255,255,0.15) 50%, rgba(255,255,255,0.15) 75%, transparent 75%, transparent)
-}
-.media, .media-body {
-	overflow: hidden;
-	zoom: 1
-}
-.media, .media .media {
-	margin-top: 15px
-}
-.media:first-child {
-	margin-top: 0
-}
-.media-object {
-	display: block
-}
-.media-heading {
-	margin: 0 0 5px
-}
-.media>.pull-left {
-	margin-right: 10px
-}
-.media>.pull-right {
-	margin-left: 10px
-}
-.media-list {
-	padding-left: 0;
-	list-style: none
-}
-.list-group {
-	padding-left: 0;
-	margin-bottom: 20px
-}
-.list-group-item {
-	position: relative;
-	display: block;
-	padding: 10px 15px;
-	margin-bottom: -1px;
-	background-color: #fff;
-	border: 1px solid #ddd
-}
-.list-group-item:first-child {
-	border-top-right-radius: 4px;
-	border-top-left-radius: 4px
-}
-.list-group-item:last-child {
-	margin-bottom: 0;
-	border-bottom-right-radius: 4px;
-	border-bottom-left-radius: 4px
-}
-.list-group-item>.badge {
-	float: right
-}
-.list-group-item>.badge+.badge {
-	margin-right: 5px
-}
-a.list-group-item {
-	color: #555
-}
-a.list-group-item .list-group-item-heading {
-	color: #333
-}
-a.list-group-item:hover, a.list-group-item:focus {
-	text-decoration: none;
-	background-color: #f5f5f5
-}
-.list-group-item.active, .list-group-item.active:hover, .list-group-item.active:focus {
-	z-index: 2;
-	color: #fff;
-	background-color: #428bca;
-	border-color: #428bca
-}
-.list-group-item.active .list-group-item-heading, .list-group-item.active:hover .list-group-it

<TRUNCATED>

[29/52] [abbrv] kylin git commit: KYLIN-1416 remove website from coding branch

Posted by li...@apache.org.
http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/assets/fonts/opensans-regular-webfont.woff
----------------------------------------------------------------------
diff --git a/website/assets/fonts/opensans-regular-webfont.woff b/website/assets/fonts/opensans-regular-webfont.woff
deleted file mode 100644
index ae0ffa4..0000000
Binary files a/website/assets/fonts/opensans-regular-webfont.woff and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/assets/images/core.png
----------------------------------------------------------------------
diff --git a/website/assets/images/core.png b/website/assets/images/core.png
deleted file mode 100644
index 0897899..0000000
Binary files a/website/assets/images/core.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/assets/images/data.png
----------------------------------------------------------------------
diff --git a/website/assets/images/data.png b/website/assets/images/data.png
deleted file mode 100644
index 11856a7..0000000
Binary files a/website/assets/images/data.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/assets/images/egg-logo.png
----------------------------------------------------------------------
diff --git a/website/assets/images/egg-logo.png b/website/assets/images/egg-logo.png
deleted file mode 100644
index c04e70d..0000000
Binary files a/website/assets/images/egg-logo.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/assets/images/feather-small.gif
----------------------------------------------------------------------
diff --git a/website/assets/images/feather-small.gif b/website/assets/images/feather-small.gif
deleted file mode 100644
index fb4445f..0000000
Binary files a/website/assets/images/feather-small.gif and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/assets/images/kylin_diagram.png
----------------------------------------------------------------------
diff --git a/website/assets/images/kylin_diagram.png b/website/assets/images/kylin_diagram.png
deleted file mode 100644
index 03aa717..0000000
Binary files a/website/assets/images/kylin_diagram.png and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/assets/images/kylin_logo.jpg
----------------------------------------------------------------------
diff --git a/website/assets/images/kylin_logo.jpg b/website/assets/images/kylin_logo.jpg
deleted file mode 100644
index b487b98..0000000
Binary files a/website/assets/images/kylin_logo.jpg and /dev/null differ

http://git-wip-us.apache.org/repos/asf/kylin/blob/0a74e9cb/website/assets/js/bootstrap.min.js
----------------------------------------------------------------------
diff --git a/website/assets/js/bootstrap.min.js b/website/assets/js/bootstrap.min.js
deleted file mode 100644
index 7c1561a..0000000
--- a/website/assets/js/bootstrap.min.js
+++ /dev/null
@@ -1,6 +0,0 @@
-/*!
- * Bootstrap v3.2.0 (http://getbootstrap.com)
- * Copyright 2011-2014 Twitter, Inc.
- * Licensed under MIT (https://github.com/twbs/bootstrap/blob/master/LICENSE)
- */
-if("undefined"==typeof jQuery)throw new Error("Bootstrap's JavaScript requires jQuery");+function(a){"use strict";function b(){var a=document.createElement("bootstrap"),b={WebkitTransition:"webkitTransitionEnd",MozTransition:"transitionend",OTransition:"oTransitionEnd otransitionend",transition:"transitionend"};for(var c in b)if(void 0!==a.style[c])return{end:b[c]};return!1}a.fn.emulateTransitionEnd=function(b){var c=!1,d=this;a(this).one("bsTransitionEnd",function(){c=!0});var e=function(){c||a(d).trigger(a.support.transition.end)};return setTimeout(e,b),this},a(function(){a.support.transition=b(),a.support.transition&&(a.event.special.bsTransitionEnd={bindType:a.support.transition.end,delegateType:a.support.transition.end,handle:function(b){return a(b.target).is(this)?b.handleObj.handler.apply(this,arguments):void 0}})})}(jQuery),+function(a){"use strict";function b(b){return this.each(function(){var c=a(this),e=c.data("bs.alert");e||c.data("bs.alert",e=new d(this)),"string"==type
 of b&&e[b].call(c)})}var c='[data-dismiss="alert"]',d=function(b){a(b).on("click",c,this.close)};d.VERSION="3.2.0",d.prototype.close=function(b){function c(){f.detach().trigger("closed.bs.alert").remove()}var d=a(this),e=d.attr("data-target");e||(e=d.attr("href"),e=e&&e.replace(/.*(?=#[^\s]*$)/,""));var f=a(e);b&&b.preventDefault(),f.length||(f=d.hasClass("alert")?d:d.parent()),f.trigger(b=a.Event("close.bs.alert")),b.isDefaultPrevented()||(f.removeClass("in"),a.support.transition&&f.hasClass("fade")?f.one("bsTransitionEnd",c).emulateTransitionEnd(150):c())};var e=a.fn.alert;a.fn.alert=b,a.fn.alert.Constructor=d,a.fn.alert.noConflict=function(){return a.fn.alert=e,this},a(document).on("click.bs.alert.data-api",c,d.prototype.close)}(jQuery),+function(a){"use strict";function b(b){return this.each(function(){var d=a(this),e=d.data("bs.button"),f="object"==typeof b&&b;e||d.data("bs.button",e=new c(this,f)),"toggle"==b?e.toggle():b&&e.setState(b)})}var c=function(b,d){this.$element=a(b)
 ,this.options=a.extend({},c.DEFAULTS,d),this.isLoading=!1};c.VERSION="3.2.0",c.DEFAULTS={loadingText:"loading..."},c.prototype.setState=function(b){var c="disabled",d=this.$element,e=d.is("input")?"val":"html",f=d.data();b+="Text",null==f.resetText&&d.data("resetText",d[e]()),d[e](null==f[b]?this.options[b]:f[b]),setTimeout(a.proxy(function(){"loadingText"==b?(this.isLoading=!0,d.addClass(c).attr(c,c)):this.isLoading&&(this.isLoading=!1,d.removeClass(c).removeAttr(c))},this),0)},c.prototype.toggle=function(){var a=!0,b=this.$element.closest('[data-toggle="buttons"]');if(b.length){var c=this.$element.find("input");"radio"==c.prop("type")&&(c.prop("checked")&&this.$element.hasClass("active")?a=!1:b.find(".active").removeClass("active")),a&&c.prop("checked",!this.$element.hasClass("active")).trigger("change")}a&&this.$element.toggleClass("active")};var d=a.fn.button;a.fn.button=b,a.fn.button.Constructor=c,a.fn.button.noConflict=function(){return a.fn.button=d,this},a(document).on("clic
 k.bs.button.data-api",'[data-toggle^="button"]',function(c){var d=a(c.target);d.hasClass("btn")||(d=d.closest(".btn")),b.call(d,"toggle"),c.preventDefault()})}(jQuery),+function(a){"use strict";function b(b){return this.each(function(){var d=a(this),e=d.data("bs.carousel"),f=a.extend({},c.DEFAULTS,d.data(),"object"==typeof b&&b),g="string"==typeof b?b:f.slide;e||d.data("bs.carousel",e=new c(this,f)),"number"==typeof b?e.to(b):g?e[g]():f.interval&&e.pause().cycle()})}var c=function(b,c){this.$element=a(b).on("keydown.bs.carousel",a.proxy(this.keydown,this)),this.$indicators=this.$element.find(".carousel-indicators"),this.options=c,this.paused=this.sliding=this.interval=this.$active=this.$items=null,"hover"==this.options.pause&&this.$element.on("mouseenter.bs.carousel",a.proxy(this.pause,this)).on("mouseleave.bs.carousel",a.proxy(this.cycle,this))};c.VERSION="3.2.0",c.DEFAULTS={interval:5e3,pause:"hover",wrap:!0},c.prototype.keydown=function(a){switch(a.which){case 37:this.prev();brea
 k;case 39:this.next();break;default:return}a.preventDefault()},c.prototype.cycle=function(b){return b||(this.paused=!1),this.interval&&clearInterval(this.interval),this.options.interval&&!this.paused&&(this.interval=setInterval(a.proxy(this.next,this),this.options.interval)),this},c.prototype.getItemIndex=function(a){return this.$items=a.parent().children(".item"),this.$items.index(a||this.$active)},c.prototype.to=function(b){var c=this,d=this.getItemIndex(this.$active=this.$element.find(".item.active"));return b>this.$items.length-1||0>b?void 0:this.sliding?this.$element.one("slid.bs.carousel",function(){c.to(b)}):d==b?this.pause().cycle():this.slide(b>d?"next":"prev",a(this.$items[b]))},c.prototype.pause=function(b){return b||(this.paused=!0),this.$element.find(".next, .prev").length&&a.support.transition&&(this.$element.trigger(a.support.transition.end),this.cycle(!0)),this.interval=clearInterval(this.interval),this},c.prototype.next=function(){return this.sliding?void 0:this.sli
 de("next")},c.prototype.prev=function(){return this.sliding?void 0:this.slide("prev")},c.prototype.slide=function(b,c){var d=this.$element.find(".item.active"),e=c||d[b](),f=this.interval,g="next"==b?"left":"right",h="next"==b?"first":"last",i=this;if(!e.length){if(!this.options.wrap)return;e=this.$element.find(".item")[h]()}if(e.hasClass("active"))return this.sliding=!1;var j=e[0],k=a.Event("slide.bs.carousel",{relatedTarget:j,direction:g});if(this.$element.trigger(k),!k.isDefaultPrevented()){if(this.sliding=!0,f&&this.pause(),this.$indicators.length){this.$indicators.find(".active").removeClass("active");var l=a(this.$indicators.children()[this.getItemIndex(e)]);l&&l.addClass("active")}var m=a.Event("slid.bs.carousel",{relatedTarget:j,direction:g});return a.support.transition&&this.$element.hasClass("slide")?(e.addClass(b),e[0].offsetWidth,d.addClass(g),e.addClass(g),d.one("bsTransitionEnd",function(){e.removeClass([b,g].join(" ")).addClass("active"),d.removeClass(["active",g].joi
 n(" ")),i.sliding=!1,setTimeout(function(){i.$element.trigger(m)},0)}).emulateTransitionEnd(1e3*d.css("transition-duration").slice(0,-1))):(d.removeClass("active"),e.addClass("active"),this.sliding=!1,this.$element.trigger(m)),f&&this.cycle(),this}};var d=a.fn.carousel;a.fn.carousel=b,a.fn.carousel.Constructor=c,a.fn.carousel.noConflict=function(){return a.fn.carousel=d,this},a(document).on("click.bs.carousel.data-api","[data-slide], [data-slide-to]",function(c){var d,e=a(this),f=a(e.attr("data-target")||(d=e.attr("href"))&&d.replace(/.*(?=#[^\s]+$)/,""));if(f.hasClass("carousel")){var g=a.extend({},f.data(),e.data()),h=e.attr("data-slide-to");h&&(g.interval=!1),b.call(f,g),h&&f.data("bs.carousel").to(h),c.preventDefault()}}),a(window).on("load",function(){a('[data-ride="carousel"]').each(function(){var c=a(this);b.call(c,c.data())})})}(jQuery),+function(a){"use strict";function b(b){return this.each(function(){var d=a(this),e=d.data("bs.collapse"),f=a.extend({},c.DEFAULTS,d.data(),
 "object"==typeof b&&b);!e&&f.toggle&&"show"==b&&(b=!b),e||d.data("bs.collapse",e=new c(this,f)),"string"==typeof b&&e[b]()})}var c=function(b,d){this.$element=a(b),this.options=a.extend({},c.DEFAULTS,d),this.transitioning=null,this.options.parent&&(this.$parent=a(this.options.parent)),this.options.toggle&&this.toggle()};c.VERSION="3.2.0",c.DEFAULTS={toggle:!0},c.prototype.dimension=function(){var a=this.$element.hasClass("width");return a?"width":"height"},c.prototype.show=function(){if(!this.transitioning&&!this.$element.hasClass("in")){var c=a.Event("show.bs.collapse");if(this.$element.trigger(c),!c.isDefaultPrevented()){var d=this.$parent&&this.$parent.find("> .panel > .in");if(d&&d.length){var e=d.data("bs.collapse");if(e&&e.transitioning)return;b.call(d,"hide"),e||d.data("bs.collapse",null)}var f=this.dimension();this.$element.removeClass("collapse").addClass("collapsing")[f](0),this.transitioning=1;var g=function(){this.$element.removeClass("collapsing").addClass("collapse in"
 )[f](""),this.transitioning=0,this.$element.trigger("shown.bs.collapse")};if(!a.support.transition)return g.call(this);var h=a.camelCase(["scroll",f].join("-"));this.$element.one("bsTransitionEnd",a.proxy(g,this)).emulateTransitionEnd(350)[f](this.$element[0][h])}}},c.prototype.hide=function(){if(!this.transitioning&&this.$element.hasClass("in")){var b=a.Event("hide.bs.collapse");if(this.$element.trigger(b),!b.isDefaultPrevented()){var c=this.dimension();this.$element[c](this.$element[c]())[0].offsetHeight,this.$element.addClass("collapsing").removeClass("collapse").removeClass("in"),this.transitioning=1;var d=function(){this.transitioning=0,this.$element.trigger("hidden.bs.collapse").removeClass("collapsing").addClass("collapse")};return a.support.transition?void this.$element[c](0).one("bsTransitionEnd",a.proxy(d,this)).emulateTransitionEnd(350):d.call(this)}}},c.prototype.toggle=function(){this[this.$element.hasClass("in")?"hide":"show"]()};var d=a.fn.collapse;a.fn.collapse=b,a.f
 n.collapse.Constructor=c,a.fn.collapse.noConflict=function(){return a.fn.collapse=d,this},a(document).on("click.bs.collapse.data-api",'[data-toggle="collapse"]',function(c){var d,e=a(this),f=e.attr("data-target")||c.preventDefault()||(d=e.attr("href"))&&d.replace(/.*(?=#[^\s]+$)/,""),g=a(f),h=g.data("bs.collapse"),i=h?"toggle":e.data(),j=e.attr("data-parent"),k=j&&a(j);h&&h.transitioning||(k&&k.find('[data-toggle="collapse"][data-parent="'+j+'"]').not(e).addClass("collapsed"),e[g.hasClass("in")?"addClass":"removeClass"]("collapsed")),b.call(g,i)})}(jQuery),+function(a){"use strict";function b(b){b&&3===b.which||(a(e).remove(),a(f).each(function(){var d=c(a(this)),e={relatedTarget:this};d.hasClass("open")&&(d.trigger(b=a.Event("hide.bs.dropdown",e)),b.isDefaultPrevented()||d.removeClass("open").trigger("hidden.bs.dropdown",e))}))}function c(b){var c=b.attr("data-target");c||(c=b.attr("href"),c=c&&/#[A-Za-z]/.test(c)&&c.replace(/.*(?=#[^\s]*$)/,""));var d=c&&a(c);return d&&d.length?d:
 b.parent()}function d(b){return this.each(function(){var c=a(this),d=c.data("bs.dropdown");d||c.data("bs.dropdown",d=new g(this)),"string"==typeof b&&d[b].call(c)})}var e=".dropdown-backdrop",f='[data-toggle="dropdown"]',g=function(b){a(b).on("click.bs.dropdown",this.toggle)};g.VERSION="3.2.0",g.prototype.toggle=function(d){var e=a(this);if(!e.is(".disabled, :disabled")){var f=c(e),g=f.hasClass("open");if(b(),!g){"ontouchstart"in document.documentElement&&!f.closest(".navbar-nav").length&&a('<div class="dropdown-backdrop"/>').insertAfter(a(this)).on("click",b);var h={relatedTarget:this};if(f.trigger(d=a.Event("show.bs.dropdown",h)),d.isDefaultPrevented())return;e.trigger("focus"),f.toggleClass("open").trigger("shown.bs.dropdown",h)}return!1}},g.prototype.keydown=function(b){if(/(38|40|27)/.test(b.keyCode)){var d=a(this);if(b.preventDefault(),b.stopPropagation(),!d.is(".disabled, :disabled")){var e=c(d),g=e.hasClass("open");if(!g||g&&27==b.keyCode)return 27==b.which&&e.find(f).trigge
 r("focus"),d.trigger("click");var h=" li:not(.divider):visible a",i=e.find('[role="menu"]'+h+', [role="listbox"]'+h);if(i.length){var j=i.index(i.filter(":focus"));38==b.keyCode&&j>0&&j--,40==b.keyCode&&j<i.length-1&&j++,~j||(j=0),i.eq(j).trigger("focus")}}}};var h=a.fn.dropdown;a.fn.dropdown=d,a.fn.dropdown.Constructor=g,a.fn.dropdown.noConflict=function(){return a.fn.dropdown=h,this},a(document).on("click.bs.dropdown.data-api",b).on("click.bs.dropdown.data-api",".dropdown form",function(a){a.stopPropagation()}).on("click.bs.dropdown.data-api",f,g.prototype.toggle).on("keydown.bs.dropdown.data-api",f+', [role="menu"], [role="listbox"]',g.prototype.keydown)}(jQuery),+function(a){"use strict";function b(b,d){return this.each(function(){var e=a(this),f=e.data("bs.modal"),g=a.extend({},c.DEFAULTS,e.data(),"object"==typeof b&&b);f||e.data("bs.modal",f=new c(this,g)),"string"==typeof b?f[b](d):g.show&&f.show(d)})}var c=function(b,c){this.options=c,this.$body=a(document.body),this.$elemen
 t=a(b),this.$backdrop=this.isShown=null,this.scrollbarWidth=0,this.options.remote&&this.$element.find(".modal-content").load(this.options.remote,a.proxy(function(){this.$element.trigger("loaded.bs.modal")},this))};c.VERSION="3.2.0",c.DEFAULTS={backdrop:!0,keyboard:!0,show:!0},c.prototype.toggle=function(a){return this.isShown?this.hide():this.show(a)},c.prototype.show=function(b){var c=this,d=a.Event("show.bs.modal",{relatedTarget:b});this.$element.trigger(d),this.isShown||d.isDefaultPrevented()||(this.isShown=!0,this.checkScrollbar(),this.$body.addClass("modal-open"),this.setScrollbar(),this.escape(),this.$element.on("click.dismiss.bs.modal",'[data-dismiss="modal"]',a.proxy(this.hide,this)),this.backdrop(function(){var d=a.support.transition&&c.$element.hasClass("fade");c.$element.parent().length||c.$element.appendTo(c.$body),c.$element.show().scrollTop(0),d&&c.$element[0].offsetWidth,c.$element.addClass("in").attr("aria-hidden",!1),c.enforceFocus();var e=a.Event("shown.bs.modal",{
 relatedTarget:b});d?c.$element.find(".modal-dialog").one("bsTransitionEnd",function(){c.$element.trigger("focus").trigger(e)}).emulateTransitionEnd(300):c.$element.trigger("focus").trigger(e)}))},c.prototype.hide=function(b){b&&b.preventDefault(),b=a.Event("hide.bs.modal"),this.$element.trigger(b),this.isShown&&!b.isDefaultPrevented()&&(this.isShown=!1,this.$body.removeClass("modal-open"),this.resetScrollbar(),this.escape(),a(document).off("focusin.bs.modal"),this.$element.removeClass("in").attr("aria-hidden",!0).off("click.dismiss.bs.modal"),a.support.transition&&this.$element.hasClass("fade")?this.$element.one("bsTransitionEnd",a.proxy(this.hideModal,this)).emulateTransitionEnd(300):this.hideModal())},c.prototype.enforceFocus=function(){a(document).off("focusin.bs.modal").on("focusin.bs.modal",a.proxy(function(a){this.$element[0]===a.target||this.$element.has(a.target).length||this.$element.trigger("focus")},this))},c.prototype.escape=function(){this.isShown&&this.options.keyboard
 ?this.$element.on("keyup.dismiss.bs.modal",a.proxy(function(a){27==a.which&&this.hide()},this)):this.isShown||this.$element.off("keyup.dismiss.bs.modal")},c.prototype.hideModal=function(){var a=this;this.$element.hide(),this.backdrop(function(){a.$element.trigger("hidden.bs.modal")})},c.prototype.removeBackdrop=function(){this.$backdrop&&this.$backdrop.remove(),this.$backdrop=null},c.prototype.backdrop=function(b){var c=this,d=this.$element.hasClass("fade")?"fade":"";if(this.isShown&&this.options.backdrop){var e=a.support.transition&&d;if(this.$backdrop=a('<div class="modal-backdrop '+d+'" />').appendTo(this.$body),this.$element.on("click.dismiss.bs.modal",a.proxy(function(a){a.target===a.currentTarget&&("static"==this.options.backdrop?this.$element[0].focus.call(this.$element[0]):this.hide.call(this))},this)),e&&this.$backdrop[0].offsetWidth,this.$backdrop.addClass("in"),!b)return;e?this.$backdrop.one("bsTransitionEnd",b).emulateTransitionEnd(150):b()}else if(!this.isShown&&this.$b
 ackdrop){this.$backdrop.removeClass("in");var f=function(){c.removeBackdrop(),b&&b()};a.support.transition&&this.$element.hasClass("fade")?this.$backdrop.one("bsTransitionEnd",f).emulateTransitionEnd(150):f()}else b&&b()},c.prototype.checkScrollbar=function(){document.body.clientWidth>=window.innerWidth||(this.scrollbarWidth=this.scrollbarWidth||this.measureScrollbar())},c.prototype.setScrollbar=function(){var a=parseInt(this.$body.css("padding-right")||0,10);this.scrollbarWidth&&this.$body.css("padding-right",a+this.scrollbarWidth)},c.prototype.resetScrollbar=function(){this.$body.css("padding-right","")},c.prototype.measureScrollbar=function(){var a=document.createElement("div");a.className="modal-scrollbar-measure",this.$body.append(a);var b=a.offsetWidth-a.clientWidth;return this.$body[0].removeChild(a),b};var d=a.fn.modal;a.fn.modal=b,a.fn.modal.Constructor=c,a.fn.modal.noConflict=function(){return a.fn.modal=d,this},a(document).on("click.bs.modal.data-api",'[data-toggle="modal
 "]',function(c){var d=a(this),e=d.attr("href"),f=a(d.attr("data-target")||e&&e.replace(/.*(?=#[^\s]+$)/,"")),g=f.data("bs.modal")?"toggle":a.extend({remote:!/#/.test(e)&&e},f.data(),d.data());d.is("a")&&c.preventDefault(),f.one("show.bs.modal",function(a){a.isDefaultPrevented()||f.one("hidden.bs.modal",function(){d.is(":visible")&&d.trigger("focus")})}),b.call(f,g,this)})}(jQuery),+function(a){"use strict";function b(b){return this.each(function(){var d=a(this),e=d.data("bs.tooltip"),f="object"==typeof b&&b;(e||"destroy"!=b)&&(e||d.data("bs.tooltip",e=new c(this,f)),"string"==typeof b&&e[b]())})}var c=function(a,b){this.type=this.options=this.enabled=this.timeout=this.hoverState=this.$element=null,this.init("tooltip",a,b)};c.VERSION="3.2.0",c.DEFAULTS={animation:!0,placement:"top",selector:!1,template:'<div class="tooltip" role="tooltip"><div class="tooltip-arrow"></div><div class="tooltip-inner"></div></div>',trigger:"hover focus",title:"",delay:0,html:!1,container:!1,viewport:{sel
 ector:"body",padding:0}},c.prototype.init=function(b,c,d){this.enabled=!0,this.type=b,this.$element=a(c),this.options=this.getOptions(d),this.$viewport=this.options.viewport&&a(this.options.viewport.selector||this.options.viewport);for(var e=this.options.trigger.split(" "),f=e.length;f--;){var g=e[f];if("click"==g)this.$element.on("click."+this.type,this.options.selector,a.proxy(this.toggle,this));else if("manual"!=g){var h="hover"==g?"mouseenter":"focusin",i="hover"==g?"mouseleave":"focusout";this.$element.on(h+"."+this.type,this.options.selector,a.proxy(this.enter,this)),this.$element.on(i+"."+this.type,this.options.selector,a.proxy(this.leave,this))}}this.options.selector?this._options=a.extend({},this.options,{trigger:"manual",selector:""}):this.fixTitle()},c.prototype.getDefaults=function(){return c.DEFAULTS},c.prototype.getOptions=function(b){return b=a.extend({},this.getDefaults(),this.$element.data(),b),b.delay&&"number"==typeof b.delay&&(b.delay={show:b.delay,hide:b.delay})
 ,b},c.prototype.getDelegateOptions=function(){var b={},c=this.getDefaults();return this._options&&a.each(this._options,function(a,d){c[a]!=d&&(b[a]=d)}),b},c.prototype.enter=function(b){var c=b instanceof this.constructor?b:a(b.currentTarget).data("bs."+this.type);return c||(c=new this.constructor(b.currentTarget,this.getDelegateOptions()),a(b.currentTarget).data("bs."+this.type,c)),clearTimeout(c.timeout),c.hoverState="in",c.options.delay&&c.options.delay.show?void(c.timeout=setTimeout(function(){"in"==c.hoverState&&c.show()},c.options.delay.show)):c.show()},c.prototype.leave=function(b){var c=b instanceof this.constructor?b:a(b.currentTarget).data("bs."+this.type);return c||(c=new this.constructor(b.currentTarget,this.getDelegateOptions()),a(b.currentTarget).data("bs."+this.type,c)),clearTimeout(c.timeout),c.hoverState="out",c.options.delay&&c.options.delay.hide?void(c.timeout=setTimeout(function(){"out"==c.hoverState&&c.hide()},c.options.delay.hide)):c.hide()},c.prototype.show=fu
 nction(){var b=a.Event("show.bs."+this.type);if(this.hasContent()&&this.enabled){this.$element.trigger(b);var c=a.contains(document.documentElement,this.$element[0]);if(b.isDefaultPrevented()||!c)return;var d=this,e=this.tip(),f=this.getUID(this.type);this.setContent(),e.attr("id",f),this.$element.attr("aria-describedby",f),this.options.animation&&e.addClass("fade");var g="function"==typeof this.options.placement?this.options.placement.call(this,e[0],this.$element[0]):this.options.placement,h=/\s?auto?\s?/i,i=h.test(g);i&&(g=g.replace(h,"")||"top"),e.detach().css({top:0,left:0,display:"block"}).addClass(g).data("bs."+this.type,this),this.options.container?e.appendTo(this.options.container):e.insertAfter(this.$element);var j=this.getPosition(),k=e[0].offsetWidth,l=e[0].offsetHeight;if(i){var m=g,n=this.$element.parent(),o=this.getPosition(n);g="bottom"==g&&j.top+j.height+l-o.scroll>o.height?"top":"top"==g&&j.top-o.scroll-l<0?"bottom":"right"==g&&j.right+k>o.width?"left":"left"==g&&j.
 left-k<o.left?"right":g,e.removeClass(m).addClass(g)}var p=this.getCalculatedOffset(g,j,k,l);this.applyPlacement(p,g);var q=function(){d.$element.trigger("shown.bs."+d.type),d.hoverState=null};a.support.transition&&this.$tip.hasClass("fade")?e.one("bsTransitionEnd",q).emulateTransitionEnd(150):q()}},c.prototype.applyPlacement=function(b,c){var d=this.tip(),e=d[0].offsetWidth,f=d[0].offsetHeight,g=parseInt(d.css("margin-top"),10),h=parseInt(d.css("margin-left"),10);isNaN(g)&&(g=0),isNaN(h)&&(h=0),b.top=b.top+g,b.left=b.left+h,a.offset.setOffset(d[0],a.extend({using:function(a){d.css({top:Math.round(a.top),left:Math.round(a.left)})}},b),0),d.addClass("in");var i=d[0].offsetWidth,j=d[0].offsetHeight;"top"==c&&j!=f&&(b.top=b.top+f-j);var k=this.getViewportAdjustedDelta(c,b,i,j);k.left?b.left+=k.left:b.top+=k.top;var l=k.left?2*k.left-e+i:2*k.top-f+j,m=k.left?"left":"top",n=k.left?"offsetWidth":"offsetHeight";d.offset(b),this.replaceArrow(l,d[0][n],m)},c.prototype.replaceArrow=function(a
 ,b,c){this.arrow().css(c,a?50*(1-a/b)+"%":"")},c.prototype.setContent=function(){var a=this.tip(),b=this.getTitle();a.find(".tooltip-inner")[this.options.html?"html":"text"](b),a.removeClass("fade in top bottom left right")},c.prototype.hide=function(){function b(){"in"!=c.hoverState&&d.detach(),c.$element.trigger("hidden.bs."+c.type)}var c=this,d=this.tip(),e=a.Event("hide.bs."+this.type);return this.$element.removeAttr("aria-describedby"),this.$element.trigger(e),e.isDefaultPrevented()?void 0:(d.removeClass("in"),a.support.transition&&this.$tip.hasClass("fade")?d.one("bsTransitionEnd",b).emulateTransitionEnd(150):b(),this.hoverState=null,this)},c.prototype.fixTitle=function(){var a=this.$element;(a.attr("title")||"string"!=typeof a.attr("data-original-title"))&&a.attr("data-original-title",a.attr("title")||"").attr("title","")},c.prototype.hasContent=function(){return this.getTitle()},c.prototype.getPosition=function(b){b=b||this.$element;var c=b[0],d="BODY"==c.tagName;return a.ex
 tend({},"function"==typeof c.getBoundingClientRect?c.getBoundingClientRect():null,{scroll:d?document.documentElement.scrollTop||document.body.scrollTop:b.scrollTop(),width:d?a(window).width():b.outerWidth(),height:d?a(window).height():b.outerHeight()},d?{top:0,left:0}:b.offset())},c.prototype.getCalculatedOffset=function(a,b,c,d){return"bottom"==a?{top:b.top+b.height,left:b.left+b.width/2-c/2}:"top"==a?{top:b.top-d,left:b.left+b.width/2-c/2}:"left"==a?{top:b.top+b.height/2-d/2,left:b.left-c}:{top:b.top+b.height/2-d/2,left:b.left+b.width}},c.prototype.getViewportAdjustedDelta=function(a,b,c,d){var e={top:0,left:0};if(!this.$viewport)return e;var f=this.options.viewport&&this.options.viewport.padding||0,g=this.getPosition(this.$viewport);if(/right|left/.test(a)){var h=b.top-f-g.scroll,i=b.top+f-g.scroll+d;h<g.top?e.top=g.top-h:i>g.top+g.height&&(e.top=g.top+g.height-i)}else{var j=b.left-f,k=b.left+f+c;j<g.left?e.left=g.left-j:k>g.width&&(e.left=g.left+g.width-k)}return e},c.prototype.
 getTitle=function(){var a,b=this.$element,c=this.options;return a=b.attr("data-original-title")||("function"==typeof c.title?c.title.call(b[0]):c.title)},c.prototype.getUID=function(a){do a+=~~(1e6*Math.random());while(document.getElementById(a));return a},c.prototype.tip=function(){return this.$tip=this.$tip||a(this.options.template)},c.prototype.arrow=function(){return this.$arrow=this.$arrow||this.tip().find(".tooltip-arrow")},c.prototype.validate=function(){this.$element[0].parentNode||(this.hide(),this.$element=null,this.options=null)},c.prototype.enable=function(){this.enabled=!0},c.prototype.disable=function(){this.enabled=!1},c.prototype.toggleEnabled=function(){this.enabled=!this.enabled},c.prototype.toggle=function(b){var c=this;b&&(c=a(b.currentTarget).data("bs."+this.type),c||(c=new this.constructor(b.currentTarget,this.getDelegateOptions()),a(b.currentTarget).data("bs."+this.type,c))),c.tip().hasClass("in")?c.leave(c):c.enter(c)},c.prototype.destroy=function(){clearTime
 out(this.timeout),this.hide().$element.off("."+this.type).removeData("bs."+this.type)};var d=a.fn.tooltip;a.fn.tooltip=b,a.fn.tooltip.Constructor=c,a.fn.tooltip.noConflict=function(){return a.fn.tooltip=d,this}}(jQuery),+function(a){"use strict";function b(b){return this.each(function(){var d=a(this),e=d.data("bs.popover"),f="object"==typeof b&&b;(e||"destroy"!=b)&&(e||d.data("bs.popover",e=new c(this,f)),"string"==typeof b&&e[b]())})}var c=function(a,b){this.init("popover",a,b)};if(!a.fn.tooltip)throw new Error("Popover requires tooltip.js");c.VERSION="3.2.0",c.DEFAULTS=a.extend({},a.fn.tooltip.Constructor.DEFAULTS,{placement:"right",trigger:"click",content:"",template:'<div class="popover" role="tooltip"><div class="arrow"></div><h3 class="popover-title"></h3><div class="popover-content"></div></div>'}),c.prototype=a.extend({},a.fn.tooltip.Constructor.prototype),c.prototype.constructor=c,c.prototype.getDefaults=function(){return c.DEFAULTS},c.prototype.setContent=function(){var a=
 this.tip(),b=this.getTitle(),c=this.getContent();a.find(".popover-title")[this.options.html?"html":"text"](b),a.find(".popover-content").empty()[this.options.html?"string"==typeof c?"html":"append":"text"](c),a.removeClass("fade top bottom left right in"),a.find(".popover-title").html()||a.find(".popover-title").hide()},c.prototype.hasContent=function(){return this.getTitle()||this.getContent()},c.prototype.getContent=function(){var a=this.$element,b=this.options;return a.attr("data-content")||("function"==typeof b.content?b.content.call(a[0]):b.content)},c.prototype.arrow=function(){return this.$arrow=this.$arrow||this.tip().find(".arrow")},c.prototype.tip=function(){return this.$tip||(this.$tip=a(this.options.template)),this.$tip};var d=a.fn.popover;a.fn.popover=b,a.fn.popover.Constructor=c,a.fn.popover.noConflict=function(){return a.fn.popover=d,this}}(jQuery),+function(a){"use strict";function b(c,d){var e=a.proxy(this.process,this);this.$body=a("body"),this.$scrollElement=a(a(c
 ).is("body")?window:c),this.options=a.extend({},b.DEFAULTS,d),this.selector=(this.options.target||"")+" .nav li > a",this.offsets=[],this.targets=[],this.activeTarget=null,this.scrollHeight=0,this.$scrollElement.on("scroll.bs.scrollspy",e),this.refresh(),this.process()}function c(c){return this.each(function(){var d=a(this),e=d.data("bs.scrollspy"),f="object"==typeof c&&c;e||d.data("bs.scrollspy",e=new b(this,f)),"string"==typeof c&&e[c]()})}b.VERSION="3.2.0",b.DEFAULTS={offset:10},b.prototype.getScrollHeight=function(){return this.$scrollElement[0].scrollHeight||Math.max(this.$body[0].scrollHeight,document.documentElement.scrollHeight)},b.prototype.refresh=function(){var b="offset",c=0;a.isWindow(this.$scrollElement[0])||(b="position",c=this.$scrollElement.scrollTop()),this.offsets=[],this.targets=[],this.scrollHeight=this.getScrollHeight();var d=this;this.$body.find(this.selector).map(function(){var d=a(this),e=d.data("target")||d.attr("href"),f=/^#./.test(e)&&a(e);return f&&f.len
 gth&&f.is(":visible")&&[[f[b]().top+c,e]]||null}).sort(function(a,b){return a[0]-b[0]}).each(function(){d.offsets.push(this[0]),d.targets.push(this[1])})},b.prototype.process=function(){var a,b=this.$scrollElement.scrollTop()+this.options.offset,c=this.getScrollHeight(),d=this.options.offset+c-this.$scrollElement.height(),e=this.offsets,f=this.targets,g=this.activeTarget;if(this.scrollHeight!=c&&this.refresh(),b>=d)return g!=(a=f[f.length-1])&&this.activate(a);if(g&&b<=e[0])return g!=(a=f[0])&&this.activate(a);for(a=e.length;a--;)g!=f[a]&&b>=e[a]&&(!e[a+1]||b<=e[a+1])&&this.activate(f[a])},b.prototype.activate=function(b){this.activeTarget=b,a(this.selector).parentsUntil(this.options.target,".active").removeClass("active");var c=this.selector+'[data-target="'+b+'"],'+this.selector+'[href="'+b+'"]',d=a(c).parents("li").addClass("active");d.parent(".dropdown-menu").length&&(d=d.closest("li.dropdown").addClass("active")),d.trigger("activate.bs.scrollspy")};var d=a.fn.scrollspy;a.fn.scr
 ollspy=c,a.fn.scrollspy.Constructor=b,a.fn.scrollspy.noConflict=function(){return a.fn.scrollspy=d,this},a(window).on("load.bs.scrollspy.data-api",function(){a('[data-spy="scroll"]').each(function(){var b=a(this);c.call(b,b.data())})})}(jQuery),+function(a){"use strict";function b(b){return this.each(function(){var d=a(this),e=d.data("bs.tab");e||d.data("bs.tab",e=new c(this)),"string"==typeof b&&e[b]()})}var c=function(b){this.element=a(b)};c.VERSION="3.2.0",c.prototype.show=function(){var b=this.element,c=b.closest("ul:not(.dropdown-menu)"),d=b.data("target");if(d||(d=b.attr("href"),d=d&&d.replace(/.*(?=#[^\s]*$)/,"")),!b.parent("li").hasClass("active")){var e=c.find(".active:last a")[0],f=a.Event("show.bs.tab",{relatedTarget:e});if(b.trigger(f),!f.isDefaultPrevented()){var g=a(d);this.activate(b.closest("li"),c),this.activate(g,g.parent(),function(){b.trigger({type:"shown.bs.tab",relatedTarget:e})})}}},c.prototype.activate=function(b,c,d){function e(){f.removeClass("active").find
 ("> .dropdown-menu > .active").removeClass("active"),b.addClass("active"),g?(b[0].offsetWidth,b.addClass("in")):b.removeClass("fade"),b.parent(".dropdown-menu")&&b.closest("li.dropdown").addClass("active"),d&&d()}var f=c.find("> .active"),g=d&&a.support.transition&&f.hasClass("fade");g?f.one("bsTransitionEnd",e).emulateTransitionEnd(150):e(),f.removeClass("in")};var d=a.fn.tab;a.fn.tab=b,a.fn.tab.Constructor=c,a.fn.tab.noConflict=function(){return a.fn.tab=d,this},a(document).on("click.bs.tab.data-api",'[data-toggle="tab"], [data-toggle="pill"]',function(c){c.preventDefault(),b.call(a(this),"show")})}(jQuery),+function(a){"use strict";function b(b){return this.each(function(){var d=a(this),e=d.data("bs.affix"),f="object"==typeof b&&b;e||d.data("bs.affix",e=new c(this,f)),"string"==typeof b&&e[b]()})}var c=function(b,d){this.options=a.extend({},c.DEFAULTS,d),this.$target=a(this.options.target).on("scroll.bs.affix.data-api",a.proxy(this.checkPosition,this)).on("click.bs.affix.data-api
 ",a.proxy(this.checkPositionWithEventLoop,this)),this.$element=a(b),this.affixed=this.unpin=this.pinnedOffset=null,this.checkPosition()};c.VERSION="3.2.0",c.RESET="affix affix-top affix-bottom",c.DEFAULTS={offset:0,target:window},c.prototype.getPinnedOffset=function(){if(this.pinnedOffset)return this.pinnedOffset;this.$element.removeClass(c.RESET).addClass("affix");var a=this.$target.scrollTop(),b=this.$element.offset();return this.pinnedOffset=b.top-a},c.prototype.checkPositionWithEventLoop=function(){setTimeout(a.proxy(this.checkPosition,this),1)},c.prototype.checkPosition=function(){if(this.$element.is(":visible")){var b=a(document).height(),d=this.$target.scrollTop(),e=this.$element.offset(),f=this.options.offset,g=f.top,h=f.bottom;"object"!=typeof f&&(h=g=f),"function"==typeof g&&(g=f.top(this.$element)),"function"==typeof h&&(h=f.bottom(this.$element));var i=null!=this.unpin&&d+this.unpin<=e.top?!1:null!=h&&e.top+this.$element.height()>=b-h?"bottom":null!=g&&g>=d?"top":!1;if(t
 his.affixed!==i){null!=this.unpin&&this.$element.css("top","");var j="affix"+(i?"-"+i:""),k=a.Event(j+".bs.affix");this.$element.trigger(k),k.isDefaultPrevented()||(this.affixed=i,this.unpin="bottom"==i?this.getPinnedOffset():null,this.$element.removeClass(c.RESET).addClass(j).trigger(a.Event(j.replace("affix","affixed"))),"bottom"==i&&this.$element.offset({top:b-this.$element.height()-h}))}}};var d=a.fn.affix;a.fn.affix=b,a.fn.affix.Constructor=c,a.fn.affix.noConflict=function(){return a.fn.affix=d,this},a(window).on("load",function(){a('[data-spy="affix"]').each(function(){var c=a(this),d=c.data();d.offset=d.offset||{},d.offsetBottom&&(d.offset.bottom=d.offsetBottom),d.offsetTop&&(d.offset.top=d.offsetTop),b.call(c,d)})})}(jQuery);
\ No newline at end of file


[44/52] [abbrv] kylin git commit: minor, patch a NPE in CubeService.isCubeInProject()

Posted by li...@apache.org.
minor, patch a NPE in CubeService.isCubeInProject()


Project: http://git-wip-us.apache.org/repos/asf/kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/24b7aa7f
Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/24b7aa7f
Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/24b7aa7f

Branch: refs/heads/1.x-HBase1.1.3
Commit: 24b7aa7f700d66e4860c6eeca78ef173e4e2ef8b
Parents: 0a74e9c
Author: Li, Yang <li...@apache.org>
Authored: Mon Feb 15 18:11:50 2016 +0800
Committer: Li, Yang <li...@apache.org>
Committed: Mon Feb 15 18:11:50 2016 +0800

----------------------------------------------------------------------
 .../org/apache/kylin/rest/service/CubeService.java    | 14 +++++++-------
 1 file changed, 7 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/24b7aa7f/server/src/main/java/org/apache/kylin/rest/service/CubeService.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/kylin/rest/service/CubeService.java b/server/src/main/java/org/apache/kylin/rest/service/CubeService.java
index 52bbdc4..601814c 100644
--- a/server/src/main/java/org/apache/kylin/rest/service/CubeService.java
+++ b/server/src/main/java/org/apache/kylin/rest/service/CubeService.java
@@ -19,7 +19,6 @@
 package org.apache.kylin.rest.service;
 
 import java.io.IOException;
-import java.net.UnknownHostException;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.Date;
@@ -64,10 +63,8 @@ import org.apache.kylin.rest.exception.InternalErrorException;
 import org.apache.kylin.rest.request.MetricsRequest;
 import org.apache.kylin.rest.response.HBaseResponse;
 import org.apache.kylin.rest.response.MetricsResponse;
-import org.apache.kylin.rest.security.AclPermission;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.springframework.beans.factory.annotation.Autowired;
 import org.springframework.cache.annotation.CacheEvict;
 import org.springframework.cache.annotation.Caching;
 import org.springframework.security.access.prepost.PostFilter;
@@ -157,7 +154,7 @@ public class CubeService extends BasicService {
         desc.init(getConfig(), getMetadataManager().getAllTablesMap());
         int cuboidCount = CuboidCLI.simulateCuboidGeneration(desc);
         logger.info("New cube " + cubeName + " has " + cuboidCount + " cuboids");
-        
+
         boolean isNew = false;
         if (getCubeDescManager().getCubeDesc(desc.getName()) == null) {
             createdDesc = getCubeDescManager().createCubeDesc(desc);
@@ -205,7 +202,10 @@ public class CubeService extends BasicService {
         for (RealizationEntry projectRealization : project.getRealizationEntries()) {
             if (projectRealization != null && projectRealization.getType() == RealizationType.CUBE) {
                 CubeInstance cube = getCubeManager().getCube(projectRealization.getRealization());
-                assert cube != null;
+                if (cube == null) {
+                    logger.error("Project " + projectName + " contains realization " + projectRealization.getRealization() + " which is not found by CubeManager");
+                    continue;
+                }
                 if (cube.equals(target)) {
                     return true;
                 }
@@ -229,7 +229,7 @@ public class CubeService extends BasicService {
             if (!updatedCubeDesc.getError().isEmpty()) {
                 return updatedCubeDesc;
             }
-            
+
             getCubeManager().updateCube(cube);
 
             return updatedCubeDesc;
@@ -494,7 +494,7 @@ public class CubeService extends BasicService {
         cube.getSegments().clear();
         CubeManager.getInstance(getConfig()).updateCube(cube);
     }
-    
+
     private void discardAllRunningJobs(CubeInstance cube) {
         final List<CubingJob> cubingJobs = listAllCubingJobs(cube.getName(), null);
         for (CubingJob cubingJob : cubingJobs) {


[48/52] [abbrv] kylin git commit: KYLIN-1132 enable user select on table tree

Posted by li...@apache.org.
KYLIN-1132 enable user select on table tree


Project: http://git-wip-us.apache.org/repos/asf/kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/9d96cfaa
Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/9d96cfaa
Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/9d96cfaa

Branch: refs/heads/1.x-HBase1.1.3
Commit: 9d96cfaa3faa3a27fced21576df482aa358a7f14
Parents: 1caa626
Author: janzhongi <ji...@ebay.com>
Authored: Thu Feb 18 13:54:33 2016 +0800
Committer: janzhongi <ji...@ebay.com>
Committed: Thu Feb 18 13:54:33 2016 +0800

----------------------------------------------------------------------
 webapp/app/less/app.less | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/9d96cfaa/webapp/app/less/app.less
----------------------------------------------------------------------
diff --git a/webapp/app/less/app.less b/webapp/app/less/app.less
index bf2ba77..4c17b9b 100644
--- a/webapp/app/less/app.less
+++ b/webapp/app/less/app.less
@@ -552,7 +552,7 @@ ul.messenger.messenger-fixed.messenger-on-top .messenger-message .messenger-mess
 }
 
 /** Enable the text selectable on notification of Messenger **/
-ul.messenger .messenger-message-inner ,.ngCellText{
+ul.messenger .messenger-message-inner ,.ngCellText,.tree-label,.tree-leaf{
   -moz-user-select: text;
   -webkit-user-select: text;
   -o-user-select: text;


[50/52] [abbrv] kylin git commit: minor, enable close-on-select for ui-select, dropDown list will not close when user pick up one option

Posted by li...@apache.org.
minor, enable close-on-select for ui-select, dropDown list will not close when user pick up one option


Project: http://git-wip-us.apache.org/repos/asf/kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/0e5be051
Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/0e5be051
Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/0e5be051

Branch: refs/heads/1.x-HBase1.1.3
Commit: 0e5be051fed9d69ee2293130e07040558f258049
Parents: 0c120c3
Author: janzhongi <ji...@ebay.com>
Authored: Fri Feb 19 16:45:44 2016 +0800
Committer: janzhongi <ji...@ebay.com>
Committed: Fri Feb 19 16:45:44 2016 +0800

----------------------------------------------------------------------
 webapp/app/js/controllers/cubeSchema.js                 | 10 +++++++---
 webapp/app/partials/cubeDesigner/advanced_settings.html |  1 +
 2 files changed, 8 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/0e5be051/webapp/app/js/controllers/cubeSchema.js
----------------------------------------------------------------------
diff --git a/webapp/app/js/controllers/cubeSchema.js b/webapp/app/js/controllers/cubeSchema.js
index bbcc1fb..ba64a1d 100755
--- a/webapp/app/js/controllers/cubeSchema.js
+++ b/webapp/app/js/controllers/cubeSchema.js
@@ -185,10 +185,14 @@ KylinApp.controller('CubeSchemaCtrl', function ($scope, QueryService, UserServic
     $scope.cubeMetaFrame.rowkey.aggregation_groups.push([]);
   };
 
-  $scope.refreshAggregationGroup = function (list, index, aggregation_groups) {
-    if (aggregation_groups) {
-      list[index] = aggregation_groups;
+  $scope.refreshAggregationGroup = function (list, index, aggregation_group) {
+    if (aggregation_group) {
+      list[index].length = aggregation_group.length;
+      for(var i=0;i<aggregation_group.length;i++){
+        list[index][i] = aggregation_group[i];
+      }
     }
+
   };
 
   $scope.removeElement = function (arr, element) {

http://git-wip-us.apache.org/repos/asf/kylin/blob/0e5be051/webapp/app/partials/cubeDesigner/advanced_settings.html
----------------------------------------------------------------------
diff --git a/webapp/app/partials/cubeDesigner/advanced_settings.html b/webapp/app/partials/cubeDesigner/advanced_settings.html
index 91d804c..e55cdbf 100755
--- a/webapp/app/partials/cubeDesigner/advanced_settings.html
+++ b/webapp/app/partials/cubeDesigner/advanced_settings.html
@@ -139,6 +139,7 @@
                 <td class="col-xs-11">
                     <!-- Dimensions -->
                   <ui-select
+                    close-on-select="false"
                     autofocus="true"
                     on-select="refreshAggregationGroup(cubeMetaFrame.rowkey.aggregation_groups, $index, aggregation_groups)"
                     on-remove="refreshAggregationGroup(cubeMetaFrame.rowkey.aggregation_groups, $index, aggregation_groups)"


[19/52] [abbrv] kylin git commit: minor, avoid verbose logging in serialier

Posted by li...@apache.org.
minor, avoid verbose logging in serialier


Project: http://git-wip-us.apache.org/repos/asf/kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/62a8cb26
Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/62a8cb26
Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/62a8cb26

Branch: refs/heads/1.x-HBase1.1.3
Commit: 62a8cb268e906738502338c7978b39bb3fd2c28d
Parents: 05df713
Author: honma <ho...@ebay.com>
Authored: Wed Feb 3 14:40:55 2016 +0800
Committer: honma <ho...@ebay.com>
Committed: Wed Feb 3 14:40:55 2016 +0800

----------------------------------------------------------------------
 .../org/apache/kylin/metadata/datatype/BigDecimalSerializer.java | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/62a8cb26/metadata/src/main/java/org/apache/kylin/metadata/datatype/BigDecimalSerializer.java
----------------------------------------------------------------------
diff --git a/metadata/src/main/java/org/apache/kylin/metadata/datatype/BigDecimalSerializer.java b/metadata/src/main/java/org/apache/kylin/metadata/datatype/BigDecimalSerializer.java
index 468e4b0..25805da 100644
--- a/metadata/src/main/java/org/apache/kylin/metadata/datatype/BigDecimalSerializer.java
+++ b/metadata/src/main/java/org/apache/kylin/metadata/datatype/BigDecimalSerializer.java
@@ -48,8 +48,8 @@ public class BigDecimalSerializer extends DataTypeSerializer<BigDecimal> {
     @Override
     public void serialize(BigDecimal value, ByteBuffer out) {
         if (value.scale() > type.getScale()) {
-            if (avoidVerbose % 10000 == 0) {
-                logger.warn("value's scale has exceeded the " + type.getScale() + ", cut it off, to ensure encoded value do not exceed maxLength " + maxLength + " times:" + (avoidVerbose++));
+            if (avoidVerbose++ % 10000 == 0) {
+                logger.warn("value's scale has exceeded the " + type.getScale() + ", cut it off, to ensure encoded value do not exceed maxLength " + maxLength + " times:" + (avoidVerbose));
             }
             value = value.setScale(type.getScale(), BigDecimal.ROUND_HALF_EVEN);
         }