From user-return-46819-apmail-hbase-user-archive=hbase.apache.org@hbase.apache.org Sat Jan 10 00:12:24 2015 Return-Path: X-Original-To: apmail-hbase-user-archive@www.apache.org Delivered-To: apmail-hbase-user-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 5B14F178EF for ; Sat, 10 Jan 2015 00:12:24 +0000 (UTC) Received: (qmail 58568 invoked by uid 500); 10 Jan 2015 00:12:23 -0000 Delivered-To: apmail-hbase-user-archive@hbase.apache.org Received: (qmail 58490 invoked by uid 500); 10 Jan 2015 00:12:23 -0000 Mailing-List: contact user-help@hbase.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: user@hbase.apache.org Delivered-To: mailing list user@hbase.apache.org Received: (qmail 58473 invoked by uid 99); 10 Jan 2015 00:12:23 -0000 Received: from nike.apache.org (HELO nike.apache.org) (192.87.106.230) by apache.org (qpsmtpd/0.29) with ESMTP; Sat, 10 Jan 2015 00:12:23 +0000 X-ASF-Spam-Status: No, hits=1.5 required=5.0 tests=HTML_MESSAGE,RCVD_IN_DNSWL_LOW,SPF_PASS X-Spam-Check-By: apache.org Received-SPF: pass (nike.apache.org: domain of yuzhihong@gmail.com designates 209.85.160.182 as permitted sender) Received: from [209.85.160.182] (HELO mail-yk0-f182.google.com) (209.85.160.182) by apache.org (qpsmtpd/0.29) with ESMTP; Sat, 10 Jan 2015 00:11:56 +0000 Received: by mail-yk0-f182.google.com with SMTP id 131so5306051ykp.13 for ; Fri, 09 Jan 2015 16:11:55 -0800 (PST) DKIM-Signature: v=1; a=rsa-sha256; c=relaxed/relaxed; d=gmail.com; s=20120113; h=mime-version:in-reply-to:references:date:message-id:subject:from:to :content-type; bh=dbuXiaHUh7QMeOcCpGYRrh8C/Y6pgEbCSc6+XiQt4nU=; b=jYg/cugkmcONW2/9/OpwNzJaG3yLKnBPkTH4Rtp8F/z3DRAQnuk+2FkgbNApPopLzT +HRQ22aTaP7KYrFZL9RAiqsYntJbAftgqVHEnJXk5g6TfdsENGwryT/cKqJw3Qta/sLO WIWjndk/xp8Q60ylgIeXYOl4RpGJ4V3aYDzsrVp4V4S8rI/HcVdI1s9WpWfvLHrH2BVT awIWaLdNomI91MBKm4mXZ1LCD2QixHSt6iSBOQ89Tw3AkBy/GP6Yz9ElK6NeNCNUP5Gz qhaImR0rAYD0gCKJsHt1L8Y9kzX85MqTGSE5Kmq4r8TfGiP6uaN9pnROuXOPIHyGqa32 SILQ== MIME-Version: 1.0 X-Received: by 10.236.32.8 with SMTP id n8mr13582710yha.74.1420848715235; Fri, 09 Jan 2015 16:11:55 -0800 (PST) Received: by 10.170.139.4 with HTTP; Fri, 9 Jan 2015 16:11:55 -0800 (PST) In-Reply-To: References: Date: Fri, 9 Jan 2015 16:11:55 -0800 Message-ID: Subject: Re: 1 table, 1 dense CF => N tables, 1 dense CF ? From: Ted Yu To: "user@hbase.apache.org" Content-Type: multipart/alternative; boundary=001a11c1c134f3873b050c411e7e X-Virus-Checked: Checked by ClamAV on apache.org --001a11c1c134f3873b050c411e7e Content-Type: text/plain; charset=UTF-8 Thanks for the confirmation, Gary. The change has been done through HBASE-12834. Cheers On Fri, Jan 9, 2015 at 1:01 PM, Gary Helmling wrote: > ScanType is a parameter of RegionObserver preCompact() and > preCompactScannerOpen(). It seems like anything we are explicitly > providing to coprocessor hooks should be LimitedPrivate. > > On Fri, Jan 9, 2015 at 12:26 PM, Ted Yu wrote: > > > w.r.t. ScanType, here is the logic used by DefaultCompactor: > > > > ScanType scanType = > > > > request.isAllFiles() ? ScanType.COMPACT_DROP_DELETES : > > ScanType. > > COMPACT_RETAIN_DELETES; > > > > BTW ScanType is currently marked InterfaceAudience.Private > > > > Should it be marked LimitedPrivate ? > > > > Cheers > > > > On Fri, Jan 9, 2015 at 12:19 PM, Gary Helmling > > wrote: > > > > > > > > > > > > > > 2) is more expensive than 1). > > > > I'm wondering if we could use Compaction Coprocessor for 2)? > HBaseHUT > > > > needs to be able to grab N rows and merge them into 1, delete those N > > > rows, > > > > and just write that 1 new row. This N could be several thousand > rows. > > > > Could Compaction Coprocessor really be used for that? > > > > > > > > > > > It would depend on the details. If you're simply aggregating the data > > into > > > one row, and: > > > * the thousands of rows are contiguous in the scan > > > * you can somehow incrementally update or emit the new row that you > want > > to > > > create so that you don't need to retain all the old rows in memory > > > * the new row you want to emit would sort sequentially into the same > > > position > > > > > > Then overriding the scanner used for compaction could be a good > solution. > > > This would allow you to transform the cells emitted during compaction, > > > including dropping the cells from the old rows and emitting new > > > (transformed) cells for the new row. > > > > > > > > > > Also, would that come into play during minor or major compactions or > > > both? > > > > > > > > > > > You can distinguish between them in your coprocessor hooks based on > > > ScanType. So up to you. > > > > > > --001a11c1c134f3873b050c411e7e--