You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@eagle.apache.org by qi...@apache.org on 2016/10/19 03:19:13 UTC

[14/28] incubator-eagle git commit: EAGLE-339: Automatically create hbase table when initializing

EAGLE-339: Automatically create hbase table when initializing

https://github.com/apache/incubator-eagle/pull/240

Author: Zhao, Qingwen <qi...@ebay.com>
Reviewer: Edward, Zhang
Closes #240


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

Branch: refs/heads/master
Commit: b49bda14cec0c92ff7224f18476fc4456faa93bd
Parents: 3553528
Author: Zhao, Qingwen <qi...@ebay.com>
Authored: Fri Jun 17 14:08:25 2016 +0800
Committer: Zhao, Qingwen <qi...@ebay.com>
Committed: Fri Jun 17 14:08:25 2016 +0800

----------------------------------------------------------------------
 .../src/main/bin/eagle-create-table.rb          |  47 ---------
 .../src/main/bin/eagle-drop-tables.sh           |  26 -----
 eagle-assembly/src/main/bin/eagle-env.sh        |   4 -
 .../src/main/bin/eagle-service-init.sh          |  35 -------
 .../apache/eagle/common/config/EagleConfig.java |   3 +
 .../eagle/common/config/EagleConfigFactory.java |   5 +
 .../storage/hbase/HBaseEntitySchemaManager.java | 102 +++++++++++++++++++
 .../eagle/storage/hbase/HBaseStorage.java       |   1 +
 8 files changed, 111 insertions(+), 112 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b49bda14/eagle-assembly/src/main/bin/eagle-create-table.rb
