Today we tried tp execute a job on the cluster instead of on local executor and we faced the problem that the hbase-site.xml was basically ignored. Is there a reason why the TableInputFormat is working correctly on local environment while it doesn't on a cluster? On Nov 10, 2014 10:56 AM, "Fabian Hueske" <[email protected]> wrote:
> I don't think we need to bundle the HBase input and output format in a > single PR. > So, I think we can proceed with the IF only and target the OF later. > However, the fix for Kryo should be in the master before merging the PR. > Till is currently working on that and said he expects this to be done by > end of the week. > > Cheers, Fabian > > > 2014-11-07 12:49 GMT+01:00 Flavio Pompermaier <[email protected]>: > > > I fixed also the profile for Cloudera CDH5.1.3. You can build it with the > > command: > > mvn clean install -Dmaven.test.skip=true -Dhadoop.profile=2 > > -Pvendor-repos,cdh5.1.3 > > > > However, it would be good to generate the specific jar when > > releasing..(e.g. > > flink-addons:flink-hbase:0.8.0-hadoop2-cdh5.1.3-incubating) > > > > Best, > > Flavio > > > > On Fri, Nov 7, 2014 at 12:44 PM, Flavio Pompermaier < > [email protected]> > > wrote: > > > > > I've just updated the code on my fork (synch with current master and > > > applied improvements coming from comments on related PR). > > > I still have to understand how to write results back to an HBase > > > Sink/OutputFormat... > > > > > > > > > On Mon, Nov 3, 2014 at 12:05 PM, Flavio Pompermaier < > > [email protected]> > > > wrote: > > > > > >> Thanks for the detailed answer. So if I run a job from my machine I'll > > >> have to download all the scanned data in a table..right? > > >> > > >> Always regarding the GenericTableOutputFormat it is not clear to me > how > > >> to proceed.. > > >> I saw in the hadoop compatibility addon that it is possible to have > such > > >> compatibility using HBaseUtils class so the open method should become > > >> something like: > > >> > > >> @Override > > >> public void open(int taskNumber, int numTasks) throws IOException { > > >> if (Integer.toString(taskNumber + 1).length() > 6) { > > >> throw new IOException("Task id too large."); > > >> } > > >> TaskAttemptID taskAttemptID = TaskAttemptID.forName("attempt__0000_r_" > > >> + String.format("%" + (6 - Integer.toString(taskNumber + 1).length()) > + > > >> "s"," ").replace(" ", "0") > > >> + Integer.toString(taskNumber + 1) > > >> + "_0"); > > >> this.configuration.set("mapred.task.id", taskAttemptID.toString()); > > >> this.configuration.setInt("mapred.task.partition", taskNumber + 1); > > >> // for hadoop 2.2 > > >> this.configuration.set("mapreduce.task.attempt.id", > > >> taskAttemptID.toString()); > > >> this.configuration.setInt("mapreduce.task.partition", taskNumber + 1); > > >> try { > > >> this.context = > > >> HadoopUtils.instantiateTaskAttemptContext(this.configuration, > > >> taskAttemptID); > > >> } catch (Exception e) { > > >> throw new RuntimeException(e); > > >> } > > >> final HFileOutputFormat2 outFormat = new HFileOutputFormat2(); > > >> try { > > >> this.writer = outFormat.getRecordWriter(this.context); > > >> } catch (InterruptedException iex) { > > >> throw new IOException("Opening the writer was interrupted.", iex); > > >> } > > >> } > > >> > > >> But I'm not sure about how to pass the JobConf to the class, if to > merge > > >> config fileas, where HFileOutputFormat2 writes the data and how to > > >> implement the public void writeRecord(Record record) API. > > >> Could I do a little chat off the mailing list with the implementor of > > >> this extension? > > >> > > >> On Mon, Nov 3, 2014 at 11:51 AM, Fabian Hueske <[email protected]> > > >> wrote: > > >> > > >>> Hi Flavio > > >>> > > >>> let me try to answer your last question on the user's list (to the > best > > >>> of > > >>> my HBase knowledge). > > >>> "I just wanted to known if and how regiom splitting is handled. Can > you > > >>> explain me in detail how Flink and HBase works?what is not fully > clear > > to > > >>> me is when computation is done by region servers and when data start > > flow > > >>> to a Flink worker (that in ky test job is only my pc) and how ro > > >>> undertsand > > >>> better the important logged info to understand if my job is > performing > > >>> well" > > >>> > > >>> HBase partitions its tables into so called "regions" of keys and > stores > > >>> the > > >>> regions distributed in the cluster using HDFS. I think an HBase > region > > >>> can > > >>> be thought of as a HDFS block. To make reading an HBase table > > efficient, > > >>> region reads should be locally done, i.e., an InputFormat should > > >>> primarily > > >>> read region that are stored on the same machine as the IF is running > > on. > > >>> Flink's InputSplits partition the HBase input by regions and add > > >>> information about the storage location of the region. During > execution, > > >>> input splits are assigned to InputFormats that can do local reads. > > >>> > > >>> Best, Fabian > > >>> > > >>> 2014-11-03 11:13 GMT+01:00 Stephan Ewen <[email protected]>: > > >>> > > >>> > Hi! > > >>> > > > >>> > The way of passing parameters through the configuration is very old > > >>> (the > > >>> > original HBase format dated back to that time). I would simply make > > the > > >>> > HBase format take those parameters through the constructor. > > >>> > > > >>> > Greetings, > > >>> > Stephan > > >>> > > > >>> > > > >>> > On Mon, Nov 3, 2014 at 10:59 AM, Flavio Pompermaier < > > >>> [email protected]> > > >>> > wrote: > > >>> > > > >>> > > The problem is that I also removed the GenericTableOutputFormat > > >>> because > > >>> > > there is an incompatibility between hadoop1 and hadoop2 for class > > >>> > > TaskAttemptContext and TaskAttemptContextImpl.. > > >>> > > then it would be nice if the user doesn't have to worry about > > passing > > >>> > > pact.hbase.jtkey and pact.job.id parameters.. > > >>> > > I think it is probably a good idea to remove hadoop1 > compatibility > > >>> and > > >>> > keep > > >>> > > enable HBase addon only for hadoop2 (as before) and decide how to > > >>> mange > > >>> > > those 2 parameters.. > > >>> > > > > >>> > > On Mon, Nov 3, 2014 at 10:19 AM, Stephan Ewen <[email protected]> > > >>> wrote: > > >>> > > > > >>> > > > It is fine to remove it, in my opinion. > > >>> > > > > > >>> > > > On Mon, Nov 3, 2014 at 10:11 AM, Flavio Pompermaier < > > >>> > > [email protected]> > > >>> > > > wrote: > > >>> > > > > > >>> > > > > That is one class I removed because it was using the > deprecated > > >>> API > > >>> > > > > GenericDataSink..I can restore them but the it will be a good > > >>> idea to > > >>> > > > > remove those warning (also because from what I understood the > > >>> Record > > >>> > > APIs > > >>> > > > > are going to be removed). > > >>> > > > > > > >>> > > > > On Mon, Nov 3, 2014 at 9:51 AM, Fabian Hueske < > > >>> [email protected]> > > >>> > > > wrote: > > >>> > > > > > > >>> > > > > > I'm not familiar with the HBase connector code, but are you > > >>> maybe > > >>> > > > looking > > >>> > > > > > for the GenericTableOutputFormat? > > >>> > > > > > > > >>> > > > > > 2014-11-03 9:44 GMT+01:00 Flavio Pompermaier < > > >>> [email protected] > > >>> > >: > > >>> > > > > > > > >>> > > > > > > | was trying to modify the example setting > > hbaseDs.output(new > > >>> > > > > > > HBaseOutputFormat()); but I can't see any > HBaseOutputFormat > > >>> > > > > class..maybe > > >>> > > > > > we > > >>> > > > > > > shall use another class? > > >>> > > > > > > > > >>> > > > > > > On Mon, Nov 3, 2014 at 9:39 AM, Flavio Pompermaier < > > >>> > > > > [email protected] > > >>> > > > > > > > > >>> > > > > > > wrote: > > >>> > > > > > > > > >>> > > > > > > > Maybe that's something I could add to the HBase example > > and > > >>> > that > > >>> > > > > could > > >>> > > > > > be > > >>> > > > > > > > better documented in the Wiki. > > >>> > > > > > > > > > >>> > > > > > > > Since we're talking about the wiki..I was looking at > the > > >>> Java > > >>> > > API ( > > >>> > > > > > > > > > >>> > > > > > > > > >>> > > > > > > > >>> > > > > > > >>> > > > > > >>> > > > > >>> > > > >>> > > > http://flink.incubator.apache.org/docs/0.6-incubating/java_api_guide.html > > >>> ) > > >>> > > > > > > > and the link to the KMeans example is not working > (where > > it > > >>> > says > > >>> > > > For > > >>> > > > > a > > >>> > > > > > > > complete example program, have a look at KMeans > > Algorithm). > > >>> > > > > > > > > > >>> > > > > > > > Best, > > >>> > > > > > > > Flavio > > >>> > > > > > > > > > >>> > > > > > > > > > >>> > > > > > > > On Mon, Nov 3, 2014 at 9:12 AM, Flavio Pompermaier < > > >>> > > > > > [email protected] > > >>> > > > > > > > > > >>> > > > > > > > wrote: > > >>> > > > > > > > > > >>> > > > > > > >> Ah ok, perfect! That was the reason why I removed it > :) > > >>> > > > > > > >> > > >>> > > > > > > >> On Mon, Nov 3, 2014 at 9:10 AM, Stephan Ewen < > > >>> > [email protected]> > > >>> > > > > > wrote: > > >>> > > > > > > >> > > >>> > > > > > > >>> You do not really need a HBase data sink. You can > call > > >>> > > > > > > >>> "DataSet.output(new > > >>> > > > > > > >>> HBaseOutputFormat())" > > >>> > > > > > > >>> > > >>> > > > > > > >>> Stephan > > >>> > > > > > > >>> Am 02.11.2014 23:05 schrieb "Flavio Pompermaier" < > > >>> > > > > > [email protected] > > >>> > > > > > > >: > > >>> > > > > > > >>> > > >>> > > > > > > >>> > Just one last thing..I removed the HbaseDataSink > > >>> because I > > >>> > > > think > > >>> > > > > it > > >>> > > > > > > was > > >>> > > > > > > >>> > using the old APIs..can someone help me in updating > > >>> that > > >>> > > class? > > >>> > > > > > > >>> > > > >>> > > > > > > >>> > On Sun, Nov 2, 2014 at 10:55 AM, Flavio > Pompermaier < > > >>> > > > > > > >>> [email protected]> > > >>> > > > > > > >>> > wrote: > > >>> > > > > > > >>> > > > >>> > > > > > > >>> > > Indeed this time the build has been successful :) > > >>> > > > > > > >>> > > > > >>> > > > > > > >>> > > On Sun, Nov 2, 2014 at 10:29 AM, Fabian Hueske < > > >>> > > > > > [email protected] > > >>> > > > > > > > > > >>> > > > > > > >>> > wrote: > > >>> > > > > > > >>> > > > > >>> > > > > > > >>> > >> You can also setup Travis to build your own > Github > > >>> > > > > repositories > > >>> > > > > > by > > >>> > > > > > > >>> > linking > > >>> > > > > > > >>> > >> it to your Github account. That way Travis can > > >>> build all > > >>> > > > your > > >>> > > > > > > >>> branches > > >>> > > > > > > >>> > >> (and > > >>> > > > > > > >>> > >> you can also trigger rebuilds if something > fails). > > >>> > > > > > > >>> > >> Not sure if we can manually trigger retrigger > > >>> builds on > > >>> > > the > > >>> > > > > > Apache > > >>> > > > > > > >>> > >> repository. > > >>> > > > > > > >>> > >> > > >>> > > > > > > >>> > >> Support for Hadoop 1 and 2 is indeed a very good > > >>> > addition > > >>> > > > :-) > > >>> > > > > > > >>> > >> > > >>> > > > > > > >>> > >> For the discusion about the PR itself, I would > > need > > >>> a > > >>> > bit > > >>> > > > more > > >>> > > > > > > time > > >>> > > > > > > >>> to > > >>> > > > > > > >>> > >> become more familiar with HBase. I do also not > > have > > >>> a > > >>> > > HBase > > >>> > > > > > setup > > >>> > > > > > > >>> > >> available > > >>> > > > > > > >>> > >> here. > > >>> > > > > > > >>> > >> Maybe somebody else of the community who was > > >>> involved > > >>> > > with a > > >>> > > > > > > >>> previous > > >>> > > > > > > >>> > >> version of the HBase connector could comment on > > your > > >>> > > > question. > > >>> > > > > > > >>> > >> > > >>> > > > > > > >>> > >> Best, Fabian > > >>> > > > > > > >>> > >> > > >>> > > > > > > >>> > >> 2014-11-02 9:57 GMT+01:00 Flavio Pompermaier < > > >>> > > > > > > [email protected] > > >>> > > > > > > >>> >: > > >>> > > > > > > >>> > >> > > >>> > > > > > > >>> > >> > As suggestes by Fabian I moved the discussion > on > > >>> this > > >>> > > > > mailing > > >>> > > > > > > >>> list. > > >>> > > > > > > >>> > >> > > > >>> > > > > > > >>> > >> > I think that what is still to be discussed is > > >>> how to > > >>> > > > > > retrigger > > >>> > > > > > > >>> the > > >>> > > > > > > >>> > >> build > > >>> > > > > > > >>> > >> > on Travis (I don't have an account) and if the > > PR > > >>> can > > >>> > be > > >>> > > > > > > >>> integrated. > > >>> > > > > > > >>> > >> > > > >>> > > > > > > >>> > >> > Maybe what I can do is to move the HBase > example > > >>> in > > >>> > the > > >>> > > > test > > >>> > > > > > > >>> package > > >>> > > > > > > >>> > >> (right > > >>> > > > > > > >>> > >> > now I left it in the main folder) so it will > > force > > >>> > > Travis > > >>> > > > to > > >>> > > > > > > >>> rebuild. > > >>> > > > > > > >>> > >> > I'll do it within a couple of hours. > > >>> > > > > > > >>> > >> > > > >>> > > > > > > >>> > >> > Another thing I forgot to say is that the > hbase > > >>> > > extension > > >>> > > > is > > >>> > > > > > now > > >>> > > > > > > >>> > >> compatible > > >>> > > > > > > >>> > >> > with both hadoop 1 and 2. > > >>> > > > > > > >>> > >> > > > >>> > > > > > > >>> > >> > Best, > > >>> > > > > > > >>> > >> > Flavio > > >>> > > > > > > >>> > >> > > >>> > > > > > > >>> > > > > >>> > > > > > > >>> > > > >>> > > > > > > >>> > > >>> > > > > > > >> > > >>> > > > > > > > > > >>> > > > > > > > > >>> > > > > > > > >>> > > > > > > >>> > > > > > >>> > > > > >>> > > > >>> > > >> > > >> > > >> > > > > > >
