Return-Path: X-Original-To: apmail-hbase-issues-archive@www.apache.org Delivered-To: apmail-hbase-issues-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 0752010A12 for ; Thu, 22 Jan 2015 11:10:35 +0000 (UTC) Received: (qmail 91007 invoked by uid 500); 22 Jan 2015 11:10:34 -0000 Delivered-To: apmail-hbase-issues-archive@hbase.apache.org Received: (qmail 90963 invoked by uid 500); 22 Jan 2015 11:10:34 -0000 Mailing-List: contact issues-help@hbase.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Delivered-To: mailing list issues@hbase.apache.org Received: (qmail 90952 invoked by uid 99); 22 Jan 2015 11:10:34 -0000 Received: from arcas.apache.org (HELO arcas.apache.org) (140.211.11.28) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 22 Jan 2015 11:10:34 +0000 Date: Thu, 22 Jan 2015 11:10:34 +0000 (UTC) From: "Weichen Ye (JIRA)" To: issues@hbase.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Updated] (HBASE-12903) Wrong configuration to enable secure bulk load MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 [ https://issues.apache.org/jira/browse/HBASE-12903?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Weichen Ye updated HBASE-12903: ------------------------------- Description: In Section "62.5. Secure Bulk Load", the doc tell us to enable secure bulkload with the configuration: {code} hbase.coprocessor.regionserver.classes org.apache.hadoop.hbase.security.token.TokenProvider,org.apache.hadoop.hbase.security.access.AccessController, org.apache.hadoop.hbase.security.access.SecureBulkLoadEndpoint {code} When we set the configuration like this, the regionserver can not start due to this ERROR: {code} 2015-01-22 15:55:46,395 FATAL org.apache.hadoop.hbase.regionserver.HRegionServer: ABORTING region server hadoop-node-xxxxx: The coprocessor org.apache.hadoop.hbase.security.access.SecureBulkLoadEndpoint threw java.lang.ClassCastException: org.apache.hadoop.hbase.regionserver.RegionServerCoprocessorHost$RegionServerEnvironment can not be cast to org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment java.lang.ClassCastException: org.apache.hadoop.hbase.regionserver.RegionServerCoprocessorHost$RegionServerEnvironment cannot be cast to org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment at org.apache.hadoop.hbase.security.access.SecureBulkLoadEndpoint.start(SecureBulkLoadEndpoint.java:125) at org.apache.hadoop.hbase.coprocessor.CoprocessorHost$Environment.startup(CoprocessorHost.java:673) at org.apache.hadoop.hbase.coprocessor.CoprocessorHost.loadInstance(CoprocessorHost.java:265) at org.apache.hadoop.hbase.coprocessor.CoprocessorHost.loadSystemCoprocessors(CoprocessorHost.java:168) at org.apache.hadoop.hbase.regionserver.RegionServerCoprocessorHost.(RegionServerCoprocessorHost.java:46) at org.apache.hadoop.hbase.regionserver.HRegionServer.run(HRegionServer.java:883) at java.lang.Thread.run(Thread.java:745) {code} In source code org.apache.hadoop.hbase.security.access.SecureBulkLoadEndpoint we can see, the environment for this coprocessor is "RegionCoprocessorEnvironment". So the coprocessor "SecureBulkLoadEndpoint" should be put into "hbase.coprocessor.region.classes", instead of "hbase.coprocessor.regionserver.classes" The correct configuration to enable secure bulk load is: {code} hbase.coprocessor.region.classes org.apache.hadoop.hbase.security.token.TokenProvider,org.apache.hadoop.hbase.security.access.AccessController, org.apache.hadoop.hbase.security.access.SecureBulkLoadEndpoint {code} was: In Section "62.5. Secure Bulk Load", the doc tell us to enable secure bulkload with the configuration: hbase.coprocessor.regionserver.classes org.apache.hadoop.hbase.security.token.TokenProvider,org.apache.hadoop.hbase.security.access.AccessController, org.apache.hadoop.hbase.security.access.SecureBulkLoadEndpoint When we set the configuration like this, the regionserver can not start due to this ERROR: 2015-01-22 15:55:46,395 FATAL org.apache.hadoop.hbase.regionserver.HRegionServer: ABORTING region server hadoop-node-xxxxx: The coprocessor org.apache.hadoop.hbase.security.access.SecureBulkLoadEndpoint threw java.lang.ClassCastException: org.apache.hadoop.hbase.regionserver.RegionServerCoprocessorHost$RegionServerEnvironment can not be cast to org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment java.lang.ClassCastException: org.apache.hadoop.hbase.regionserver.RegionServerCoprocessorHost$RegionServerEnvironment cannot be cast to org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment at org.apache.hadoop.hbase.security.access.SecureBulkLoadEndpoint.start(SecureBulkLoadEndpoint.java:125) at org.apache.hadoop.hbase.coprocessor.CoprocessorHost$Environment.startup(CoprocessorHost.java:673) at org.apache.hadoop.hbase.coprocessor.CoprocessorHost.loadInstance(CoprocessorHost.java:265) at org.apache.hadoop.hbase.coprocessor.CoprocessorHost.loadSystemCoprocessors(CoprocessorHost.java:168) at org.apache.hadoop.hbase.regionserver.RegionServerCoprocessorHost.(RegionServerCoprocessorHost.java:46) at org.apache.hadoop.hbase.regionserver.HRegionServer.run(HRegionServer.java:883) at java.lang.Thread.run(Thread.java:745) In source code org.apache.hadoop.hbase.security.access.SecureBulkLoadEndpoint we can see, the environment for this coprocessor is "RegionCoprocessorEnvironment". So the coprocessor "SecureBulkLoadEndpoint" should be put into "hbase.coprocessor.region.classes", instead of "hbase.coprocessor.regionserver.classes" The correct configuration to enable secure bulk load is: hbase.coprocessor.region.classes org.apache.hadoop.hbase.security.token.TokenProvider,org.apache.hadoop.hbase.security.access.AccessController, org.apache.hadoop.hbase.security.access.SecureBulkLoadEndpoint > Wrong configuration to enable secure bulk load > ---------------------------------------------- > > Key: HBASE-12903 > URL: https://issues.apache.org/jira/browse/HBASE-12903 > Project: HBase > Issue Type: Bug > Components: Coprocessors, documentation > Affects Versions: 2.0.0, 0.98.6 > Reporter: Weichen Ye > Assignee: Weichen Ye > Attachments: HBASE-12903.patch > > > In Section "62.5. Secure Bulk Load", the doc tell us to enable secure bulkload with the configuration: > {code} > > hbase.coprocessor.regionserver.classes > org.apache.hadoop.hbase.security.token.TokenProvider,org.apache.hadoop.hbase.security.access.AccessController, > org.apache.hadoop.hbase.security.access.SecureBulkLoadEndpoint > > {code} > When we set the configuration like this, the regionserver can not start due to this ERROR: > {code} > 2015-01-22 15:55:46,395 FATAL org.apache.hadoop.hbase.regionserver.HRegionServer: ABORTING region server hadoop-node-xxxxx: The coprocessor org.apache.hadoop.hbase.security.access.SecureBulkLoadEndpoint threw java.lang.ClassCastException: org.apache.hadoop.hbase.regionserver.RegionServerCoprocessorHost$RegionServerEnvironment can not be cast to org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment > java.lang.ClassCastException: org.apache.hadoop.hbase.regionserver.RegionServerCoprocessorHost$RegionServerEnvironment cannot be cast to org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment > at org.apache.hadoop.hbase.security.access.SecureBulkLoadEndpoint.start(SecureBulkLoadEndpoint.java:125) > at org.apache.hadoop.hbase.coprocessor.CoprocessorHost$Environment.startup(CoprocessorHost.java:673) > at org.apache.hadoop.hbase.coprocessor.CoprocessorHost.loadInstance(CoprocessorHost.java:265) > at org.apache.hadoop.hbase.coprocessor.CoprocessorHost.loadSystemCoprocessors(CoprocessorHost.java:168) > at org.apache.hadoop.hbase.regionserver.RegionServerCoprocessorHost.(RegionServerCoprocessorHost.java:46) > at org.apache.hadoop.hbase.regionserver.HRegionServer.run(HRegionServer.java:883) > at java.lang.Thread.run(Thread.java:745) > {code} > In source code org.apache.hadoop.hbase.security.access.SecureBulkLoadEndpoint we can see, the environment for this coprocessor is "RegionCoprocessorEnvironment". So the coprocessor "SecureBulkLoadEndpoint" should be put into "hbase.coprocessor.region.classes", instead of "hbase.coprocessor.regionserver.classes" > The correct configuration to enable secure bulk load is: > {code} > > hbase.coprocessor.region.classes org.apache.hadoop.hbase.security.token.TokenProvider,org.apache.hadoop.hbase.security.access.AccessController, > org.apache.hadoop.hbase.security.access.SecureBulkLoadEndpoint > > {code} -- This message was sent by Atlassian JIRA (v6.3.4#6332)