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 4305911588 for ; Fri, 25 Jul 2014 10:22:41 +0000 (UTC) Received: (qmail 19044 invoked by uid 500); 25 Jul 2014 10:22:41 -0000 Delivered-To: apmail-hbase-issues-archive@hbase.apache.org Received: (qmail 19005 invoked by uid 500); 25 Jul 2014 10:22:41 -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 18987 invoked by uid 99); 25 Jul 2014 10:22:40 -0000 Received: from arcas.apache.org (HELO arcas.apache.org) (140.211.11.28) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 25 Jul 2014 10:22:40 +0000 Date: Fri, 25 Jul 2014 10:22:40 +0000 (UTC) From: "ramkrishna.s.vasudevan (JIRA)" To: issues@hbase.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Commented] (HBASE-11339) HBase MOB 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-11339?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14074262#comment-14074262 ] ramkrishna.s.vasudevan commented on HBASE-11339: ------------------------------------------------ Bulk loading mob files is what was discussed in internal discussions and why use table.put() in the sweep tool. Using table.put is again flushing the data to the memstore and internally causes the flushes to happen thus affecting the write path of the system. Bulk loading mob is possible and it should work fine considering HBASE-6630 available where the bulk loaded files are also assigned with a sequence number and the same sequence number can be used to resolve a conflict in case the keyvalueheap finds two cells with same row, ts but different values. In our case of sweep tool one thing to note is that by using this tool we are trying to create a new store file for a same row, ts, cf, cq cell but update it with a new value. Here the new value is that of the new path that we are generating after the sweep tool merges some of the mob data into one single file. So consider in our case row1, cf,c1, ts1 = path1. The above data is written in Storefile 1 The updated path is path 2 and so we try to bulk load that new info into a new store file row1,cf1,c1,ts1 = path2. Now the HFile containing the new value is bulk loaded into the system and we try to scan for row1. What we would expect is to get the cell with path2 as the value and that should come from the bulk loaded file. *Does this happen - Yes in case of 0.96 - No in case of 0.98+* . In 0.96 case the compacted file will have kvs with mvcc as 0 if the kvs are smaller than the smallest read point. So in case where a scanner is opened after a set of files have been compacted all the kvs will have mvcc = 0 in it. In 0.98+ above that is not the case because {code} long oldestHFileTimeStampToKeepMVCC = System.currentTimeMillis() - (1000L * 60 * 60 * 24 * this.keepSeqIdPeriod); for (StoreFile file : filesToCompact) { if(allFiles && (file.getModificationTimeStamp() < oldestHFileTimeStampToKeepMVCC)) { // when isAllFiles is true, all files are compacted so we can calculate the smallest // MVCC value to keep if(fd.minSeqIdToKeep < file.getMaxMemstoreTS()) { fd.minSeqIdToKeep = file.getMaxMemstoreTS(); } } {code} And so the performCompaction() {code} KeyValue kv = KeyValueUtil.ensureKeyValue(c); if (cleanSeqId && kv.getSequenceId() <= smallestReadPoint) { kv.setSequenceId(0); } {code} is not able to setSeqId to 0 as atleast for 5 days we expect the value to be retained. Remember that in the above case we are assigning seq numbers to bulk loaded files also and the case there is that when the scanner starts the bulk loaded file is having the highest seq id and that is ensured by using HFileOutputFormat2 which writes {code} w.appendFileInfo(StoreFile.BULKLOAD_TIME_KEY, Bytes.toBytes(System.currentTimeMillis())); {code} So on opening the reader for this bulk loaded store file we are able to get the sequence id. {code} if (isBulkLoadResult()){ // generate the sequenceId from the fileName // fileName is of the form _SeqId__ String fileName = this.getPath().getName(); int startPos = fileName.indexOf("SeqId_"); if (startPos != -1) { this.sequenceid = Long.parseLong(fileName.substring(startPos + 6, fileName.indexOf('_', startPos + 6))); // Handle reference files as done above. if (fileInfo.isTopReference()) { this.sequenceid += 1; } } } this.reader.setSequenceID(this.sequenceid); {code} Now when the scanner tries to read from the above two files which has same cells in it for row1,cf,c1,ts1 but with path1 and path 2 as the values, the mvcc in the compacted store files that has path 1 (is a non-zero positive value) in 0.98+ and 0 in 0.96 case) and the mvcc for the KV in the store file generated by bulk load will have 0 in it (both 0.98+ and 0.96). In KeyValueHeap.java {code} public int compare(KeyValueScanner left, KeyValueScanner right) { int comparison = compare(left.peek(), right.peek()); if (comparison != 0) { return comparison; } else { // Since both the keys are exactly the same, we break the tie in favor // of the key which came latest. long leftSequenceID = left.getSequenceID(); long rightSequenceID = right.getSequenceID(); if (leftSequenceID > rightSequenceID) { return -1; } else if (leftSequenceID < rightSequenceID) { return 1; } else { return 0; } } } {code} In 0.96 when the scanner tries to compare the different StoreFileScanner to retrieve from which file the scan has to happen, the if condition will give a '0' because the KV will have all items same - row1,cf,c1,ts1 and mvcc =0. So it tries to get the reader's sequence id (else part of the code) and in the above case the bulk loaded file has the highest sequence id and so that row1,cf1,c1,ts1 with path2 is the KV that is returned. In 0.98 case since the mvcc of the kv in the compacted file is a non-zero value we always tend to return the compacted file and so the result would be row1,cf1,c1,ts1 with path1. So this is a behavioral change between 0.96 and 0.98 and also considering that the seq id of the bulk loaded file is higher than the compacted file it makes sense to read from the bulk loaded file than the compacted file as it is the newest value. If this is an issue we can raise a JIRA and find a soln for it. Correct me if am wrong. Feedback appreciated. > HBase MOB > --------- > > Key: HBASE-11339 > URL: https://issues.apache.org/jira/browse/HBASE-11339 > Project: HBase > Issue Type: New Feature > Components: regionserver, Scanners > Reporter: Jingcheng Du > Assignee: Jingcheng Du > Attachments: HBase MOB Design-v2.pdf, HBase MOB Design.pdf, MOB user guide.docx, hbase-11339-in-dev.patch > > > It's quite useful to save the medium binary data like images, documents into Apache HBase. Unfortunately directly saving the binary MOB(medium object) to HBase leads to a worse performance since the frequent split and compaction. > In this design, the MOB data are stored in an more efficient way, which keeps a high write/read performance and guarantees the data consistency in Apache HBase. -- This message was sent by Atlassian JIRA (v6.2#6252)