[jira] [Comment Edited] (KYLIN-3613) Kylin with Standalone HBase Cluster (enabled kerberos) could not find the main cluster namespace at "Create HTable" step

2018-10-24 Thread WangSheng (JIRA)


[ 
https://issues.apache.org/jira/browse/KYLIN-3613?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16662015#comment-16662015
 ] 

WangSheng edited comment on KYLIN-3613 at 10/24/18 9:45 AM:


Hi, I met the same problem, I found that when deploy coprocessor, kylin get 
wrong hbase conf
{code:java}
// DeployCoprocessorCLI.java
private static void initHTableCoprocessor(HTableDescriptor desc) throws 
IOException {
KylinConfig kylinConfig = KylinConfig.getInstanceFromEnv();
// here, hconf is not corrent
Configuration hconf = HBaseConnection.getCurrentHBaseConfiguration();
FileSystem fileSystem = FileSystem.get(hconf);

String localCoprocessorJar = kylinConfig.getCoprocessorLocalJar();
Path hdfsCoprocessorJar = 
DeployCoprocessorCLI.uploadCoprocessorJar(localCoprocessorJar, fileSystem, 
null);

DeployCoprocessorCLI.addCoprocessorOnHTable(desc, hdfsCoprocessorJar);
}

// HBaseConnection.java
public static Configuration getCurrentHBaseConfiguration() {
if (configThreadLocal.get() == null) {
// when restart kylin, here is correct
StorageURL storageUrl = 
KylinConfig.getInstanceFromEnv().getStorageUrl();
configThreadLocal.set(newHBaseConfiguration(storageUrl));
}
// after a while, here get wrong config
return configThreadLocal.get();
}
{code}
When restart kylin, configThreadLocal.get().get("fs.defaultFS") is hbase 
cluster nameservice, so everything is normal. But after a while, 
configThreadLocal.get().get("fs.defaultFS") became kylin local cluster 
nameservice, so build error, but I not sure why, still debug..



was (Author: skyyws):
Hi, I met the same problem, I found that when deploy coprocessor, kylin get 
wrong hbase conf
{code:java}
// DeployCoprocessorCLI.java
private static void initHTableCoprocessor(HTableDescriptor desc) throws 
IOException {
KylinConfig kylinConfig = KylinConfig.getInstanceFromEnv();
// here, hbase conf is not corrent
Configuration hconf = HBaseConnection.getCurrentHBaseConfiguration();
FileSystem fileSystem = FileSystem.get(hconf);

String localCoprocessorJar = kylinConfig.getCoprocessorLocalJar();
Path hdfsCoprocessorJar = 
DeployCoprocessorCLI.uploadCoprocessorJar(localCoprocessorJar, fileSystem, 
null);

DeployCoprocessorCLI.addCoprocessorOnHTable(desc, hdfsCoprocessorJar);
}

// HBaseConnection.java
public static Configuration getCurrentHBaseConfiguration() {
if (configThreadLocal.get() == null) {
// when restart kylin, here is correct
StorageURL storageUrl = 
KylinConfig.getInstanceFromEnv().getStorageUrl();
configThreadLocal.set(newHBaseConfiguration(storageUrl));
}
// after a while, here get wrong config
return configThreadLocal.get();
}
{code}
When restart kylin, configThreadLocal.get().get("fs.defaultFS") is hbase 
cluster nameservice, so everything is normal. But after a while, 
configThreadLocal.get().get("fs.defaultFS") became kylin local cluster 
nameservice, so build error, but I not sure why, still debug..


> Kylin with Standalone HBase Cluster (enabled kerberos) could not find the 
> main cluster namespace at  "Create HTable" step
> -
>
> Key: KYLIN-3613
> URL: https://issues.apache.org/jira/browse/KYLIN-3613
> Project: Kylin
>  Issue Type: Bug
>  Components: Environment 
>Affects Versions: v2.4.0
>Reporter: powerinf
>Priority: Major
> Attachments: Kylin_standalone_hbase.log
>
>
> I deployed two hadoop cluster(also enabled kerberos ,with cross-realm trust) 
> the main cluster and hbase cluster,Kylin Server can access both clusters 
> using hdfs shell with fully qualifiered path ,can submit MR job to main 
> cluster, and can use hive shell to access data warehouse
> on Kylin Server, the configurations of hadoop and hive points to main 
> cluster,and can access hbase cluster using hbase shell.
> when I build the cube, at "Create HTable" step, it reported the error 
> "java.net.UnknownHostException: ctyunbigdata Set hbase.table.sanity.checks to 
> false 
> at conf or table descriptor if you want to bypass sanity checks",but after I 
> restart Kylin serer , resume it can run normally,Why?
> more detail message on Kylin_standalone_hbase.log



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Comment Edited] (KYLIN-3613) Kylin with Standalone HBase Cluster (enabled kerberos) could not find the main cluster namespace at "Create HTable" step

2018-10-24 Thread WangSheng (JIRA)


[ 
https://issues.apache.org/jira/browse/KYLIN-3613?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16662020#comment-16662020
 ] 

WangSheng edited comment on KYLIN-3613 at 10/24/18 9:43 AM:


By the way, remove configThreadLocal may work:
{code:java}
// HBaseConnection.java
public static Configuration getCurrentHBaseConfiguration() {
StorageURL storageUrl = 
KylinConfig.getInstanceFromEnv().getStorageUrl();
return newHBaseConfiguration(storageUrl);
}
{code}



was (Author: skyyws):
By the way, remove configThreadLocal may work:
{code:java}
// HBaseConnection.java
public static Configuration getCurrentHBaseConfiguration() {
StorageURL storageUrl = 
KylinConfig.getInstanceFromEnv().getStorageUrl();
return newHBaseConfiguration(storageUrl);
}
{code}


> Kylin with Standalone HBase Cluster (enabled kerberos) could not find the 
> main cluster namespace at  "Create HTable" step
> -
>
> Key: KYLIN-3613
> URL: https://issues.apache.org/jira/browse/KYLIN-3613
> Project: Kylin
>  Issue Type: Bug
>  Components: Environment 
>Affects Versions: v2.4.0
>Reporter: powerinf
>Priority: Major
> Attachments: Kylin_standalone_hbase.log
>
>
> I deployed two hadoop cluster(also enabled kerberos ,with cross-realm trust) 
> the main cluster and hbase cluster,Kylin Server can access both clusters 
> using hdfs shell with fully qualifiered path ,can submit MR job to main 
> cluster, and can use hive shell to access data warehouse
> on Kylin Server, the configurations of hadoop and hive points to main 
> cluster,and can access hbase cluster using hbase shell.
> when I build the cube, at "Create HTable" step, it reported the error 
> "java.net.UnknownHostException: ctyunbigdata Set hbase.table.sanity.checks to 
> false 
> at conf or table descriptor if you want to bypass sanity checks",but after I 
> restart Kylin serer , resume it can run normally,Why?
> more detail message on Kylin_standalone_hbase.log



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)