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 31CA710239 for ; Mon, 13 Jan 2014 23:56:15 +0000 (UTC) Received: (qmail 84187 invoked by uid 500); 13 Jan 2014 23:56:04 -0000 Delivered-To: apmail-hbase-issues-archive@hbase.apache.org Received: (qmail 84125 invoked by uid 500); 13 Jan 2014 23:55:59 -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 84023 invoked by uid 99); 13 Jan 2014 23:55:57 -0000 Received: from arcas.apache.org (HELO arcas.apache.org) (140.211.11.28) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 13 Jan 2014 23:55:57 +0000 Date: Mon, 13 Jan 2014 23:55:57 +0000 (UTC) From: "Andrew Purtell (JIRA)" To: issues@hbase.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Updated] (HBASE-6873) Clean up Coprocessor loading failure handling 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-6873?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Andrew Purtell updated HBASE-6873: ---------------------------------- Attachment: 6873.patch Here is what I'm thinking. Not tested yet, needs a unit test. Back with an update shortly. > Clean up Coprocessor loading failure handling > --------------------------------------------- > > Key: HBASE-6873 > URL: https://issues.apache.org/jira/browse/HBASE-6873 > Project: HBase > Issue Type: Sub-task > Components: Coprocessors, regionserver > Affects Versions: 0.98.0 > Reporter: David Arthur > Assignee: Andrew Purtell > Priority: Blocker > Fix For: 0.98.0 > > Attachments: 6873.patch > > > When registering a coprocessor with a missing dependency, the regionserver gets stuck in an infinite fail loop. Restarting the regionserver and/or master has no affect. > E.g., > Load coprocessor from my-coproc.jar, that uses an external dependency (kafka) that is not included with HBase. > {code} > 12/09/24 13:13:15 INFO handler.OpenRegionHandler: Opening of region {NAME => 'documents,,1348505987177.6d1e1b7bb93486f096173bd401e8ef6b.', STARTKEY => '', ENDKEY => '', ENCODED => 6d1e1b7bb93486f096173bd401e8ef6b,} failed, marking as FAILED_OPEN in ZK > 12/09/24 13:13:15 DEBUG zookeeper.ZKAssign: regionserver:60020-0x139f43af2a70043 Attempting to transition node 6d1e1b7bb93486f096173bd401e8ef6b from RS_ZK_REGION_OPENING to RS_ZK_REGION_FAILED_OPEN > 12/09/24 13:13:15 DEBUG zookeeper.ZKAssign: regionserver:60020-0x139f43af2a70043 Successfully transitioned node 6d1e1b7bb93486f096173bd401e8ef6b from RS_ZK_REGION_OPENING to RS_ZK_REGION_FAILED_OPEN > 12/09/24 13:13:15 INFO regionserver.HRegionServer: Received request to open region: documents,,1348505987177.6d1e1b7bb93486f096173bd401e8ef6b. > 12/09/24 13:13:15 DEBUG zookeeper.ZKAssign: regionserver:60020-0x139f43af2a70043 Attempting to transition node 6d1e1b7bb93486f096173bd401e8ef6b from M_ZK_REGION_OFFLINE to RS_ZK_REGION_OPENING > 12/09/24 13:13:15 DEBUG zookeeper.ZKAssign: regionserver:60020-0x139f43af2a70043 Successfully transitioned node 6d1e1b7bb93486f096173bd401e8ef6b from M_ZK_REGION_OFFLINE to RS_ZK_REGION_OPENING > 12/09/24 13:13:15 DEBUG regionserver.HRegion: Opening region: {NAME => 'documents,,1348505987177.6d1e1b7bb93486f096173bd401e8ef6b.', STARTKEY => '', ENDKEY => '', ENCODED => 6d1e1b7bb93486f096173bd401e8ef6b,} > 12/09/24 13:13:15 INFO regionserver.HRegion: Setting up tabledescriptor config now ... > 12/09/24 13:13:15 INFO coprocessor.CoprocessorHost: Class com.mycompany.hbase.documents.DocumentObserverCoprocessor needs to be loaded from a file - file:/path/to/my-coproc.jar. > 12/09/24 13:13:16 ERROR handler.OpenRegionHandler: Failed open of region=documents,,1348505987177.6d1e1b7bb93486f096173bd401e8ef6b., starting to roll back the global memstore size. > java.lang.IllegalStateException: Could not instantiate a region instance. > at org.apache.hadoop.hbase.regionserver.HRegion.newHRegion(HRegion.java:3595) > at org.apache.hadoop.hbase.regionserver.HRegion.openHRegion(HRegion.java:3733) > at org.apache.hadoop.hbase.regionserver.handler.OpenRegionHandler.openRegion(OpenRegionHandler.java:332) > at org.apache.hadoop.hbase.regionserver.handler.OpenRegionHandler.process(OpenRegionHandler.java:108) > at org.apache.hadoop.hbase.executor.EventHandler.run(EventHandler.java:169) > at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886) > at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908) > at java.lang.Thread.run(Thread.java:680) > Caused by: java.lang.reflect.InvocationTargetException > at sun.reflect.GeneratedConstructorAccessor15.newInstance(Unknown Source) > at sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:27) > at java.lang.reflect.Constructor.newInstance(Constructor.java:513) > at org.apache.hadoop.hbase.regionserver.HRegion.newHRegion(HRegion.java:3592) > ... 7 more > Caused by: java.lang.NoClassDefFoundError: kafka/common/NoBrokersForPartitionException > at java.lang.Class.getDeclaredConstructors0(Native Method) > at java.lang.Class.privateGetDeclaredConstructors(Class.java:2389) > at java.lang.Class.getConstructor0(Class.java:2699) > at java.lang.Class.newInstance0(Class.java:326) > at java.lang.Class.newInstance(Class.java:308) > at org.apache.hadoop.hbase.coprocessor.CoprocessorHost.loadInstance(CoprocessorHost.java:254) > at org.apache.hadoop.hbase.coprocessor.CoprocessorHost.load(CoprocessorHost.java:227) > at org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost.loadTableCoprocessors(RegionCoprocessorHost.java:162) > at org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost.(RegionCoprocessorHost.java:126) > at org.apache.hadoop.hbase.regionserver.HRegion.(HRegion.java:417) > ... 11 more > Caused by: java.lang.ClassNotFoundException: kafka.common.NoBrokersForPartitionException > at java.net.URLClassLoader$1.run(URLClassLoader.java:202) > at java.security.AccessController.doPrivileged(Native Method) > at java.net.URLClassLoader.findClass(URLClassLoader.java:190) > at java.lang.ClassLoader.loadClass(ClassLoader.java:306) > at java.lang.ClassLoader.loadClass(ClassLoader.java:247) > ... 21 more > 12/09/24 13:13:16 INFO handler.OpenRegionHandler: Opening of region {NAME => 'documents,,1348505987177.6d1e1b7bb93486f096173bd401e8ef6b.', STARTKEY => '', ENDKEY => '', ENCODED => 6d1e1b7bb93486f096173bd401e8ef6b,} failed, marking as FAILED_OPEN in ZK > 12/09/24 13:13:16 DEBUG zookeeper.ZKAssign: regionserver:60020-0x139f43af2a70043 Attempting to transition node 6d1e1b7bb93486f096173bd401e8ef6b from RS_ZK_REGION_OPENING to RS_ZK_REGION_FAILED_OPEN > 12/09/24 13:13:16 DEBUG zookeeper.ZKAssign: regionserver:60020-0x139f43af2a70043 Successfully transitioned node 6d1e1b7bb93486f096173bd401e8ef6b from RS_ZK_REGION_OPENING to RS_ZK_REGION_FAILED_OPEN > 12/09/24 13:13:16 INFO regionserver.HRegionServer: Received request to open region: documents,,1348505987177.6d1e1b7bb93486f096173bd401e8ef6b. > 12/09/24 13:13:16 DEBUG zookeeper.ZKAssign: regionserver:60020-0x139f43af2a70043 Attempting to transition node 6d1e1b7bb93486f096173bd401e8ef6b from M_ZK_REGION_OFFLINE to RS_ZK_REGION_OPENING > 12/09/24 13:13:16 DEBUG zookeeper.ZKAssign: regionserver:60020-0x139f43af2a70043 Successfully transitioned node 6d1e1b7bb93486f096173bd401e8ef6b from M_ZK_REGION_OFFLINE to RS_ZK_REGION_OPENING > 12/09/24 13:13:16 DEBUG regionserver.HRegion: Opening region: {NAME => 'documents,,1348505987177.6d1e1b7bb93486f096173bd401e8ef6b.', STARTKEY => '', ENDKEY => '', ENCODED => 6d1e1b7bb93486f096173bd401e8ef6b,} > 12/09/24 13:13:16 INFO regionserver.HRegion: Setting up tabledescriptor config now ... > 12/09/24 13:13:16 INFO coprocessor.CoprocessorHost: Class com.mycompany.hbase.documents.DocumentObserverCoprocessor needs to be loaded from a file - file:/path/to/my-coproc.jar. > 12/09/24 13:13:17 ERROR handler.OpenRegionHandler: Failed open of region=documents,,1348505987177.6d1e1b7bb93486f096173bd401e8ef6b., starting to roll back the global memstore size. > java.lang.IllegalStateException: Could not instantiate a region instance. > at org.apache.hadoop.hbase.regionserver.HRegion.newHRegion(HRegion.java:3595) > at org.apache.hadoop.hbase.regionserver.HRegion.openHRegion(HRegion.java:3733) > at org.apache.hadoop.hbase.regionserver.handler.OpenRegionHandler.openRegion(OpenRegionHandler.java:332) > at org.apache.hadoop.hbase.regionserver.handler.OpenRegionHandler.process(OpenRegionHandler.java:108) > at org.apache.hadoop.hbase.executor.EventHandler.run(EventHandler.java:169) > at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886) > at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908) > at java.lang.Thread.run(Thread.java:680) > Caused by: java.lang.reflect.InvocationTargetException > at sun.reflect.GeneratedConstructorAccessor15.newInstance(Unknown Source) > at sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:27) > at java.lang.reflect.Constructor.newInstance(Constructor.java:513) > at org.apache.hadoop.hbase.regionserver.HRegion.newHRegion(HRegion.java:3592) > ... 7 more > Caused by: java.lang.NoClassDefFoundError: kafka/common/NoBrokersForPartitionException > at java.lang.Class.getDeclaredConstructors0(Native Method) > at java.lang.Class.privateGetDeclaredConstructors(Class.java:2389) > at java.lang.Class.getConstructor0(Class.java:2699) > at java.lang.Class.newInstance0(Class.java:326) > at java.lang.Class.newInstance(Class.java:308) > at org.apache.hadoop.hbase.coprocessor.CoprocessorHost.loadInstance(CoprocessorHost.java:254) > at org.apache.hadoop.hbase.coprocessor.CoprocessorHost.load(CoprocessorHost.java:227) > at org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost.loadTableCoprocessors(RegionCoprocessorHost.java:162) > at org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost.(RegionCoprocessorHost.java:126) > at org.apache.hadoop.hbase.regionserver.HRegion.(HRegion.java:417) > ... 11 more > Caused by: java.lang.ClassNotFoundException: kafka.common.NoBrokersForPartitionException > at java.net.URLClassLoader$1.run(URLClassLoader.java:202) > at java.security.AccessController.doPrivileged(Native Method) > at java.net.URLClassLoader.findClass(URLClassLoader.java:190) > at java.lang.ClassLoader.loadClass(ClassLoader.java:306) > at java.lang.ClassLoader.loadClass(ClassLoader.java:247) > ... 21 more > {code} > Ad infinitum. > It seems that upon failing to open a region after adding a coprocessor, that coprocessor should be unregister or at least disabled/blacklisted. -- This message was sent by Atlassian JIRA (v6.1.5#6160)