----------------------------------------------------------------------
diff --git a/eagle-assembly/src/main/bin/eagle-create-table.rb b/eagle-assembly/src/main/bin/eagle-create-table.rb
deleted file mode 100755
index b1b790c..0000000
--- a/eagle-assembly/src/main/bin/eagle-create-table.rb
+++ /dev/null
@@ -1,47 +0,0 @@
-# Create HBase tables for Eagle
-
-# 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.
-
-include Java
-import org.apache.hadoop.hbase.HBaseConfiguration
-import org.apache.hadoop.hbase.client.HBaseAdmin
-
-def createEagleTable(admin, tableName)
-  if !admin.tableExists(tableName)
-    # create tableName, {NAME => 'f', VERSIONS => '1', BLOOMFILTER => 'ROW', COMPRESSION => 'GZ'}
-    create tableName, {NAME => 'f', VERSIONS => '1', BLOOMFILTER => 'ROW', COMPRESSION => 'SNAPPY'}
-    puts "Create Table #{tableName} successfully"
-  elsif admin.isTableDisabled(tableName)
-    admin.enableTable(tableName)
-    puts "Table #{tableName} already exists"
-  else
-    puts "Table #{tableName} already exists"
-  end
-end
-
-conf = HBaseConfiguration.new
-admin = HBaseAdmin.new(conf)
-
-if ARGV.empty?
-  puts "Table list is empty, please go back to bin/eagle-env.sh and export EAGLE_TABLE_LIST"
-  exit 1
-end
-
-tableListVal=ARGV.first
-
-tableListVal.split(' ').map { |i| createEagleTable(admin, i) }
-
-exit 0

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b49bda14/eagle-assembly/src/main/bin/eagle-drop-tables.sh
----------------------------------------------------------------------
diff --git a/eagle-assembly/src/main/bin/eagle-drop-tables.sh b/eagle-assembly/src/main/bin/eagle-drop-tables.sh
deleted file mode 100755
index 37fb540..0000000
--- a/eagle-assembly/src/main/bin/eagle-drop-tables.sh
+++ /dev/null
@@ -1,26 +0,0 @@
-#!/bin/bash
-
-# 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.
-
-source $(dirname $0)/eagle-env.sh
-
-commands=''
-for i in $EAGLE_TABLE_LIST; do
-    commands="disable '$i'\ndrop '$i'\n"$commands
-done
-
-echo -e $commands | hbase shell
-#echo commands | hbase shell -n > /dev/null 2>&1
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b49bda14/eagle-assembly/src/main/bin/eagle-env.sh
----------------------------------------------------------------------
diff --git a/eagle-assembly/src/main/bin/eagle-env.sh b/eagle-assembly/src/main/bin/eagle-env.sh
index 66d72ae..2e01dcd 100755
--- a/eagle-assembly/src/main/bin/eagle-env.sh
+++ b/eagle-assembly/src/main/bin/eagle-env.sh
@@ -49,7 +49,3 @@ export EAGLE_STORM_CLASSPATH=$EAGLE_CLASSPATH
 for file in $EAGLE_HOME/lib/storm/*;do
 	EAGLE_STORM_CLASSPATH=$EAGLE_STORM_CLASSPATH:$file
 done
-
-# EAGLE_TABLE_LIST
-# TODO: Automatically create hbase table when initializing
-export EAGLE_TABLE_LIST='alertdef ipzone streamMetadata alertdetail fileSensitivity eaglehdfs_alert streamdef eagle_metric alertExecutor alertStream alertStreamSchema hiveResourceSensitivity hbaseResourceSensitivity mlmodel userprofile hfdsusercommandpattern appCommand appDefinition serviceAudit aggregatedef alertNotifications eagleSiteDesc eagleSiteApplication eagleApplicationDesc eagleFeatureDesc eagle_metadata'
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b49bda14/eagle-assembly/src/main/bin/eagle-service-init.sh
----------------------------------------------------------------------
diff --git a/eagle-assembly/src/main/bin/eagle-service-init.sh b/eagle-assembly/src/main/bin/eagle-service-init.sh
deleted file mode 100755
index a9c1cb2..0000000
--- a/eagle-assembly/src/main/bin/eagle-service-init.sh
+++ /dev/null
@@ -1,35 +0,0 @@
-#!/bin/bash
-
-# 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.
-
-source $(dirname $0)/eagle-env.sh
-
-###################################################
-###        create hbase tables for eagle
-###################################################
-
-echo "Creating hbase tables for eagle ... "
-
-hbase shell ${EAGLE_HOME}/bin/eagle-create-table.rb "$EAGLE_TABLE_LIST"
-
-if [ $? = 0 ];then
-	echo "==> Successfully created hbase tables"
-else
-	echo "==> Failed creating hbase tables"
-	exit 1
-fi
-
-exit 0
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b49bda14/eagle-core/eagle-query/eagle-common/src/main/java/org/apache/eagle/common/config/EagleConfig.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-query/eagle-common/src/main/java/org/apache/eagle/common/config/EagleConfig.java b/eagle-core/eagle-query/eagle-common/src/main/java/org/apache/eagle/common/config/EagleConfig.java
index afd095b..6226057 100755
--- a/eagle-core/eagle-query/eagle-common/src/main/java/org/apache/eagle/common/config/EagleConfig.java
+++ b/eagle-core/eagle-query/eagle-common/src/main/java/org/apache/eagle/common/config/EagleConfig.java
@@ -17,6 +17,7 @@
 package org.apache.eagle.common.config;
 
 import com.typesafe.config.Config;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.client.HTableInterface;
 
 import java.util.TimeZone;
@@ -28,6 +29,8 @@ public interface EagleConfig {
 
 	HTableInterface getHTable(String tableName);
 
+    Configuration getHbaseConf();
+
     String getStorageType();
 
     ThreadPoolExecutor getExecutor();

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b49bda14/eagle-core/eagle-query/eagle-common/src/main/java/org/apache/eagle/common/config/EagleConfigFactory.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-query/eagle-common/src/main/java/org/apache/eagle/common/config/EagleConfigFactory.java b/eagle-core/eagle-query/eagle-common/src/main/java/org/apache/eagle/common/config/EagleConfigFactory.java
index 3e26f96..0d73743 100755
--- a/eagle-core/eagle-query/eagle-common/src/main/java/org/apache/eagle/common/config/EagleConfigFactory.java
+++ b/eagle-core/eagle-query/eagle-common/src/main/java/org/apache/eagle/common/config/EagleConfigFactory.java
@@ -139,6 +139,11 @@ public class EagleConfigFactory implements EagleConfig {
 	}
 
     @Override
+	public Configuration getHbaseConf() {
+		return hbaseConf;
+	}
+
+    @Override
 	public String getZKQuorum(){
 		return this.zkQuorum;
     }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b49bda14/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/HBaseEntitySchemaManager.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/HBaseEntitySchemaManager.java b/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/HBaseEntitySchemaManager.java
new file mode 100644
index 0000000..03c0ec2
--- /dev/null
+++ b/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/HBaseEntitySchemaManager.java
@@ -0,0 +1,102 @@
+/*
+ * 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.eagle.storage.hbase;
+
+import org.apache.eagle.common.config.EagleConfigFactory;
+import org.apache.eagle.log.entity.meta.EntityDefinition;
+import org.apache.eagle.log.entity.meta.EntityDefinitionManager;
+import org.apache.hadoop.conf.Configuration;
+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.io.compress.Compression;
+import org.apache.hadoop.hbase.regionserver.BloomType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Map;
+
+public class HBaseEntitySchemaManager {
+
+    private static final Logger LOG = LoggerFactory.getLogger(HBaseEntitySchemaManager.class);
+    private static HBaseEntitySchemaManager instance;
+    private volatile HBaseAdmin admin;
+
+    private final int DEFAULT_MAX_VERSIONS = 1;
+
+    private HBaseEntitySchemaManager() {}
+
+    public static HBaseEntitySchemaManager getInstance() {
+        if (instance == null) {
+            synchronized (HBaseEntitySchemaManager.class) {
+                if (instance == null) {
+                    instance = new HBaseEntitySchemaManager();
+                }
+            }
+        }
+        return instance;
+    }
+
+    public void init() {
+        Configuration conf = EagleConfigFactory.load().getHbaseConf();
+        try {
+            admin = new HBaseAdmin(conf);
+            Map<String, EntityDefinition> entityServiceMap = EntityDefinitionManager.entities();
+            if (entityServiceMap != null || entityServiceMap.values() != null) {
+                for (EntityDefinition entityDefinition : entityServiceMap.values()) {
+                    createTable(entityDefinition);
+                }
+            }
+        } catch (Exception e) {
+            e.printStackTrace();
+        } finally {
+            if (admin != null) {
+                try {
+                    admin.close();
+                } catch (IOException e) {
+                    e.printStackTrace();
+                }
+            }
+        }
+    }
+
+    private void createTable(EntityDefinition entityDefinition) throws IOException {
+        String tableName = entityDefinition.getTable();
+        if (admin.tableExists(tableName)) {
+            LOG.info("Table {} already exists", tableName);
+        } else {
+            HTableDescriptor tableDescriptor = new HTableDescriptor(TableName.valueOf(tableName));
+
+            // Adding column families to table descriptor
+            HColumnDescriptor columnDescriptor = new HColumnDescriptor(entityDefinition.getColumnFamily());
+            columnDescriptor.setBloomFilterType(BloomType.ROW);
+            //columnDescriptor.setCompressionType(Compression.Algorithm.SNAPPY);
+            columnDescriptor.setMaxVersions(DEFAULT_MAX_VERSIONS);
+
+            tableDescriptor.addFamily(columnDescriptor);
+
+            // Execute the table through admin
+            admin.createTable(tableDescriptor);
+            LOG.info("Successfully create Table {}", tableName);
+        }
+    }
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/b49bda14/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/HBaseStorage.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/HBaseStorage.java b/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/HBaseStorage.java
index 3cea080..1810401 100644
--- a/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/HBaseStorage.java
+++ b/eagle-core/eagle-query/eagle-storage-hbase/src/main/java/org/apache/eagle/storage/hbase/HBaseStorage.java
@@ -52,6 +52,7 @@ public class HBaseStorage extends DataStorageBase {
     
     @Override
     public void init() throws IOException {
+        HBaseEntitySchemaManager.getInstance().init();
         LOG.info("Initializing");
     }