You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@pig.apache.org by Hema Kumar Sunapu <he...@gmail.com> on 2015/09/17 20:23:07 UTC
Re: Using secure HBase from Pig UDF?
Niels,
Did that code worked for you?
To run pig in secured (Kerberos) environment.
* Set "set hbase.security.authentication 'kerberos';" in pig script.
* And kinit with appropriate user/principal.
* Run your pig script.
if authentication is kerboros, UserGroupInformation class will read * user
identity *and used across all map-reduce code which is generated by pig
Note: Your code execution should be completed before the ticket expires.
On Tue, Jun 9, 2015 at 3:16 AM, Niels Basjes <Ni...@basjes.nl> wrote:
> Thanks, we'll give this a try!
>
> Niels Basjes
>
> On Mon, Jun 8, 2015 at 10:04 PM, Rohini Palaniswamy <
> rohini.aditya@gmail.com
> > wrote:
>
> > Niels,
> > I plan to have PIG-3038 in next two weeks which should simplify
> > accessing secure hbase, but it will only be in 0.16 and that is at least
> > 3-4 months away.
> >
> > In the meantime, a hacky way to get this done is:
> >
> > When running the pig script from commandline, do
> > ## Makes bin/pig add hbase jars and hbase-site.xml to classpath
> > export HBASE_HOME=/home/gs/hbase/current
> > export HBASE_CONF_DIR=/home/gs/conf/hbase
> >
> > pig --conf /home/gs/conf/hbase/hbase-site.xml
> >
> >
> -Dmapreduce.job.credentials.binary=/home/<username>/hbase-creds/myhbaseudf-`date
> > +%Y%m%d%H%M`.token myscript.pig
> >
> > Also define your UDF in the pig script as follows. This is required for
> the
> > pig script to connect to hbase and fetch the hbase delegation token on
> the
> > front end. This is not required if you are running the script through
> > Oozie.
> >
> >
> > define myudf com.yahoo.myudfs.MyHBaseUDF();
> >
> >
> > MyHBaseUDF.java :
> >
> > package com.yahoo.myudfs;
> >
> > import java.io.File;
> > import java.io.IOException;
> > import java.lang.reflect.Method;
> > import java.lang.reflect.UndeclaredThrowableException;
> >
> > import org.apache.hadoop.conf.Configuration;
> > import org.apache.hadoop.fs.Path;
> > import org.apache.hadoop.hbase.HBaseConfiguration;
> > import org.apache.hadoop.hbase.client.HTable;
> > import org.apache.hadoop.mapred.JobConf;
> > import org.apache.hadoop.mapreduce.Job;
> > import org.apache.hadoop.security.Credentials;
> > import org.apache.hadoop.security.UserGroupInformation;
> > import org.apache.pig.EvalFunc;
> > import org.apache.pig.data.Tuple;
> > import org.apache.pig.impl.util.UDFContext;
> >
> > public class MyHBaseUDF extends EvalFunc<Tuple> {
> >
> > private final static String HBASE_SECURITY_CONF_KEY =
> > "hbase.security.authentication";
> > private JobConf hbaseConf;
> > private HTable hTable;
> >
> > public MyHBaseUDF() {
> > try {
> > if (UDFContext.getUDFContext().isFrontend()) {
> > hbaseConf = new JobConf(HBaseConfiguration.create());
> >
> >
> > if(UserGroupInformation.getCurrentUser().hasKerberosCredentials()) {
> > // Fetch the hbase delegation and write to a
> location,
> > which can be added to the Job
> > // when it is launched. This is not required if you
> are
> > using Oozie to run pig scripts.
> > // HBase credentials should be fetched by configuring
> > the credentials section through Oozie.
> > // and will be available via
> HADOOP_TOKEN_FILE_LOCATION
> > in both FrontEnd and BackEnd.
> >
> > addHBaseDelegationToken(hbaseConf);
> > // Pass
> >
> >
> -Dmapreduce.job.credentials.binary=/homes/<username>/hbase-creds/myhbaseudf.token
> > // to pig script. It can be any local filesystem
> > location where read and write access
> > // are restricted to you.
> > String binaryTokenFilename =
> > System.getProperty("mapreduce.job.credentials.binary");
> > Credentials hbaseCreds = hbaseConf.getCredentials();
> > hbaseCreds.writeTokenStorageFile(new Path("file:///"
> +
> > binaryTokenFilename),
> > hbaseConf);
> > } else {
> > // Case of Oozie
> > String tokenFileLocation = System
> >
> > .getenv(UserGroupInformation.HADOOP_TOKEN_FILE_LOCATION);
> > if (tokenFileLocation != null) {
> > Credentials cred =
> > Credentials.readTokenStorageFile(new Path("file:///" +
> tokenFileLocation),
> > hbaseConf);
> > hbaseConf.getCredentials().addAll(cred);
> > }
> > }
> >
> > } else {
> > // Construct JobConf with the hbase delegation tokens to
> > talk to hbase
> > Configuration conf =
> > UDFContext.getUDFContext().getJobConf();
> > // 1) You need to either ship hbase-site.xml through
> > distributed cache (-Dmapred.cache.files)
> > // so that it is in classpath (or)
> > // 2) pass it to pig using --conf
> > /home/gs/conf/hbase/hbase-site.xml, so that it
> > // becomes part of the job configuration
> > hbaseConf = new JobConf(HBaseConfiguration.create(conf));
> > String tokenFileLocation = System
> >
> > .getenv(UserGroupInformation.HADOOP_TOKEN_FILE_LOCATION);
> > if (tokenFileLocation != null) {
> > Credentials cred =
> Credentials.readTokenStorageFile(new
> > Path("file:///" + tokenFileLocation), hbaseConf);
> > hbaseConf.getCredentials().addAll(cred);
> > }
> > }
> > hTable = new HTable(hbaseConf, "myhbaseTable");
> > } catch (IOException e) {
> > throw new RuntimeException(e);
> > }
> > }
> >
> > // Not required if using Oozie
> > @SuppressWarnings({ "rawtypes", "unchecked" })
> > private void addHBaseDelegationToken(JobConf hbaseConf) {
> >
> > if (!UDFContext.getUDFContext().isFrontend()) {
> > return;
> > }
> >
> > if
> > ("kerberos".equalsIgnoreCase(hbaseConf.get(HBASE_SECURITY_CONF_KEY))) {
> > try {
> > if
> > (UserGroupInformation.getCurrentUser().hasKerberosCredentials()) {
> > // Class and method are available only from 0.92
> > security release
> > Class tokenUtilClass = Class
> >
> > .forName("org.apache.hadoop.hbase.security.token.TokenUtil");
> > Method m =
> > tokenUtilClass.getMethod("obtainTokenForJob", new Class[] {
> > JobConf.class, UserGroupInformation.class });
> > m.invoke(null, new Object[] { hbaseConf,
> > UserGroupInformation.getCurrentUser() });
> > } else {
> > System.out.println("Not fetching hbase delegation
> token
> > as no Kerberos TGT is available");
> > }
> > } catch (ClassNotFoundException cnfe) {
> > throw new RuntimeException("Failure loading TokenUtil
> > class, "
> > + "is secure RPC available?", cnfe);
> > } catch (Exception e) {
> > throw new UndeclaredThrowableException(e,
> > "Unexpected error calling
> > TokenUtil.obtainTokenForJob()");
> > }
> > }
> > }
> >
> > @Override
> > public Tuple exec(Tuple input) throws IOException {
> > // Your code goes here
> > return null;
> > }
> >
> > }
> >
> >
> >
> > Regards,
> > Rohini
> >
> > On Thu, May 28, 2015 at 7:50 AM, Niels Basjes <Ni...@basjes.nl> wrote:
> >
> > > Hi,
> > >
> > > I have a UDF that needs to go into HBase to get a single value when
> > called.
> > > The HBase we have has been secured (Kerberos) and so far I have not yet
> > > been able to figure out how to get it all running.
> > >
> > > I did find these two issues but no working example I can copy:
> > >
> > > "Helper class for dealing with security in HBase for UDFs"
> > > https://issues.apache.org/jira/browse/PIG-3030
> > >
> > > and
> > >
> > > "Support for Credentials for UDF,Loader and Storer"
> > > https://issues.apache.org/jira/browse/PIG-3038
> > >
> > > Does anyone have a suggestion on how to approach this correctly ??
> > >
> > > --
> > > Best regards / Met vriendelijke groeten,
> > >
> > > Niels Basjes
> > >
> >
>
>
>
> --
> Best regards / Met vriendelijke groeten,
>
> Niels Basjes
>