[jira] [Commented] (HADOOP-16350) Ability to tell Hadoop not to request KMS Information from Remote NN

2019-06-13 Thread Tsz Wo Nicholas Sze (JIRA)


[ 
https://issues.apache.org/jira/browse/HADOOP-16350?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16863505#comment-16863505
 ] 

Tsz Wo Nicholas Sze commented on HADOOP-16350:
--

{code}
hadoop distcp -Dhadoop.security.kms.client.allow.remote.kms=false 
-Ddfs.namenode.kerberos.principal.pattern=* 
-Dmapreduce.job.hdfs-servers.token-renewal.exclude=unit 
hdfs:///processed/public/opendata/samples/distcp_test/distcp_file.txt 
hdfs://unit/processed/public/opendata/samples/distcp_test/distcp_file2.txt
{code}
[~gss2002], I understand that your patch is working but it is not a good idea 
to add a new conf.  You may already have feeling that Hadoop has too many confs!

With [my 
suggestion|https://issues.apache.org/jira/browse/HADOOP-16350?focusedCommentId=16862537&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-16862537],
 the distcp command will looks like
- hadoop distcp *-Dhadoop.security.key.provider.path=""* 
-Ddfs.namenode.kerberos.principal.pattern=* 
-Dmapreduce.job.hdfs-servers.token-renewal.exclude=unit 
hdfs:///processed/public/opendata/samples/distcp_test/distcp_file.txt 
hdfs://unit/processed/public/opendata/samples/distcp_test/distcp_file2.txt


> Ability to tell Hadoop not to request KMS Information from Remote NN 
> -
>
> Key: HADOOP-16350
> URL: https://issues.apache.org/jira/browse/HADOOP-16350
> Project: Hadoop Common
>  Issue Type: Improvement
>  Components: common, kms
>Affects Versions: 2.8.3, 3.0.0, 2.7.6, 3.1.2
>Reporter: Greg Senia
>Assignee: Greg Senia
>Priority: Major
> Fix For: 3.3.0
>
> Attachments: HADOOP-16350.patch
>
>
> Before HADOOP-14104 Remote KMSServer URIs were not requested from the remote 
> NameNode and their associated remote KMSServer delegation token. Many 
> customers were using this as a security feature to prevent TDE/Encryption 
> Zone data from being distcped to remote clusters. But there was still a use 
> case to allow distcp of data residing in folders that are not being encrypted 
> with a KMSProvider/Encrypted Zone.
> So after upgrading to a version of Hadoop that contained HADOOP-14104 distcp 
> now fails as we along with other customers (HDFS-13696) DO NOT allow 
> KMSServer endpoints to be exposed out of our cluster network as data residing 
> in these TDE/Zones contain very critical data that cannot be distcped between 
> clusters.
> I propose adding a new code block with the following custom property 
> "hadoop.security.kms.client.allow.remote.kms" it will default to "true" so 
> keeping current feature of HADOOP-14104 but if specified to "false" will 
> allow this area of code to operate as it did before HADOOP-14104. I can see 
> the value in HADOOP-14104 but the way Hadoop worked before this JIRA/Issue 
> should of at least had an option specified to allow Hadoop/KMS code to 
> operate similar to how it did before by not requesting remote KMSServer URIs 
> which would than attempt to get a delegation token even if not operating on 
> encrypted zones.
> Error when KMS Server traffic is not allowed between cluster networks per 
> enterprise security standard which cannot be changed they denied the request 
> for exception so the only solution is to allow a feature to not attempt to 
> request tokens. 
> {code:java}
> $ hadoop distcp -Ddfs.namenode.kerberos.principal.pattern=* 
> -Dmapreduce.job.hdfs-servers.token-renewal.exclude=tech 
> hdfs:///processed/public/opendata/samples/distcp_test/distcp_file.txt 
> hdfs://tech/processed/public/opendata/samples/distcp_test/distcp_file2.txt
> 19/05/29 14:06:09 INFO tools.DistCp: Input Options: DistCpOptions
> {atomicCommit=false, syncFolder=false, deleteMissing=false, 
> ignoreFailures=false, overwrite=false, append=false, useDiff=false, 
> fromSnapshot=null, toSnapshot=null, skipCRC=false, blocking=true, 
> numListstatusThreads=0, maxMaps=20, mapBandwidth=100, 
> sslConfigurationFile='null', copyStrategy='uniformsize', preserveStatus=[], 
> preserveRawXattrs=false, atomicWorkPath=null, logPath=null, 
> sourceFileListing=null, 
> sourcePaths=[hdfs:/processed/public/opendata/samples/distcp_test/distcp_file.txt],
>  
> targetPath=hdfs://tech/processed/public/opendata/samples/distcp_test/distcp_file2.txt,
>  targetPathExists=true, filtersFile='null', verboseLog=false}
> 19/05/29 14:06:09 INFO client.AHSProxy: Connecting to Application History 
> server at ha21d53mn.unit.hdp.example.com/10.70.49.2:10200
> 19/05/29 14:06:10 INFO hdfs.DFSClient: Created HDFS_DELEGATION_TOKEN token 
> 5093920 for gss2002 on ha-hdfs:unit
> 19/05/29 14:06:10 INFO security.TokenCache: Got dt for hdfs://unit; Kind: 
> HDFS_DELEGATION_TOKEN, Service: ha-hdfs:unit, Ident: (HDFS_DELEGATION_TOKEN 
> token 5093920 for gss2002)
> 19/05/29 14

[jira] [Commented] (HADOOP-16350) Ability to tell Hadoop not to request KMS Information from Remote NN

2019-06-12 Thread Tsz Wo Nicholas Sze (JIRA)


[ 
https://issues.apache.org/jira/browse/HADOOP-16350?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16862537#comment-16862537
 ] 

Tsz Wo Nicholas Sze commented on HADOOP-16350:
--

> ... Please provide more details on how you plan to go about making it operate 
> as it did before HADOOP-14104  ...

The idea is to let client override the server default as below.
{code}
//DFSClient.java
@@ -3057,7 +3034,12 @@ DFSHedgedReadMetrics getHedgedReadMetrics() {
   @Override
   public URI getKeyProviderUri() throws IOException {
 return HdfsKMSUtil.getKeyProviderUri(ugi, namenodeUri,
-getServerDefaults().getKeyProviderUri(), conf);
+getKeyProviderUriString(), conf);
+  }
+
+  String getKeyProviderUriString() {
+final String p = 
conf.get(CommonConfigurationKeys.HADOOP_SECURITY_KEY_PROVIDER_PATH);
+return p != null? p: serverDefaults.getKeyProviderUri();
   }
{code}

> Ability to tell Hadoop not to request KMS Information from Remote NN 
> -
>
> Key: HADOOP-16350
> URL: https://issues.apache.org/jira/browse/HADOOP-16350
> Project: Hadoop Common
>  Issue Type: Improvement
>  Components: common, kms
>Affects Versions: 2.8.3, 3.0.0, 2.7.6, 3.1.2
>Reporter: Greg Senia
>Assignee: Greg Senia
>Priority: Major
> Fix For: 3.3.0
>
> Attachments: HADOOP-16350.patch
>
>
> Before HADOOP-14104 Remote KMSServer URIs were not requested from the remote 
> NameNode and their associated remote KMSServer delegation token. Many 
> customers were using this as a security feature to prevent TDE/Encryption 
> Zone data from being distcped to remote clusters. But there was still a use 
> case to allow distcp of data residing in folders that are not being encrypted 
> with a KMSProvider/Encrypted Zone.
> So after upgrading to a version of Hadoop that contained HADOOP-14104 distcp 
> now fails as we along with other customers (HDFS-13696) DO NOT allow 
> KMSServer endpoints to be exposed out of our cluster network as data residing 
> in these TDE/Zones contain very critical data that cannot be distcped between 
> clusters.
> I propose adding a new code block with the following custom property 
> "hadoop.security.kms.client.allow.remote.kms" it will default to "true" so 
> keeping current feature of HADOOP-14104 but if specified to "false" will 
> allow this area of code to operate as it did before HADOOP-14104. I can see 
> the value in HADOOP-14104 but the way Hadoop worked before this JIRA/Issue 
> should of at least had an option specified to allow Hadoop/KMS code to 
> operate similar to how it did before by not requesting remote KMSServer URIs 
> which would than attempt to get a delegation token even if not operating on 
> encrypted zones.
> Error when KMS Server traffic is not allowed between cluster networks per 
> enterprise security standard which cannot be changed they denied the request 
> for exception so the only solution is to allow a feature to not attempt to 
> request tokens. 
> {code:java}
> $ hadoop distcp -Ddfs.namenode.kerberos.principal.pattern=* 
> -Dmapreduce.job.hdfs-servers.token-renewal.exclude=tech 
> hdfs:///processed/public/opendata/samples/distcp_test/distcp_file.txt 
> hdfs://tech/processed/public/opendata/samples/distcp_test/distcp_file2.txt
> 19/05/29 14:06:09 INFO tools.DistCp: Input Options: DistCpOptions
> {atomicCommit=false, syncFolder=false, deleteMissing=false, 
> ignoreFailures=false, overwrite=false, append=false, useDiff=false, 
> fromSnapshot=null, toSnapshot=null, skipCRC=false, blocking=true, 
> numListstatusThreads=0, maxMaps=20, mapBandwidth=100, 
> sslConfigurationFile='null', copyStrategy='uniformsize', preserveStatus=[], 
> preserveRawXattrs=false, atomicWorkPath=null, logPath=null, 
> sourceFileListing=null, 
> sourcePaths=[hdfs:/processed/public/opendata/samples/distcp_test/distcp_file.txt],
>  
> targetPath=hdfs://tech/processed/public/opendata/samples/distcp_test/distcp_file2.txt,
>  targetPathExists=true, filtersFile='null', verboseLog=false}
> 19/05/29 14:06:09 INFO client.AHSProxy: Connecting to Application History 
> server at ha21d53mn.unit.hdp.example.com/10.70.49.2:10200
> 19/05/29 14:06:10 INFO hdfs.DFSClient: Created HDFS_DELEGATION_TOKEN token 
> 5093920 for gss2002 on ha-hdfs:unit
> 19/05/29 14:06:10 INFO security.TokenCache: Got dt for hdfs://unit; Kind: 
> HDFS_DELEGATION_TOKEN, Service: ha-hdfs:unit, Ident: (HDFS_DELEGATION_TOKEN 
> token 5093920 for gss2002)
> 19/05/29 14:06:10 INFO security.TokenCache: Got dt for hdfs://unit; Kind: 
> kms-dt, Service: ha21d53en.unit.hdp.example.com:9292, Ident: (owner=gss2002, 
> renewer=yarn, realUser=, issueDate=1559153170120, maxDate=1559757970120, 
> sequenceNumber=237, masterKeyId=2)
> 19/05/29 14:06:10 INFO tools.SimpleCopyListing

[jira] [Assigned] (HADOOP-16350) Ability to tell Hadoop not to request KMS Information from Remote NN

2019-06-11 Thread Tsz Wo Nicholas Sze (JIRA)


 [ 
https://issues.apache.org/jira/browse/HADOOP-16350?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Tsz Wo Nicholas Sze reassigned HADOOP-16350:


Assignee: Greg Senia

> Ability to tell Hadoop not to request KMS Information from Remote NN 
> -
>
> Key: HADOOP-16350
> URL: https://issues.apache.org/jira/browse/HADOOP-16350
> Project: Hadoop Common
>  Issue Type: Improvement
>  Components: common, kms
>Affects Versions: 2.8.3, 3.0.0, 2.7.6, 3.1.2
>Reporter: Greg Senia
>Assignee: Greg Senia
>Priority: Major
> Fix For: 3.3.0
>
> Attachments: HADOOP-16350.patch
>
>
> Before HADOOP-14104 Remote KMSServer URIs were not requested from the remote 
> NameNode and their associated remote KMSServer delegation token. Many 
> customers were using this as a security feature to prevent TDE/Encryption 
> Zone data from being distcped to remote clusters. But there was still a use 
> case to allow distcp of data residing in folders that are not being encrypted 
> with a KMSProvider/Encrypted Zone.
> So after upgrading to a version of Hadoop that contained HADOOP-14104 distcp 
> now fails as we along with other customers (HDFS-13696) DO NOT allow 
> KMSServer endpoints to be exposed out of our cluster network as data residing 
> in these TDE/Zones contain very critical data that cannot be distcped between 
> clusters.
> I propose adding a new code block with the following custom property 
> "hadoop.security.kms.client.allow.remote.kms" it will default to "true" so 
> keeping current feature of HADOOP-14104 but if specified to "false" will 
> allow this area of code to operate as it did before HADOOP-14104. I can see 
> the value in HADOOP-14104 but the way Hadoop worked before this JIRA/Issue 
> should of at least had an option specified to allow Hadoop/KMS code to 
> operate similar to how it did before by not requesting remote KMSServer URIs 
> which would than attempt to get a delegation token even if not operating on 
> encrypted zones.
> Error when KMS Server traffic is not allowed between cluster networks per 
> enterprise security standard which cannot be changed they denied the request 
> for exception so the only solution is to allow a feature to not attempt to 
> request tokens. 
> {code:java}
> $ hadoop distcp -Ddfs.namenode.kerberos.principal.pattern=* 
> -Dmapreduce.job.hdfs-servers.token-renewal.exclude=tech 
> hdfs:///processed/public/opendata/samples/distcp_test/distcp_file.txt 
> hdfs://tech/processed/public/opendata/samples/distcp_test/distcp_file2.txt
> 19/05/29 14:06:09 INFO tools.DistCp: Input Options: DistCpOptions
> {atomicCommit=false, syncFolder=false, deleteMissing=false, 
> ignoreFailures=false, overwrite=false, append=false, useDiff=false, 
> fromSnapshot=null, toSnapshot=null, skipCRC=false, blocking=true, 
> numListstatusThreads=0, maxMaps=20, mapBandwidth=100, 
> sslConfigurationFile='null', copyStrategy='uniformsize', preserveStatus=[], 
> preserveRawXattrs=false, atomicWorkPath=null, logPath=null, 
> sourceFileListing=null, 
> sourcePaths=[hdfs:/processed/public/opendata/samples/distcp_test/distcp_file.txt],
>  
> targetPath=hdfs://tech/processed/public/opendata/samples/distcp_test/distcp_file2.txt,
>  targetPathExists=true, filtersFile='null', verboseLog=false}
> 19/05/29 14:06:09 INFO client.AHSProxy: Connecting to Application History 
> server at ha21d53mn.unit.hdp.example.com/10.70.49.2:10200
> 19/05/29 14:06:10 INFO hdfs.DFSClient: Created HDFS_DELEGATION_TOKEN token 
> 5093920 for gss2002 on ha-hdfs:unit
> 19/05/29 14:06:10 INFO security.TokenCache: Got dt for hdfs://unit; Kind: 
> HDFS_DELEGATION_TOKEN, Service: ha-hdfs:unit, Ident: (HDFS_DELEGATION_TOKEN 
> token 5093920 for gss2002)
> 19/05/29 14:06:10 INFO security.TokenCache: Got dt for hdfs://unit; Kind: 
> kms-dt, Service: ha21d53en.unit.hdp.example.com:9292, Ident: (owner=gss2002, 
> renewer=yarn, realUser=, issueDate=1559153170120, maxDate=1559757970120, 
> sequenceNumber=237, masterKeyId=2)
> 19/05/29 14:06:10 INFO tools.SimpleCopyListing: Paths (files+dirs) cnt = 1; 
> dirCnt = 0
> 19/05/29 14:06:10 INFO tools.SimpleCopyListing: Build file listing completed.
> 19/05/29 14:06:10 INFO tools.DistCp: Number of paths in the copy list: 1
> 19/05/29 14:06:10 INFO tools.DistCp: Number of paths in the copy list: 1
> 19/05/29 14:06:10 INFO client.AHSProxy: Connecting to Application History 
> server at ha21d53mn.unit.hdp.example.com/10.70.49.2:10200
> 19/05/29 14:06:10 INFO hdfs.DFSClient: Created HDFS_DELEGATION_TOKEN token 
> 556079 for gss2002 on ha-hdfs:tech
> 19/05/29 14:06:10 ERROR tools.DistCp: Exception encountered 
> java.io.IOException: java.net.NoRouteToHostException: No route to host (Host 
> unreachable)
> at 
> org.apache.hadoop.crypto.key.kms.KMSClientProvide

[jira] [Commented] (HADOOP-16350) Ability to tell Hadoop not to request KMS Information from Remote NN

2019-06-11 Thread Tsz Wo Nicholas Sze (JIRA)


[ 
https://issues.apache.org/jira/browse/HADOOP-16350?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16861658#comment-16861658
 ] 

Tsz Wo Nicholas Sze commented on HADOOP-16350:
--

[~gss2002], thanks for filing the JIRA and providing a patch!

HADOOP-14104 has changed client to always ask namenode for kms provider path.  
Instead of add a new conf, we should further change client to allow overriding 
the server conf.  I.e. client also reads the kms provider path from conf.  If 
the conf is set to empty string, don't use KMSClientProvider.

> Ability to tell Hadoop not to request KMS Information from Remote NN 
> -
>
> Key: HADOOP-16350
> URL: https://issues.apache.org/jira/browse/HADOOP-16350
> Project: Hadoop Common
>  Issue Type: Improvement
>  Components: common, kms
>Affects Versions: 2.8.3, 3.0.0, 2.7.6, 3.1.2
>Reporter: Greg Senia
>Priority: Major
> Fix For: 3.3.0
>
> Attachments: HADOOP-16350.patch
>
>
> Before HADOOP-14104 Remote KMSServer URIs were not requested from the remote 
> NameNode and their associated remote KMSServer delegation token. Many 
> customers were using this as a security feature to prevent TDE/Encryption 
> Zone data from being distcped to remote clusters. But there was still a use 
> case to allow distcp of data residing in folders that are not being encrypted 
> with a KMSProvider/Encrypted Zone.
> So after upgrading to a version of Hadoop that contained HADOOP-14104 distcp 
> now fails as we along with other customers (HDFS-13696) DO NOT allow 
> KMSServer endpoints to be exposed out of our cluster network as data residing 
> in these TDE/Zones contain very critical data that cannot be distcped between 
> clusters.
> I propose adding a new code block with the following custom property 
> "hadoop.security.kms.client.allow.remote.kms" it will default to "true" so 
> keeping current feature of HADOOP-14104 but if specified to "false" will 
> allow this area of code to operate as it did before HADOOP-14104. I can see 
> the value in HADOOP-14104 but the way Hadoop worked before this JIRA/Issue 
> should of at least had an option specified to allow Hadoop/KMS code to 
> operate similar to how it did before by not requesting remote KMSServer URIs 
> which would than attempt to get a delegation token even if not operating on 
> encrypted zones.
> Error when KMS Server traffic is not allowed between cluster networks per 
> enterprise security standard which cannot be changed they denied the request 
> for exception so the only solution is to allow a feature to not attempt to 
> request tokens. 
> {code:java}
> $ hadoop distcp -Ddfs.namenode.kerberos.principal.pattern=* 
> -Dmapreduce.job.hdfs-servers.token-renewal.exclude=tech 
> hdfs:///processed/public/opendata/samples/distcp_test/distcp_file.txt 
> hdfs://tech/processed/public/opendata/samples/distcp_test/distcp_file2.txt
> 19/05/29 14:06:09 INFO tools.DistCp: Input Options: DistCpOptions
> {atomicCommit=false, syncFolder=false, deleteMissing=false, 
> ignoreFailures=false, overwrite=false, append=false, useDiff=false, 
> fromSnapshot=null, toSnapshot=null, skipCRC=false, blocking=true, 
> numListstatusThreads=0, maxMaps=20, mapBandwidth=100, 
> sslConfigurationFile='null', copyStrategy='uniformsize', preserveStatus=[], 
> preserveRawXattrs=false, atomicWorkPath=null, logPath=null, 
> sourceFileListing=null, 
> sourcePaths=[hdfs:/processed/public/opendata/samples/distcp_test/distcp_file.txt],
>  
> targetPath=hdfs://tech/processed/public/opendata/samples/distcp_test/distcp_file2.txt,
>  targetPathExists=true, filtersFile='null', verboseLog=false}
> 19/05/29 14:06:09 INFO client.AHSProxy: Connecting to Application History 
> server at ha21d53mn.unit.hdp.example.com/10.70.49.2:10200
> 19/05/29 14:06:10 INFO hdfs.DFSClient: Created HDFS_DELEGATION_TOKEN token 
> 5093920 for gss2002 on ha-hdfs:unit
> 19/05/29 14:06:10 INFO security.TokenCache: Got dt for hdfs://unit; Kind: 
> HDFS_DELEGATION_TOKEN, Service: ha-hdfs:unit, Ident: (HDFS_DELEGATION_TOKEN 
> token 5093920 for gss2002)
> 19/05/29 14:06:10 INFO security.TokenCache: Got dt for hdfs://unit; Kind: 
> kms-dt, Service: ha21d53en.unit.hdp.example.com:9292, Ident: (owner=gss2002, 
> renewer=yarn, realUser=, issueDate=1559153170120, maxDate=1559757970120, 
> sequenceNumber=237, masterKeyId=2)
> 19/05/29 14:06:10 INFO tools.SimpleCopyListing: Paths (files+dirs) cnt = 1; 
> dirCnt = 0
> 19/05/29 14:06:10 INFO tools.SimpleCopyListing: Build file listing completed.
> 19/05/29 14:06:10 INFO tools.DistCp: Number of paths in the copy list: 1
> 19/05/29 14:06:10 INFO tools.DistCp: Number of paths in the copy list: 1
> 19/05/29 14:06:10 INFO client.AHSProxy: Connecting to Application History 
> server at ha21d53mn.unit.hdp.exa

[jira] [Commented] (HADOOP-15720) rpcTimeout may not have been applied correctly

2019-03-04 Thread Tsz Wo Nicholas Sze (JIRA)


[ 
https://issues.apache.org/jira/browse/HADOOP-15720?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16783807#comment-16783807
 ] 

Tsz Wo Nicholas Sze commented on HADOOP-15720:
--

Hi [~yzhangal], if the bug is no rpc timeout, it should be very easy to test 
the bug.  Why don't we illustrate it in a unit test?

> rpcTimeout may not have been applied correctly
> --
>
> Key: HADOOP-15720
> URL: https://issues.apache.org/jira/browse/HADOOP-15720
> Project: Hadoop Common
>  Issue Type: Bug
>  Components: common
>Reporter: Yongjun Zhang
>Priority: Major
>
> org.apache.hadoop.ipc.Client send multiple RPC calls to server synchronously 
> via the same connection as in the following synchronized code block:
> {code:java}
>   synchronized (sendRpcRequestLock) {
> Future senderFuture = sendParamsExecutor.submit(new Runnable() {
>   @Override
>   public void run() {
> try {
>   synchronized (Connection.this.out) {
> if (shouldCloseConnection.get()) {
>   return;
> }
> 
> if (LOG.isDebugEnabled()) {
>   LOG.debug(getName() + " sending #" + call.id
>   + " " + call.rpcRequest);
> }
>  
> byte[] data = d.getData();
> int totalLength = d.getLength();
> out.writeInt(totalLength); // Total Length
> out.write(data, 0, totalLength);// RpcRequestHeader + 
> RpcRequest
> out.flush();
>   }
> } catch (IOException e) {
>   // exception at this point would leave the connection in an
>   // unrecoverable state (eg half a call left on the wire).
>   // So, close the connection, killing any outstanding calls
>   markClosed(e);
> } finally {
>   //the buffer is just an in-memory buffer, but it is still 
> polite to
>   // close early
>   IOUtils.closeStream(d);
> }
>   }
> });
>   
> try {
>   senderFuture.get();
> } catch (ExecutionException e) {
>   Throwable cause = e.getCause();
>   
>   // cause should only be a RuntimeException as the Runnable above
>   // catches IOException
>   if (cause instanceof RuntimeException) {
> throw (RuntimeException) cause;
>   } else {
> throw new RuntimeException("unexpected checked exception", cause);
>   }
> }
>   }
> {code}
> And it then waits for result asynchronously via
> {code:java}
> /* Receive a response.
>  * Because only one receiver, so no synchronization on in.
>  */
> private void receiveRpcResponse() {
>   if (shouldCloseConnection.get()) {
> return;
>   }
>   touch();
>   
>   try {
> int totalLen = in.readInt();
> RpcResponseHeaderProto header = 
> RpcResponseHeaderProto.parseDelimitedFrom(in);
> checkResponse(header);
> int headerLen = header.getSerializedSize();
> headerLen += CodedOutputStream.computeRawVarint32Size(headerLen);
> int callId = header.getCallId();
> if (LOG.isDebugEnabled())
>   LOG.debug(getName() + " got value #" + callId);
> Call call = calls.get(callId);
> RpcStatusProto status = header.getStatus();
> ..
> {code}
> However, we can see that the {{call}} returned by {{receiveRpcResonse()}} 
> above may be in any order.
> The following code
> {code:java}
> int totalLen = in.readInt();
> {code}
> eventually calls one of the following two methods, where rpcTimeOut is 
> checked against:
> {code:java}
>   /** Read a byte from the stream.
>* Send a ping if timeout on read. Retries if no failure is detected
>* until a byte is read.
>* @throws IOException for any IO problem other than socket timeout
>*/
>   @Override
>   public int read() throws IOException {
> int waiting = 0;
> do {
>   try {
> return super.read();
>   } catch (SocketTimeoutException e) {
> waiting += soTimeout;
> handleTimeout(e, waiting);
>   }
> } while (true);
>   }
>   /** Read bytes into a buffer starting from offset off
>* Send a ping if timeout on read. Retries if no failure is detected
>* until a byte is read.
>* 
>* @return the total number of bytes read; -1 if the connection is 
> closed.
>*/
>   @Override
>   public int read(byte[] buf, int off, int len) throws IOException {
> int waiting = 0;
> do {

[jira] [Commented] (HADOOP-16126) ipc.Client.stop() may sleep too long to wait for all connections

2019-02-27 Thread Tsz Wo Nicholas Sze (JIRA)


[ 
https://issues.apache.org/jira/browse/HADOOP-16126?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16779887#comment-16779887
 ] 

Tsz Wo Nicholas Sze commented on HADOOP-16126:
--

Thanks [~tasanuma0829].

For this (HADOOP-16126), we may backport it to any branches since it is just a 
very small and safe change.

For HADOOP-16127, let's keep it in branch-3 for the moment to let it get 
stabilized.

> ipc.Client.stop() may sleep too long to wait for all connections
> 
>
> Key: HADOOP-16126
> URL: https://issues.apache.org/jira/browse/HADOOP-16126
> Project: Hadoop Common
>  Issue Type: Improvement
>  Components: ipc
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
>Priority: Major
> Fix For: 3.3.0
>
> Attachments: c16126_20190219.patch, c16126_20190220.patch, 
> c16126_20190221.patch
>
>
> {code}
> //Client.java
>   public void stop() {
> ...
> // wait until all connections are closed
> while (!connections.isEmpty()) {
>   try {
> Thread.sleep(100);
>   } catch (InterruptedException e) {
>   }
> }
> ...
>   }
> {code}
> In the code above, the sleep time is 100ms.  We found that simply changing 
> the sleep time to 10ms could improve a Hive job running time by 10x.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Updated] (HADOOP-16127) In ipc.Client, put a new connection could happen after stop

2019-02-26 Thread Tsz Wo Nicholas Sze (JIRA)


 [ 
https://issues.apache.org/jira/browse/HADOOP-16127?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Tsz Wo Nicholas Sze updated HADOOP-16127:
-
   Resolution: Fixed
 Hadoop Flags: Reviewed
Fix Version/s: 3.3.0
   Status: Resolved  (was: Patch Available)

Thanks [~ste...@apache.org] for reviewing the patches.

I have committed this.

> In ipc.Client, put a new connection could happen after stop
> ---
>
> Key: HADOOP-16127
> URL: https://issues.apache.org/jira/browse/HADOOP-16127
> Project: Hadoop Common
>  Issue Type: Bug
>  Components: ipc
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
>Priority: Major
> Fix For: 3.3.0
>
> Attachments: c16127_20190219.patch, c16127_20190220.patch, 
> c16127_20190225.patch
>
>
> In getConnection(..), running can be initially true but becomes false before 
> putIfAbsent.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Commented] (HADOOP-16127) In ipc.Client, put a new connection could happen after stop

2019-02-26 Thread Tsz Wo Nicholas Sze (JIRA)


[ 
https://issues.apache.org/jira/browse/HADOOP-16127?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16778681#comment-16778681
 ] 

Tsz Wo Nicholas Sze commented on HADOOP-16127:
--

Since the 25.patch only has whitespace changes compared to 20.patch for fixing 
checkstyle, I will commit it shortly.

> In ipc.Client, put a new connection could happen after stop
> ---
>
> Key: HADOOP-16127
> URL: https://issues.apache.org/jira/browse/HADOOP-16127
> Project: Hadoop Common
>  Issue Type: Bug
>  Components: ipc
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
>Priority: Major
> Attachments: c16127_20190219.patch, c16127_20190220.patch, 
> c16127_20190225.patch
>
>
> In getConnection(..), running can be initially true but becomes false before 
> putIfAbsent.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Updated] (HADOOP-16127) In ipc.Client, put a new connection could happen after stop

2019-02-25 Thread Tsz Wo Nicholas Sze (JIRA)


 [ 
https://issues.apache.org/jira/browse/HADOOP-16127?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Tsz Wo Nicholas Sze updated HADOOP-16127:
-
Attachment: c16127_20190225.patch

> In ipc.Client, put a new connection could happen after stop
> ---
>
> Key: HADOOP-16127
> URL: https://issues.apache.org/jira/browse/HADOOP-16127
> Project: Hadoop Common
>  Issue Type: Bug
>  Components: ipc
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
>Priority: Major
> Attachments: c16127_20190219.patch, c16127_20190220.patch, 
> c16127_20190225.patch
>
>
> In getConnection(..), running can be initially true but becomes false before 
> putIfAbsent.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Updated] (HADOOP-16127) In ipc.Client, put a new connection could happen after stop

2019-02-25 Thread Tsz Wo Nicholas Sze (JIRA)


 [ 
https://issues.apache.org/jira/browse/HADOOP-16127?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Tsz Wo Nicholas Sze updated HADOOP-16127:
-
Attachment: (was: c16127_20190225.patch)

> In ipc.Client, put a new connection could happen after stop
> ---
>
> Key: HADOOP-16127
> URL: https://issues.apache.org/jira/browse/HADOOP-16127
> Project: Hadoop Common
>  Issue Type: Bug
>  Components: ipc
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
>Priority: Major
> Attachments: c16127_20190219.patch, c16127_20190220.patch, 
> c16127_20190225.patch
>
>
> In getConnection(..), running can be initially true but becomes false before 
> putIfAbsent.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Commented] (HADOOP-16127) In ipc.Client, put a new connection could happen after stop

2019-02-25 Thread Tsz Wo Nicholas Sze (JIRA)


[ 
https://issues.apache.org/jira/browse/HADOOP-16127?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16777301#comment-16777301
 ] 

Tsz Wo Nicholas Sze commented on HADOOP-16127:
--

c16127_20190225.patch: fixes checkstyle warning.

> In ipc.Client, put a new connection could happen after stop
> ---
>
> Key: HADOOP-16127
> URL: https://issues.apache.org/jira/browse/HADOOP-16127
> Project: Hadoop Common
>  Issue Type: Bug
>  Components: ipc
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
>Priority: Major
> Attachments: c16127_20190219.patch, c16127_20190220.patch, 
> c16127_20190225.patch
>
>
> In getConnection(..), running can be initially true but becomes false before 
> putIfAbsent.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Updated] (HADOOP-16127) In ipc.Client, put a new connection could happen after stop

2019-02-25 Thread Tsz Wo Nicholas Sze (JIRA)


 [ 
https://issues.apache.org/jira/browse/HADOOP-16127?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Tsz Wo Nicholas Sze updated HADOOP-16127:
-
Attachment: c16127_20190225.patch

> In ipc.Client, put a new connection could happen after stop
> ---
>
> Key: HADOOP-16127
> URL: https://issues.apache.org/jira/browse/HADOOP-16127
> Project: Hadoop Common
>  Issue Type: Bug
>  Components: ipc
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
>Priority: Major
> Attachments: c16127_20190219.patch, c16127_20190220.patch, 
> c16127_20190225.patch
>
>
> In getConnection(..), running can be initially true but becomes false before 
> putIfAbsent.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Updated] (HADOOP-16127) In ipc.Client, put a new connection could happen after stop

2019-02-25 Thread Tsz Wo Nicholas Sze (JIRA)


 [ 
https://issues.apache.org/jira/browse/HADOOP-16127?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Tsz Wo Nicholas Sze updated HADOOP-16127:
-
Target Version/s: 3.3.0  (was: 3.1.2)

> Just have set target version to 3.1.2. Thanks.

Oops, it should be 3.3.0, the newest unreleased version.

> In ipc.Client, put a new connection could happen after stop
> ---
>
> Key: HADOOP-16127
> URL: https://issues.apache.org/jira/browse/HADOOP-16127
> Project: Hadoop Common
>  Issue Type: Bug
>  Components: ipc
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
>Priority: Major
> Attachments: c16127_20190219.patch, c16127_20190220.patch
>
>
> In getConnection(..), running can be initially true but becomes false before 
> putIfAbsent.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Updated] (HADOOP-16126) ipc.Client.stop() may sleep too long to wait for all connections

2019-02-25 Thread Tsz Wo Nicholas Sze (JIRA)


 [ 
https://issues.apache.org/jira/browse/HADOOP-16126?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Tsz Wo Nicholas Sze updated HADOOP-16126:
-
   Resolution: Fixed
Fix Version/s: 3.3.0
   Status: Resolved  (was: Patch Available)

Thanks [~arpitagarwal] for the review and [~ste...@apache.org] for the comments.

I have committed this.

> ipc.Client.stop() may sleep too long to wait for all connections
> 
>
> Key: HADOOP-16126
> URL: https://issues.apache.org/jira/browse/HADOOP-16126
> Project: Hadoop Common
>  Issue Type: Improvement
>  Components: ipc
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
>Priority: Major
> Fix For: 3.3.0
>
> Attachments: c16126_20190219.patch, c16126_20190220.patch, 
> c16126_20190221.patch
>
>
> {code}
> //Client.java
>   public void stop() {
> ...
> // wait until all connections are closed
> while (!connections.isEmpty()) {
>   try {
> Thread.sleep(100);
>   } catch (InterruptedException e) {
>   }
> }
> ...
>   }
> {code}
> In the code above, the sleep time is 100ms.  We found that simply changing 
> the sleep time to 10ms could improve a Hive job running time by 10x.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Commented] (HADOOP-16127) In ipc.Client, put a new connection could happen after stop

2019-02-21 Thread Tsz Wo Nicholas Sze (JIRA)


[ 
https://issues.apache.org/jira/browse/HADOOP-16127?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16774614#comment-16774614
 ] 

Tsz Wo Nicholas Sze commented on HADOOP-16127:
--

[~ste...@apache.org], thanks for reviewing this.

Could you also review HADOOP-16126?  We should commit that JIRA before this.

> In ipc.Client, put a new connection could happen after stop
> ---
>
> Key: HADOOP-16127
> URL: https://issues.apache.org/jira/browse/HADOOP-16127
> Project: Hadoop Common
>  Issue Type: Bug
>  Components: ipc
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
>Priority: Major
> Attachments: c16127_20190219.patch, c16127_20190220.patch
>
>
> In getConnection(..), running can be initially true but becomes false before 
> putIfAbsent.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Updated] (HADOOP-16126) ipc.Client.stop() may sleep too long to wait for all connections

2019-02-21 Thread Tsz Wo Nicholas Sze (JIRA)


 [ 
https://issues.apache.org/jira/browse/HADOOP-16126?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Tsz Wo Nicholas Sze updated HADOOP-16126:
-
Attachment: c16126_20190221.patch

> ipc.Client.stop() may sleep too long to wait for all connections
> 
>
> Key: HADOOP-16126
> URL: https://issues.apache.org/jira/browse/HADOOP-16126
> Project: Hadoop Common
>  Issue Type: Improvement
>  Components: ipc
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
>Priority: Major
> Attachments: c16126_20190219.patch, c16126_20190220.patch, 
> c16126_20190221.patch
>
>
> {code}
> //Client.java
>   public void stop() {
> ...
> // wait until all connections are closed
> while (!connections.isEmpty()) {
>   try {
> Thread.sleep(100);
>   } catch (InterruptedException e) {
>   }
> }
> ...
>   }
> {code}
> In the code above, the sleep time is 100ms.  We found that simply changing 
> the sleep time to 10ms could improve a Hive job running time by 10x.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Commented] (HADOOP-16127) In ipc.Client, put a new connection could happen after stop

2019-02-20 Thread Tsz Wo Nicholas Sze (JIRA)


[ 
https://issues.apache.org/jira/browse/HADOOP-16127?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16773522#comment-16773522
 ] 

Tsz Wo Nicholas Sze commented on HADOOP-16127:
--

> L475: include the toString value of the caught IOE in the new one, so if the 
> full stack trace is lost, the root cause is preserved

Sure.

> L1360: you've removed all sleeps here entirely. Is that OK?

Yes, we use wait-notify instead of sleep to minimize the sleep/wait time.

Thanks for the review.  Here is a new patch: c16127_20190220.patch 

> In ipc.Client, put a new connection could happen after stop
> ---
>
> Key: HADOOP-16127
> URL: https://issues.apache.org/jira/browse/HADOOP-16127
> Project: Hadoop Common
>  Issue Type: Bug
>  Components: ipc
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
>Priority: Major
> Attachments: c16127_20190219.patch, c16127_20190220.patch
>
>
> In getConnection(..), running can be initially true but becomes false before 
> putIfAbsent.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Commented] (HADOOP-16126) ipc.Client.stop() may sleep too long to wait for all connections

2019-02-20 Thread Tsz Wo Nicholas Sze (JIRA)


[ 
https://issues.apache.org/jira/browse/HADOOP-16126?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16773525#comment-16773525
 ] 

Tsz Wo Nicholas Sze commented on HADOOP-16126:
--

c16126_20190220.patch: address [~ste...@apache.org]'s comments.

> ipc.Client.stop() may sleep too long to wait for all connections
> 
>
> Key: HADOOP-16126
> URL: https://issues.apache.org/jira/browse/HADOOP-16126
> Project: Hadoop Common
>  Issue Type: Improvement
>  Components: ipc
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
>Priority: Major
> Attachments: c16126_20190219.patch, c16126_20190220.patch
>
>
> {code}
> //Client.java
>   public void stop() {
> ...
> // wait until all connections are closed
> while (!connections.isEmpty()) {
>   try {
> Thread.sleep(100);
>   } catch (InterruptedException e) {
>   }
> }
> ...
>   }
> {code}
> In the code above, the sleep time is 100ms.  We found that simply changing 
> the sleep time to 10ms could improve a Hive job running time by 10x.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Commented] (HADOOP-16127) In ipc.Client, put a new connection could happen after stop

2019-02-20 Thread Tsz Wo Nicholas Sze (JIRA)


[ 
https://issues.apache.org/jira/browse/HADOOP-16127?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16773526#comment-16773526
 ] 

Tsz Wo Nicholas Sze commented on HADOOP-16127:
--

> ps: set your version info for where you intend to apply this

Just have set target version to 3.1.2.  Thanks.

> In ipc.Client, put a new connection could happen after stop
> ---
>
> Key: HADOOP-16127
> URL: https://issues.apache.org/jira/browse/HADOOP-16127
> Project: Hadoop Common
>  Issue Type: Bug
>  Components: ipc
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
>Priority: Major
> Attachments: c16127_20190219.patch, c16127_20190220.patch
>
>
> In getConnection(..), running can be initially true but becomes false before 
> putIfAbsent.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Updated] (HADOOP-16127) In ipc.Client, put a new connection could happen after stop

2019-02-20 Thread Tsz Wo Nicholas Sze (JIRA)


 [ 
https://issues.apache.org/jira/browse/HADOOP-16127?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Tsz Wo Nicholas Sze updated HADOOP-16127:
-
Target Version/s: 3.1.2

> In ipc.Client, put a new connection could happen after stop
> ---
>
> Key: HADOOP-16127
> URL: https://issues.apache.org/jira/browse/HADOOP-16127
> Project: Hadoop Common
>  Issue Type: Bug
>  Components: ipc
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
>Priority: Major
> Attachments: c16127_20190219.patch, c16127_20190220.patch
>
>
> In getConnection(..), running can be initially true but becomes false before 
> putIfAbsent.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Updated] (HADOOP-16126) ipc.Client.stop() may sleep too long to wait for all connections

2019-02-20 Thread Tsz Wo Nicholas Sze (JIRA)


 [ 
https://issues.apache.org/jira/browse/HADOOP-16126?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Tsz Wo Nicholas Sze updated HADOOP-16126:
-
Attachment: c16126_20190220.patch

> ipc.Client.stop() may sleep too long to wait for all connections
> 
>
> Key: HADOOP-16126
> URL: https://issues.apache.org/jira/browse/HADOOP-16126
> Project: Hadoop Common
>  Issue Type: Improvement
>  Components: ipc
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
>Priority: Major
> Attachments: c16126_20190219.patch, c16126_20190220.patch
>
>
> {code}
> //Client.java
>   public void stop() {
> ...
> // wait until all connections are closed
> while (!connections.isEmpty()) {
>   try {
> Thread.sleep(100);
>   } catch (InterruptedException e) {
>   }
> }
> ...
>   }
> {code}
> In the code above, the sleep time is 100ms.  We found that simply changing 
> the sleep time to 10ms could improve a Hive job running time by 10x.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Updated] (HADOOP-16127) In ipc.Client, put a new connection could happen after stop

2019-02-20 Thread Tsz Wo Nicholas Sze (JIRA)


 [ 
https://issues.apache.org/jira/browse/HADOOP-16127?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Tsz Wo Nicholas Sze updated HADOOP-16127:
-
Attachment: c16127_20190220.patch

> In ipc.Client, put a new connection could happen after stop
> ---
>
> Key: HADOOP-16127
> URL: https://issues.apache.org/jira/browse/HADOOP-16127
> Project: Hadoop Common
>  Issue Type: Bug
>  Components: ipc
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
>Priority: Major
> Attachments: c16127_20190219.patch, c16127_20190220.patch
>
>
> In getConnection(..), running can be initially true but becomes false before 
> putIfAbsent.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Updated] (HADOOP-16127) In ipc.Client, put a new connection could happen after stop

2019-02-20 Thread Tsz Wo Nicholas Sze (JIRA)


 [ 
https://issues.apache.org/jira/browse/HADOOP-16127?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Tsz Wo Nicholas Sze updated HADOOP-16127:
-
Attachment: (was: c16127_20190220.patch)

> In ipc.Client, put a new connection could happen after stop
> ---
>
> Key: HADOOP-16127
> URL: https://issues.apache.org/jira/browse/HADOOP-16127
> Project: Hadoop Common
>  Issue Type: Bug
>  Components: ipc
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
>Priority: Major
> Attachments: c16127_20190219.patch, c16127_20190220.patch
>
>
> In getConnection(..), running can be initially true but becomes false before 
> putIfAbsent.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Updated] (HADOOP-16127) In ipc.Client, put a new connection could happen after stop

2019-02-20 Thread Tsz Wo Nicholas Sze (JIRA)


 [ 
https://issues.apache.org/jira/browse/HADOOP-16127?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Tsz Wo Nicholas Sze updated HADOOP-16127:
-
Attachment: c16127_20190220.patch

> In ipc.Client, put a new connection could happen after stop
> ---
>
> Key: HADOOP-16127
> URL: https://issues.apache.org/jira/browse/HADOOP-16127
> Project: Hadoop Common
>  Issue Type: Bug
>  Components: ipc
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
>Priority: Major
> Attachments: c16127_20190219.patch, c16127_20190220.patch
>
>
> In getConnection(..), running can be initially true but becomes false before 
> putIfAbsent.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Commented] (HADOOP-16126) ipc.Client.stop() may sleep too long to wait for all connections

2019-02-20 Thread Tsz Wo Nicholas Sze (JIRA)


[ 
https://issues.apache.org/jira/browse/HADOOP-16126?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16773424#comment-16773424
 ] 

Tsz Wo Nicholas Sze commented on HADOOP-16126:
--

> Why the choice of 10ms?

It is because 10ms works well in the test.  10ms is a long time in modern 
computers.  So, it is still far away from busy waiting.

As you already aware, we have HADOOP-16127 for a better (but more complicated) 
fix.  The patch here is a safe, short term fix for the clusters which do not 
welcome big changes.

> Can you tease this out as a private constant in the IPC file, just so it's 
> less hidden deep in the code.

Will do.

> ipc.Client.stop() may sleep too long to wait for all connections
> 
>
> Key: HADOOP-16126
> URL: https://issues.apache.org/jira/browse/HADOOP-16126
> Project: Hadoop Common
>  Issue Type: Improvement
>  Components: ipc
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
>Priority: Major
> Attachments: c16126_20190219.patch
>
>
> {code}
> //Client.java
>   public void stop() {
> ...
> // wait until all connections are closed
> while (!connections.isEmpty()) {
>   try {
> Thread.sleep(100);
>   } catch (InterruptedException e) {
>   }
> }
> ...
>   }
> {code}
> In the code above, the sleep time is 100ms.  We found that simply changing 
> the sleep time to 10ms could improve a Hive job running time by 10x.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Updated] (HADOOP-16127) In ipc.Client, put a new connection could happen after stop

2019-02-19 Thread Tsz Wo Nicholas Sze (JIRA)


 [ 
https://issues.apache.org/jira/browse/HADOOP-16127?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Tsz Wo Nicholas Sze updated HADOOP-16127:
-
Status: Patch Available  (was: Open)

> In ipc.Client, put a new connection could happen after stop
> ---
>
> Key: HADOOP-16127
> URL: https://issues.apache.org/jira/browse/HADOOP-16127
> Project: Hadoop Common
>  Issue Type: Bug
>  Components: ipc
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
>Priority: Major
> Attachments: c16127_20190219.patch
>
>
> In getConnection(..), running can be initially true but becomes false before 
> putIfAbsent.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Updated] (HADOOP-16127) In ipc.Client, put a new connection could happen after stop

2019-02-19 Thread Tsz Wo Nicholas Sze (JIRA)


 [ 
https://issues.apache.org/jira/browse/HADOOP-16127?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Tsz Wo Nicholas Sze updated HADOOP-16127:
-
Attachment: c16127_20190219.patch

> In ipc.Client, put a new connection could happen after stop
> ---
>
> Key: HADOOP-16127
> URL: https://issues.apache.org/jira/browse/HADOOP-16127
> Project: Hadoop Common
>  Issue Type: Bug
>  Components: ipc
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
>Priority: Major
> Attachments: c16127_20190219.patch
>
>
> In getConnection(..), running can be initially true but becomes false before 
> putIfAbsent.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Commented] (HADOOP-16126) ipc.Client.stop() may sleep too long to wait for all connections

2019-02-19 Thread Tsz Wo Nicholas Sze (JIRA)


[ 
https://issues.apache.org/jira/browse/HADOOP-16126?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16772465#comment-16772465
 ] 

Tsz Wo Nicholas Sze commented on HADOOP-16126:
--

Filed HADOOP-16127.

> ipc.Client.stop() may sleep too long to wait for all connections
> 
>
> Key: HADOOP-16126
> URL: https://issues.apache.org/jira/browse/HADOOP-16126
> Project: Hadoop Common
>  Issue Type: Improvement
>  Components: ipc
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
>Priority: Major
> Attachments: c16126_20190219.patch
>
>
> {code}
> //Client.java
>   public void stop() {
> ...
> // wait until all connections are closed
> while (!connections.isEmpty()) {
>   try {
> Thread.sleep(100);
>   } catch (InterruptedException e) {
>   }
> }
> ...
>   }
> {code}
> In the code above, the sleep time is 100ms.  We found that simply changing 
> the sleep time to 10ms could improve a Hive job running time by 10x.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Created] (HADOOP-16127) In ipc.Client, put a new connection could happen after stop

2019-02-19 Thread Tsz Wo Nicholas Sze (JIRA)
Tsz Wo Nicholas Sze created HADOOP-16127:


 Summary: In ipc.Client, put a new connection could happen after 
stop
 Key: HADOOP-16127
 URL: https://issues.apache.org/jira/browse/HADOOP-16127
 Project: Hadoop Common
  Issue Type: Bug
  Components: ipc
Reporter: Tsz Wo Nicholas Sze
Assignee: Tsz Wo Nicholas Sze


In getConnection(..), running can be initially true but becomes false before 
putIfAbsent.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Updated] (HADOOP-16126) ipc.Client.stop() may sleep too long to wait for all connections

2019-02-19 Thread Tsz Wo Nicholas Sze (JIRA)


 [ 
https://issues.apache.org/jira/browse/HADOOP-16126?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Tsz Wo Nicholas Sze updated HADOOP-16126:
-
Status: Patch Available  (was: Open)

> ipc.Client.stop() may sleep too long to wait for all connections
> 
>
> Key: HADOOP-16126
> URL: https://issues.apache.org/jira/browse/HADOOP-16126
> Project: Hadoop Common
>  Issue Type: Improvement
>  Components: ipc
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
>Priority: Major
> Attachments: c16126_20190219.patch
>
>
> {code}
> //Client.java
>   public void stop() {
> ...
> // wait until all connections are closed
> while (!connections.isEmpty()) {
>   try {
> Thread.sleep(100);
>   } catch (InterruptedException e) {
>   }
> }
> ...
>   }
> {code}
> In the code above, the sleep time is 100ms.  We found that simply changing 
> the sleep time to 10ms could improve a Hive job running time by 10x.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Updated] (HADOOP-16126) ipc.Client.stop() may sleep too long to wait for all connections

2019-02-19 Thread Tsz Wo Nicholas Sze (JIRA)


 [ 
https://issues.apache.org/jira/browse/HADOOP-16126?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Tsz Wo Nicholas Sze updated HADOOP-16126:
-
Attachment: c16126_20190219.patch

> ipc.Client.stop() may sleep too long to wait for all connections
> 
>
> Key: HADOOP-16126
> URL: https://issues.apache.org/jira/browse/HADOOP-16126
> Project: Hadoop Common
>  Issue Type: Improvement
>  Components: ipc
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
>Priority: Major
> Attachments: c16126_20190219.patch
>
>
> {code}
> //Client.java
>   public void stop() {
> ...
> // wait until all connections are closed
> while (!connections.isEmpty()) {
>   try {
> Thread.sleep(100);
>   } catch (InterruptedException e) {
>   }
> }
> ...
>   }
> {code}
> In the code above, the sleep time is 100ms.  We found that simply changing 
> the sleep time to 10ms could improve a Hive job running time by 10x.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Commented] (HADOOP-16126) ipc.Client.stop() may sleep too long to wait for all connections

2019-02-19 Thread Tsz Wo Nicholas Sze (JIRA)


[ 
https://issues.apache.org/jira/browse/HADOOP-16126?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16772452#comment-16772452
 ] 

Tsz Wo Nicholas Sze commented on HADOOP-16126:
--

Tried to change the sleep to wait-notify.  However, found some race conditions 
such as
- put a new connection could happen after stop.
- stop can be called twice.

Therefore, we will just change the sleep time here and then fix the race 
conditions and change to wait-notify in a separated JIRA.

> ipc.Client.stop() may sleep too long to wait for all connections
> 
>
> Key: HADOOP-16126
> URL: https://issues.apache.org/jira/browse/HADOOP-16126
> Project: Hadoop Common
>  Issue Type: Improvement
>  Components: ipc
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
>Priority: Major
>
> {code}
> //Client.java
>   public void stop() {
> ...
> // wait until all connections are closed
> while (!connections.isEmpty()) {
>   try {
> Thread.sleep(100);
>   } catch (InterruptedException e) {
>   }
> }
> ...
>   }
> {code}
> In the code above, the sleep time is 100ms.  We found that simply changing 
> the sleep time to 10ms could improve a Hive job running time by 10x.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Created] (HADOOP-16126) ipc.Client.stop() may sleep too long to wait for all connections

2019-02-19 Thread Tsz Wo Nicholas Sze (JIRA)
Tsz Wo Nicholas Sze created HADOOP-16126:


 Summary: ipc.Client.stop() may sleep too long to wait for all 
connections
 Key: HADOOP-16126
 URL: https://issues.apache.org/jira/browse/HADOOP-16126
 Project: Hadoop Common
  Issue Type: Improvement
  Components: ipc
Reporter: Tsz Wo Nicholas Sze
Assignee: Tsz Wo Nicholas Sze


{code}
//Client.java
  public void stop() {
...
// wait until all connections are closed
while (!connections.isEmpty()) {
  try {
Thread.sleep(100);
  } catch (InterruptedException e) {
  }
}
...
  }
{code}
In the code above, the sleep time is 100ms.  We found that simply changing the 
sleep time to 10ms could improve a Hive job running time by 10x.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Commented] (HADOOP-15857) Remove ozonefs class name definition from core-default.xml

2018-10-16 Thread Tsz Wo Nicholas Sze (JIRA)


[ 
https://issues.apache.org/jira/browse/HADOOP-15857?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16651516#comment-16651516
 ] 

Tsz Wo Nicholas Sze commented on HADOOP-15857:
--

+1 the patch looks good.

> Remove ozonefs class name definition from core-default.xml
> --
>
> Key: HADOOP-15857
> URL: https://issues.apache.org/jira/browse/HADOOP-15857
> Project: Hadoop Common
>  Issue Type: Bug
>  Components: fs
>Reporter: Elek, Marton
>Assignee: Elek, Marton
>Priority: Blocker
> Attachments: HADOOP-15857-branch-3.2.001.patch
>
>
> Ozone file system is under renaming in HDDS-651 from o3:// to o3fs://. But 
> branch-3.2 still contains a reference with o3://.
> The easiest way to fix it just remove the fs.o3.imp definition from 
> core-default.xml from branch-3.2 as since HDDS-654 the file system could be 
> registered with Service Provider Interface (META-INF/services...)



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Commented] (HADOOP-15725) FileSystem.deleteOnExit should check user permissions

2018-09-10 Thread Tsz Wo Nicholas Sze (JIRA)


[ 
https://issues.apache.org/jira/browse/HADOOP-15725?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16609663#comment-16609663
 ] 

Tsz Wo Nicholas Sze commented on HADOOP-15725:
--

[~oshevchenko], thanks for testing it.  We also need to get a new FileSystem 
inside doAs(..) since FileSystem.get(..) perform the authentication.  (Just 
like that if we open a file and pass the OutputStream to another user.  That 
user can read the file regardless of permission.)

> FileSystem.deleteOnExit should check user permissions
> -
>
> Key: HADOOP-15725
> URL: https://issues.apache.org/jira/browse/HADOOP-15725
> Project: Hadoop Common
>  Issue Type: Bug
>Reporter: Oleksandr Shevchenko
>Priority: Major
>  Labels: Security
> Attachments: deleteOnExitReproduce
>
>
> For now, we able to add any file to FileSystem deleteOnExit list. It leads to 
> security problems. Some user (Intruder) can get file system instance which 
> was created by another user (Owner) and mark any files to delete even if 
> "Intruder" doesn't have any access to this files. Later when "Owner" invoke 
> close method (or JVM is shut down since we have ShutdownHook which able to 
> close all file systems) marked files will be deleted successfully since 
> deleting was do behalf of "Owner" (or behalf of a user who ran a program).
> I attached the patch [^deleteOnExitReproduce] which reproduces this 
> possibility and also I able to reproduce it on a cluster with both Local and 
> Distributed file systems:
> {code:java}
> public class Main {
> public static void main(String[] args) throws Exception {
> final FileSystem fs;
>  Configuration conf = new Configuration();
>  conf.set("fs.default.name", "hdfs://node:9000");
>  conf.set("fs.hdfs.impl",
>  org.apache.hadoop.hdfs.DistributedFileSystem.class.getName()
>  );
>  fs = FileSystem.get(conf);
>  System.out.println(fs);
> Path f = new Path("/user/root/testfile");
>  System.out.println(f);
> UserGroupInformation hive = UserGroupInformation.createRemoteUser("hive");
> hive.doAs((PrivilegedExceptionAction) () -> fs.deleteOnExit(f));
> fs.close();
>  }
> {code}
> Result:
> {noformat}
> root@node:/# hadoop fs -put testfile /user/root
> root@node:/# hadoop fs -chmod 700 /user/root/testfile
> root@node:/# hadoop fs -ls /user/root
> Found 1 items
> -rw--- 1 root supergroup 0 2018-09-06 18:07 /user/root/testfile
> root@node:/# java -jar testDeleteOther.jar 
> log4j:WARN No appenders could be found for logger 
> (org.apache.hadoop.conf.Configuration.deprecation).
> log4j:WARN Please initialize the log4j system properly.
> log4j:WARN See http://logging.apache.org/log4j/1.2/faq.html#noconfig for more 
> info.
> DFS[DFSClient[clientName=DFSClient_NONMAPREDUCE_309539034_1, ugi=root 
> (auth:SIMPLE)]]
> /user/root/testfile
> []
> root@node:/# hadoop fs -ls /user/root
> root@node:/# 
> {noformat}
> We should add a check user permissions before mark a file to delete. 
>  Could someone evaluate this? And if no one objects I would like to start 
> working on this.
>  Thanks a lot for any comments.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Commented] (HADOOP-15725) FileSystem.deleteOnExit should check user permissions

2018-09-10 Thread Tsz Wo Nicholas Sze (JIRA)


[ 
https://issues.apache.org/jira/browse/HADOOP-15725?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16609656#comment-16609656
 ] 

Tsz Wo Nicholas Sze commented on HADOOP-15725:
--

[~vinayrpet] is right.  We should call FileSystem.get(..) inside doAs(..) .  
doAs(..) changes the current user but it won't change the cached user in the 
FileSystem object.

> FileSystem.deleteOnExit should check user permissions
> -
>
> Key: HADOOP-15725
> URL: https://issues.apache.org/jira/browse/HADOOP-15725
> Project: Hadoop Common
>  Issue Type: Bug
>Reporter: Oleksandr Shevchenko
>Priority: Major
>  Labels: Security
> Attachments: deleteOnExitReproduce
>
>
> For now, we able to add any file to FileSystem deleteOnExit list. It leads to 
> security problems. Some user (Intruder) can get file system instance which 
> was created by another user (Owner) and mark any files to delete even if 
> "Intruder" doesn't have any access to this files. Later when "Owner" invoke 
> close method (or JVM is shut down since we have ShutdownHook which able to 
> close all file systems) marked files will be deleted successfully since 
> deleting was do behalf of "Owner" (or behalf of a user who ran a program).
> I attached the patch [^deleteOnExitReproduce] which reproduces this 
> possibility and also I able to reproduce it on a cluster with both Local and 
> Distributed file systems:
> {code:java}
> public class Main {
> public static void main(String[] args) throws Exception {
> final FileSystem fs;
>  Configuration conf = new Configuration();
>  conf.set("fs.default.name", "hdfs://node:9000");
>  conf.set("fs.hdfs.impl",
>  org.apache.hadoop.hdfs.DistributedFileSystem.class.getName()
>  );
>  fs = FileSystem.get(conf);
>  System.out.println(fs);
> Path f = new Path("/user/root/testfile");
>  System.out.println(f);
> UserGroupInformation hive = UserGroupInformation.createRemoteUser("hive");
> hive.doAs((PrivilegedExceptionAction) () -> fs.deleteOnExit(f));
> fs.close();
>  }
> {code}
> Result:
> {noformat}
> root@node:/# hadoop fs -put testfile /user/root
> root@node:/# hadoop fs -chmod 700 /user/root/testfile
> root@node:/# hadoop fs -ls /user/root
> Found 1 items
> -rw--- 1 root supergroup 0 2018-09-06 18:07 /user/root/testfile
> root@node:/# java -jar testDeleteOther.jar 
> log4j:WARN No appenders could be found for logger 
> (org.apache.hadoop.conf.Configuration.deprecation).
> log4j:WARN Please initialize the log4j system properly.
> log4j:WARN See http://logging.apache.org/log4j/1.2/faq.html#noconfig for more 
> info.
> DFS[DFSClient[clientName=DFSClient_NONMAPREDUCE_309539034_1, ugi=root 
> (auth:SIMPLE)]]
> /user/root/testfile
> []
> root@node:/# hadoop fs -ls /user/root
> root@node:/# 
> {noformat}
> We should add a check user permissions before mark a file to delete. 
>  Could someone evaluate this? And if no one objects I would like to start 
> working on this.
>  Thanks a lot for any comments.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Commented] (HADOOP-15725) FileSystem.deleteOnExit should check user permissions

2018-09-10 Thread Tsz Wo Nicholas Sze (JIRA)


[ 
https://issues.apache.org/jira/browse/HADOOP-15725?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16609551#comment-16609551
 ] 

Tsz Wo Nicholas Sze commented on HADOOP-15725:
--

bq. root@node:/# hadoop fs -chmod 700 /user/root/testfile
Wrong command?  It only changes testfile permission.  We need to change the 
directory permission in order to disallow deletion.

[~oshevchenko], could you try to replace it with the command below?
{code}
hadoop fs -chmod 700 /user/root/
{code}


> FileSystem.deleteOnExit should check user permissions
> -
>
> Key: HADOOP-15725
> URL: https://issues.apache.org/jira/browse/HADOOP-15725
> Project: Hadoop Common
>  Issue Type: Bug
>Reporter: Oleksandr Shevchenko
>Priority: Major
>  Labels: Security
> Attachments: deleteOnExitReproduce
>
>
> For now, we able to add any file to FileSystem deleteOnExit list. It leads to 
> security problems. Some user (Intruder) can get file system instance which 
> was created by another user (Owner) and mark any files to delete even if 
> "Intruder" doesn't have any access to this files. Later when "Owner" invoke 
> close method (or JVM is shut down since we have ShutdownHook which able to 
> close all file systems) marked files will be deleted successfully since 
> deleting was do behalf of "Owner" (or behalf of a user who ran a program).
> I attached the patch [^deleteOnExitReproduce] which reproduces this 
> possibility and also I able to reproduce it on a cluster with both Local and 
> Distributed file systems:
> {code:java}
> public class Main {
> public static void main(String[] args) throws Exception {
> final FileSystem fs;
>  Configuration conf = new Configuration();
>  conf.set("fs.default.name", "hdfs://node:9000");
>  conf.set("fs.hdfs.impl",
>  org.apache.hadoop.hdfs.DistributedFileSystem.class.getName()
>  );
>  fs = FileSystem.get(conf);
>  System.out.println(fs);
> Path f = new Path("/user/root/testfile");
>  System.out.println(f);
> UserGroupInformation hive = UserGroupInformation.createRemoteUser("hive");
> hive.doAs((PrivilegedExceptionAction) () -> fs.deleteOnExit(f));
> fs.close();
>  }
> {code}
> Result:
> {noformat}
> root@node:/# hadoop fs -put testfile /user/root
> root@node:/# hadoop fs -chmod 700 /user/root/testfile
> root@node:/# hadoop fs -ls /user/root
> Found 1 items
> -rw--- 1 root supergroup 0 2018-09-06 18:07 /user/root/testfile
> root@node:/# java -jar testDeleteOther.jar 
> log4j:WARN No appenders could be found for logger 
> (org.apache.hadoop.conf.Configuration.deprecation).
> log4j:WARN Please initialize the log4j system properly.
> log4j:WARN See http://logging.apache.org/log4j/1.2/faq.html#noconfig for more 
> info.
> DFS[DFSClient[clientName=DFSClient_NONMAPREDUCE_309539034_1, ugi=root 
> (auth:SIMPLE)]]
> /user/root/testfile
> []
> root@node:/# hadoop fs -ls /user/root
> root@node:/# 
> {noformat}
> We should add a check user permissions before mark a file to delete. 
>  Could someone evaluate this? And if no one objects I would like to start 
> working on this.
>  Thanks a lot for any comments.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Commented] (HADOOP-14854) DistCp should not issue file status calls for files in the filter list

2018-07-18 Thread Tsz Wo Nicholas Sze (JIRA)


[ 
https://issues.apache.org/jira/browse/HADOOP-14854?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16548264#comment-16548264
 ] 

Tsz Wo Nicholas Sze commented on HADOOP-14854:
--

The shouldCopy(..) should be invoked before sourceFS.getFileStatus(..) so that 
it can avoid the RPC.

> DistCp should not issue file status calls for files in the filter list
> --
>
> Key: HADOOP-14854
> URL: https://issues.apache.org/jira/browse/HADOOP-14854
> Project: Hadoop Common
>  Issue Type: Improvement
>  Components: tools/distcp
>Affects Versions: 2.8.1
>Reporter: Mukul Kumar Singh
>Assignee: Mukul Kumar Singh
>Priority: Minor
> Attachments: HADOOP-14854.001.patch, HADOOP-14854.002.patch
>
>
> DistCp currently excludes the files in the filter list only when the files 
> are added to the copy list.
> However distcp can be optimized by not issuing file status/get attr calls for 
> the files in the filter.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Commented] (HADOOP-14453) Split the maven modules into several profiles

2018-03-06 Thread Tsz Wo Nicholas Sze (JIRA)

[ 
https://issues.apache.org/jira/browse/HADOOP-14453?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16388619#comment-16388619
 ] 

Tsz Wo Nicholas Sze commented on HADOOP-14453:
--

[~ste...@apache.org], your command works with minor modification.  Thanks.
{code}
$mvn -T 1C -DskipShade -Dtest=TestReadWhileWriting --pl 
hadoop-hdfs-project/hadoop-hdfs-client --pl hadoop-hdfs-project/hadoop-hdfs test
{code}


> Split the maven modules into several profiles
> -
>
> Key: HADOOP-14453
> URL: https://issues.apache.org/jira/browse/HADOOP-14453
> Project: Hadoop Common
>  Issue Type: Improvement
>  Components: build
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
>Priority: Major
> Attachments: c14453_20170524.patch
>
>
> Current all the modules are defined at directly under .  As a 
> result, we cannot select to build only some of the modules.  We have to build 
> all the modules in any cases and, unfortunately, it takes a long time.
> We propose split all the modules into multiple profiles so that we could 
> build some of the modules by disabling some of the profiles.  All the 
> profiles are enabled by default so that all the modules will be built by 
> default. 
> For example, when we are making change in common.  We could build and run 
> tests under common by disabling hdfs, yarn, mapreduce, etc. modules.  This 
> will reduce the development time spend on compiling unrelated modules.
> Note that this is for local maven builds.   We are not proposing to change 
> Jenkins builds, which always build all the modules.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Commented] (HADOOP-14655) Update httpcore version to 4.4.6

2017-09-23 Thread Tsz Wo Nicholas Sze (JIRA)

[ 
https://issues.apache.org/jira/browse/HADOOP-14655?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16177631#comment-16177631
 ] 

Tsz Wo Nicholas Sze commented on HADOOP-14655:
--

[~elek], thanks a lot for figuring out the details!

> Update httpcore version to 4.4.6
> 
>
> Key: HADOOP-14655
> URL: https://issues.apache.org/jira/browse/HADOOP-14655
> Project: Hadoop Common
>  Issue Type: Sub-task
>Reporter: Ray Chiang
>Assignee: Ray Chiang
> Attachments: HADOOP-14655.001.patch
>
>
> Update the dependency
> org.apache.httpcomponents:httpcore:4.4.4
> to the latest (4.4.6).



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Commented] (HADOOP-14655) Update httpcore version to 4.4.6

2017-09-20 Thread Tsz Wo Nicholas Sze (JIRA)

[ 
https://issues.apache.org/jira/browse/HADOOP-14655?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16172915#comment-16172915
 ] 

Tsz Wo Nicholas Sze commented on HADOOP-14655:
--

The error disappears after reverting the patch locally.  Please take a look.  
Thanks.

> Update httpcore version to 4.4.6
> 
>
> Key: HADOOP-14655
> URL: https://issues.apache.org/jira/browse/HADOOP-14655
> Project: Hadoop Common
>  Issue Type: Sub-task
>Reporter: Ray Chiang
>Assignee: Ray Chiang
> Fix For: 3.0.0-beta1
>
> Attachments: HADOOP-14655.001.patch
>
>
> Update the dependency
> org.apache.httpcomponents:httpcore:4.4.4
> to the latest (4.4.6).



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Commented] (HADOOP-14655) Update httpcore version to 4.4.6

2017-09-20 Thread Tsz Wo Nicholas Sze (JIRA)

[ 
https://issues.apache.org/jira/browse/HADOOP-14655?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16172910#comment-16172910
 ] 

Tsz Wo Nicholas Sze commented on HADOOP-14655:
--

It seems that this causes HDFS-12507.

> Update httpcore version to 4.4.6
> 
>
> Key: HADOOP-14655
> URL: https://issues.apache.org/jira/browse/HADOOP-14655
> Project: Hadoop Common
>  Issue Type: Sub-task
>Reporter: Ray Chiang
>Assignee: Ray Chiang
> Fix For: 3.0.0-beta1
>
> Attachments: HADOOP-14655.001.patch
>
>
> Update the dependency
> org.apache.httpcomponents:httpcore:4.4.4
> to the latest (4.4.6).



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Commented] (HADOOP-14453) Split the maven modules into several profiles

2017-05-27 Thread Tsz Wo Nicholas Sze (JIRA)

[ 
https://issues.apache.org/jira/browse/HADOOP-14453?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16027324#comment-16027324
 ] 

Tsz Wo Nicholas Sze commented on HADOOP-14453:
--

> ... its what the --pl command does.

It would be great if this works.  Suppose we want to run TestReadWhileWriting 
in HDFS and the build should stop after the test is passed.  What is the full 
command to do it?

> Split the maven modules into several profiles
> -
>
> Key: HADOOP-14453
> URL: https://issues.apache.org/jira/browse/HADOOP-14453
> Project: Hadoop Common
>  Issue Type: Improvement
>  Components: build
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
> Attachments: c14453_20170524.patch
>
>
> Current all the modules are defined at directly under .  As a 
> result, we cannot select to build only some of the modules.  We have to build 
> all the modules in any cases and, unfortunately, it takes a long time.
> We propose split all the modules into multiple profiles so that we could 
> build some of the modules by disabling some of the profiles.  All the 
> profiles are enabled by default so that all the modules will be built by 
> default. 
> For example, when we are making change in common.  We could build and run 
> tests under common by disabling hdfs, yarn, mapreduce, etc. modules.  This 
> will reduce the development time spend on compiling unrelated modules.
> Note that this is for local maven builds.   We are not proposing to change 
> Jenkins builds, which always build all the modules.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Commented] (HADOOP-14453) Split the maven modules into several profiles

2017-05-24 Thread Tsz Wo Nicholas Sze (JIRA)

[ 
https://issues.apache.org/jira/browse/HADOOP-14453?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16024057#comment-16024057
 ] 

Tsz Wo Nicholas Sze commented on HADOOP-14453:
--

> For the once-per-day cost? ...

Why once-per-day?  We may work on multiple dependent patches on multiple 
instance of git clones and branches at the same time.  For example, I am 
currently mostly working on trunk and the HDFS-7240 branch, and I have 5 git 
clones in my machine.

> Split the maven modules into several profiles
> -
>
> Key: HADOOP-14453
> URL: https://issues.apache.org/jira/browse/HADOOP-14453
> Project: Hadoop Common
>  Issue Type: Improvement
>  Components: build
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
> Attachments: c14453_20170524.patch
>
>
> Current all the modules are defined at directly under .  As a 
> result, we cannot select to build only some of the modules.  We have to build 
> all the modules in any cases and, unfortunately, it takes a long time.
> We propose split all the modules into multiple profiles so that we could 
> build some of the modules by disabling some of the profiles.  All the 
> profiles are enabled by default so that all the modules will be built by 
> default. 
> For example, when we are making change in common.  We could build and run 
> tests under common by disabling hdfs, yarn, mapreduce, etc. modules.  This 
> will reduce the development time spend on compiling unrelated modules.
> Note that this is for local maven builds.   We are not proposing to change 
> Jenkins builds, which always build all the modules.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Commented] (HADOOP-14453) Split the maven modules into several profiles

2017-05-24 Thread Tsz Wo Nicholas Sze (JIRA)

[ 
https://issues.apache.org/jira/browse/HADOOP-14453?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16023794#comment-16023794
 ] 

Tsz Wo Nicholas Sze commented on HADOOP-14453:
--

{{mvn install -DskipTests}} seems taking a really long time, no?

> Split the maven modules into several profiles
> -
>
> Key: HADOOP-14453
> URL: https://issues.apache.org/jira/browse/HADOOP-14453
> Project: Hadoop Common
>  Issue Type: Improvement
>  Components: build
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
> Attachments: c14453_20170524.patch
>
>
> Current all the modules are defined at directly under .  As a 
> result, we cannot select to build only some of the modules.  We have to build 
> all the modules in any cases and, unfortunately, it takes a long time.
> We propose split all the modules into multiple profiles so that we could 
> build some of the modules by disabling some of the profiles.  All the 
> profiles are enabled by default so that all the modules will be built by 
> default. 
> For example, when we are making change in common.  We could build and run 
> tests under common by disabling hdfs, yarn, mapreduce, etc. modules.  This 
> will reduce the development time spend on compiling unrelated modules.
> Note that this is for local maven builds.   We are not proposing to change 
> Jenkins builds, which always build all the modules.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Updated] (HADOOP-14453) Split the maven modules into several profiles

2017-05-24 Thread Tsz Wo Nicholas Sze (JIRA)

 [ 
https://issues.apache.org/jira/browse/HADOOP-14453?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Tsz Wo Nicholas Sze updated HADOOP-14453:
-
Status: Patch Available  (was: Open)

> Split the maven modules into several profiles
> -
>
> Key: HADOOP-14453
> URL: https://issues.apache.org/jira/browse/HADOOP-14453
> Project: Hadoop Common
>  Issue Type: Improvement
>  Components: build
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
> Attachments: c14453_20170524.patch
>
>
> Current all the modules are defined at directly under .  As a 
> result, we cannot select to build only some of the modules.  We have to build 
> all the modules in any cases and, unfortunately, it takes a long time.
> We propose split all the modules into multiple profiles so that we could 
> build some of the modules by disabling some of the profiles.  All the 
> profiles are enabled by default so that all the modules will be built by 
> default. 
> For example, when we are making change in common.  We could build and run 
> tests under common by disabling hdfs, yarn, mapreduce, etc. modules.  This 
> will reduce the development time spend on compiling unrelated modules.
> Note that this is for local maven builds.   We are not proposing to change 
> Jenkins builds, which always build all the modules.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Updated] (HADOOP-14453) Split the maven modules into several profiles

2017-05-24 Thread Tsz Wo Nicholas Sze (JIRA)

 [ 
https://issues.apache.org/jira/browse/HADOOP-14453?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Tsz Wo Nicholas Sze updated HADOOP-14453:
-
Attachment: c14453_20170524.patch

c14453_20170524.patch: splits the modules into 4 profiles:
- common-modules
- hdfs-modules
- yarn-modules
- other-modules

Then, we may skip building some of the modules by disabling some profiles. For 
example, the following command build on common and hdfs modules.
{code}
mvn -P-yarn-modules,-other-modules test -Dtest=...
{code}

> Split the maven modules into several profiles
> -
>
> Key: HADOOP-14453
> URL: https://issues.apache.org/jira/browse/HADOOP-14453
> Project: Hadoop Common
>  Issue Type: Improvement
>  Components: build
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
> Attachments: c14453_20170524.patch
>
>
> Current all the modules are defined at directly under .  As a 
> result, we cannot select to build only some of the modules.  We have to build 
> all the modules in any cases and, unfortunately, it takes a long time.
> We propose split all the modules into multiple profiles so that we could 
> build some of the modules by disabling some of the profiles.  All the 
> profiles are enabled by default so that all the modules will be built by 
> default. 
> For example, when we are making change in common.  We could build and run 
> tests under common by disabling hdfs, yarn, mapreduce, etc. modules.  This 
> will reduce the development time spend on compiling unrelated modules.
> Note that this is for local maven builds.   We are not proposing to change 
> Jenkins builds, which always build all the modules.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Created] (HADOOP-14453) Split the maven modules into several profiles

2017-05-24 Thread Tsz Wo Nicholas Sze (JIRA)
Tsz Wo Nicholas Sze created HADOOP-14453:


 Summary: Split the maven modules into several profiles
 Key: HADOOP-14453
 URL: https://issues.apache.org/jira/browse/HADOOP-14453
 Project: Hadoop Common
  Issue Type: Improvement
  Components: build
Reporter: Tsz Wo Nicholas Sze
Assignee: Tsz Wo Nicholas Sze


Current all the modules are defined at directly under .  As a result, 
we cannot select to build only some of the modules.  We have to build all the 
modules in any cases and, unfortunately, it takes a long time.

We propose split all the modules into multiple profiles so that we could build 
some of the modules by disabling some of the profiles.  All the profiles are 
enabled by default so that all the modules will be built by default. 

For example, when we are making change in common.  We could build and run tests 
under common by disabling hdfs, yarn, mapreduce, etc. modules.  This will 
reduce the development time spend on compiling unrelated modules.

Note that this is for local maven builds.   We are not proposing to change 
Jenkins builds, which always build all the modules.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Commented] (HADOOP-14405) Fix performance regression due to incorrect use of DataChecksum

2017-05-10 Thread Tsz Wo Nicholas Sze (JIRA)

[ 
https://issues.apache.org/jira/browse/HADOOP-14405?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16005106#comment-16005106
 ] 

Tsz Wo Nicholas Sze commented on HADOOP-14405:
--

Thanks [~GeLiXin] for the contribution and [~iwasakims] for committing the 
patch.

> Fix performance regression due to incorrect use of DataChecksum
> ---
>
> Key: HADOOP-14405
> URL: https://issues.apache.org/jira/browse/HADOOP-14405
> Project: Hadoop Common
>  Issue Type: Bug
>  Components: native, performance
>Affects Versions: 2.8.0, 3.0.0-alpha1
>Reporter: LiXin Ge
>Assignee: LiXin Ge
> Fix For: 2.9.0, 2.8.1, 3.0.0-alpha3
>
> Attachments: HDFS-11765.patch
>
>
> Recently I have upgraded my Hadoop version from 2.6 to 3.0, and I find that 
> the write performance decreased by 13%. After some days comparative analysis, 
> It's seems introduced by HADOOP-10865. 
> Since James Thomas have done the work that native checksum can run against 
> byte[] arrays instead of just against byte buffers, we may use native method 
> preferential because it runs faster than others.
> [~szetszwo] and [~iwasakims] could you take a look at this to see if  it make 
> bad effect on your benchmark test? [~tlipcon] could you help to see if I have 
> make mistakes in this patch?
> thanks!



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Commented] (HADOOP-14217) Object Storage: support colon in object path

2017-04-11 Thread Tsz Wo Nicholas Sze (JIRA)

[ 
https://issues.apache.org/jira/browse/HADOOP-14217?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15965236#comment-15965236
 ] 

Tsz Wo Nicholas Sze commented on HADOOP-14217:
--

Yes, we should fix it.

We probably should first define a grammar so that our implementation stands on 
a more rigorous ground.

> Object Storage: support colon in object path
> 
>
> Key: HADOOP-14217
> URL: https://issues.apache.org/jira/browse/HADOOP-14217
> Project: Hadoop Common
>  Issue Type: Sub-task
>  Components: fs, fs/oss
>Reporter: Genmao Yu
>




--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Commented] (HADOOP-14084) Shell.joinThread swallows InterruptedException

2017-03-28 Thread Tsz Wo Nicholas Sze (JIRA)

[ 
https://issues.apache.org/jira/browse/HADOOP-14084?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15946462#comment-15946462
 ] 

Tsz Wo Nicholas Sze commented on HADOOP-14084:
--

We need to decide what is the expected behavior of joinThread(..) when it is 
interrupted.  The behavior probably depends on which thread it is joining.  In 
our case, it is joining the errThread.  Then, I think re-throwing the 
InterruptedException is fine, i.e. something similar to HADOOP-14084.000.patch.

BTW, please see if you could add a test.

> Shell.joinThread swallows InterruptedException
> --
>
> Key: HADOOP-14084
> URL: https://issues.apache.org/jira/browse/HADOOP-14084
> Project: Hadoop Common
>  Issue Type: Bug
>Reporter: Shixiong Zhu
>Assignee: Xiaobing Zhou
>Priority: Minor
> Attachments: HADOOP-14084.000.patch, HADOOP-14084.001.patch
>
>
> In "Shell.joinThread", when the user tries to interrupt the thread that runs 
> Shell.joinThread, it will catch InterruptedException and propagate it to 
> thread t. However, it doesn't set the interrupt state of the current thread 
> before returning, so the user codes won't know it's already interrupted.
> See 
> https://github.com/apache/hadoop/blob/9e19f758c1950cbcfcd1969461a8a910efca0767/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java#L1035



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Commented] (HADOOP-14217) Object Storage: support colon in object path

2017-03-24 Thread Tsz Wo Nicholas Sze (JIRA)

[ 
https://issues.apache.org/jira/browse/HADOOP-14217?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15941070#comment-15941070
 ] 

Tsz Wo Nicholas Sze commented on HADOOP-14217:
--

Unable to support ':' is a long standing issue, as early as HDFS-13.  I have a 
patch 
([2066_20071022.patch|https://issues.apache.org/jira/secure/attachment/12368184/2066_20071022.patch])
 there (wow) almost 10 years ago.

One problem is that the general URI syntax is very general.  We could safely 
assume that our URIs are [hierarchical 
URIs|http://docs.oracle.com/javase/8/docs/api/java/net/URI.html], i.e. 
{code}
[scheme:][//authority][path][?query][#fragment] 
{code}
or even
{code}
[[scheme:]//authority]path
{code}
Then, the problem becomes fixable as shown in [this 
comment|https://issues.apache.org/jira/browse/HDFS-13?focusedCommentId=12536875&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-12536875].
  

> Object Storage: support colon in object path
> 
>
> Key: HADOOP-14217
> URL: https://issues.apache.org/jira/browse/HADOOP-14217
> Project: Hadoop Common
>  Issue Type: Sub-task
>  Components: fs, fs/oss
>Reporter: Genmao Yu
>




--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Updated] (HADOOP-10101) Update guava dependency to the latest version

2017-03-24 Thread Tsz Wo Nicholas Sze (JIRA)

 [ 
https://issues.apache.org/jira/browse/HADOOP-10101?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Tsz Wo Nicholas Sze updated HADOOP-10101:
-
Hadoop Flags: Incompatible change,Reviewed  (was: Incompatible change)

+1 the 018 patch is perfect, thanks!

> Update guava dependency to the latest version
> -
>
> Key: HADOOP-10101
> URL: https://issues.apache.org/jira/browse/HADOOP-10101
> Project: Hadoop Common
>  Issue Type: Improvement
>Affects Versions: 3.0.0-alpha2
>Reporter: Rakesh R
>Assignee: Tsuyoshi Ozawa
>  Labels: BB2015-05-TBR
> Attachments: HADOOP-10101-002.patch, HADOOP-10101-004.patch, 
> HADOOP-10101-005.patch, HADOOP-10101-006.patch, HADOOP-10101-007.patch, 
> HADOOP-10101-008.patch, HADOOP-10101-009.patch, HADOOP-10101-009.patch, 
> HADOOP-10101-010.patch, HADOOP-10101-010.patch, HADOOP-10101-011.patch, 
> HADOOP-10101.012.patch, HADOOP-10101.013.patch, HADOOP-10101.014.patch, 
> HADOOP-10101.015.patch, HADOOP-10101.016.patch, HADOOP-10101.017.patch, 
> HADOOP-10101.018.patch, HADOOP-10101.patch, HADOOP-10101.patch
>
>
> The existing guava version is 11.0.2 which is quite old. This issue tries to 
> update the version to as latest version as possible. 



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Commented] (HADOOP-14214) DomainSocketWatcher::add()/delete() should not self interrupt while looping await()

2017-03-23 Thread Tsz Wo Nicholas Sze (JIRA)

[ 
https://issues.apache.org/jira/browse/HADOOP-14214?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15939179#comment-15939179
 ] 

Tsz Wo Nicholas Sze commented on HADOOP-14214:
--

The try-await-catch-InterruptedException-interrupt is clearly a bug.  Using 
awaitUninterruptibly sounds good.

+1 on the patch.

> DomainSocketWatcher::add()/delete() should not self interrupt while looping 
> await()
> ---
>
> Key: HADOOP-14214
> URL: https://issues.apache.org/jira/browse/HADOOP-14214
> Project: Hadoop Common
>  Issue Type: Bug
>  Components: hdfs-client
>Reporter: Mingliang Liu
>Assignee: Mingliang Liu
>Priority: Critical
> Attachments: HADOOP-14214.000.patch
>
>
> Our hive team found a TPCDS job whose queries running on LLAP seem to be 
> getting stuck. Dozens of threads were waiting for the 
> {{DfsClientShmManager::lock}}, as following jstack:
> {code}
> Thread 251 (IO-Elevator-Thread-5):
>   State: WAITING
>   Blocked count: 3871
>   Wtaited count: 4565
>   Waiting on 
> java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject@16ead198
>   Stack:
> sun.misc.Unsafe.park(Native Method)
> java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
> 
> java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitUninterruptibly(AbstractQueuedSynchronizer.java:1976)
> 
> org.apache.hadoop.hdfs.shortcircuit.DfsClientShmManager$EndpointShmManager.allocSlot(DfsClientShmManager.java:255)
> 
> org.apache.hadoop.hdfs.shortcircuit.DfsClientShmManager.allocSlot(DfsClientShmManager.java:434)
> 
> org.apache.hadoop.hdfs.shortcircuit.ShortCircuitCache.allocShmSlot(ShortCircuitCache.java:1017)
> 
> org.apache.hadoop.hdfs.BlockReaderFactory.createShortCircuitReplicaInfo(BlockReaderFactory.java:476)
> 
> org.apache.hadoop.hdfs.shortcircuit.ShortCircuitCache.create(ShortCircuitCache.java:784)
> 
> org.apache.hadoop.hdfs.shortcircuit.ShortCircuitCache.fetchOrCreate(ShortCircuitCache.java:718)
> 
> org.apache.hadoop.hdfs.BlockReaderFactory.getBlockReaderLocal(BlockReaderFactory.java:422)
> 
> org.apache.hadoop.hdfs.BlockReaderFactory.build(BlockReaderFactory.java:333)
> 
> org.apache.hadoop.hdfs.DFSInputStream.actualGetFromOneDataNode(DFSInputStream.java:1181)
> 
> org.apache.hadoop.hdfs.DFSInputStream.fetchBlockByteRange(DFSInputStream.java:1118)
> org.apache.hadoop.hdfs.DFSInputStream.pread(DFSInputStream.java:1478)
> org.apache.hadoop.hdfs.DFSInputStream.read(DFSInputStream.java:1441)
> org.apache.hadoop.fs.FSInputStream.readFully(FSInputStream.java:121)
> 
> org.apache.hadoop.fs.FSDataInputStream.readFully(FSDataInputStream.java:111)
> 
> org.apache.orc.impl.RecordReaderUtils$DefaultDataReader.readStripeFooter(RecordReaderUtils.java:166)
> 
> org.apache.hadoop.hive.llap.io.metadata.OrcStripeMetadata.(OrcStripeMetadata.java:64)
> 
> org.apache.hadoop.hive.llap.io.encoded.OrcEncodedDataReader.readStripesMetadata(OrcEncodedDataReader.java:622)
> {code}
> The thread that is expected to signal those threads is calling 
> {{DomainSocketWatcher::add()}} method, but it gets stuck there dealing with 
> InterruptedException infinitely. The jstack is like:
> {code}
> Thread 44417 (TezTR-257387_2840_12_10_52_0):
>   State: RUNNABLE
>   Blocked count: 3
>   Wtaited count: 5
>   Stack:
> java.lang.Throwable.fillInStackTrace(Native Method)
> java.lang.Throwable.fillInStackTrace(Throwable.java:783)
> java.lang.Throwable.(Throwable.java:250)
> java.lang.Exception.(Exception.java:54)
> java.lang.InterruptedException.(InterruptedException.java:57)
> 
> java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2034)
> 
> org.apache.hadoop.net.unix.DomainSocketWatcher.add(DomainSocketWatcher.java:325)
> 
> org.apache.hadoop.hdfs.shortcircuit.DfsClientShmManager$EndpointShmManager.allocSlot(DfsClientShmManager.java:266)
> 
> org.apache.hadoop.hdfs.shortcircuit.DfsClientShmManager.allocSlot(DfsClientShmManager.java:434)
> 
> org.apache.hadoop.hdfs.shortcircuit.ShortCircuitCache.allocShmSlot(ShortCircuitCache.java:1017)
> 
> org.apache.hadoop.hdfs.BlockReaderFactory.createShortCircuitReplicaInfo(BlockReaderFactory.java:476)
> 
> org.apache.hadoop.hdfs.shortcircuit.ShortCircuitCache.create(ShortCircuitCache.java:784)
> 
> org.apache.hadoop.hdfs.shortcircuit.ShortCircuitCache.fetchOrCreate(ShortCircuitCache.java:718)
> 
> org.apache.hadoop.hdfs.BlockReaderFactory.getBlockReaderLocal(BlockReaderFactory.java:422)
> 
> org.apache.hadoop.hdfs.BlockReaderFactory.build(BlockReaderFactory.java:333)
> 
> org.apache.hadoop.hdfs.DFSInputStream.actualGetFromOneDataN

[jira] [Commented] (HADOOP-10101) Update guava dependency to the latest version

2017-03-23 Thread Tsz Wo Nicholas Sze (JIRA)

[ 
https://issues.apache.org/jira/browse/HADOOP-10101?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15938962#comment-15938962
 ] 

Tsz Wo Nicholas Sze commented on HADOOP-10101:
--

Patch looks good.  Thanks for working on this well known hard issue -- Hadoop 
is stuck on Guava 11.

Please take a look the javac/checkstyle warnings.  The test failures seem 
unrelated.  Please take a look as well.  Thanks.

> Update guava dependency to the latest version
> -
>
> Key: HADOOP-10101
> URL: https://issues.apache.org/jira/browse/HADOOP-10101
> Project: Hadoop Common
>  Issue Type: Improvement
>Affects Versions: 3.0.0-alpha2
>Reporter: Rakesh R
>Assignee: Tsuyoshi Ozawa
>  Labels: BB2015-05-TBR
> Attachments: HADOOP-10101-002.patch, HADOOP-10101-004.patch, 
> HADOOP-10101-005.patch, HADOOP-10101-006.patch, HADOOP-10101-007.patch, 
> HADOOP-10101-008.patch, HADOOP-10101-009.patch, HADOOP-10101-009.patch, 
> HADOOP-10101-010.patch, HADOOP-10101-010.patch, HADOOP-10101-011.patch, 
> HADOOP-10101.012.patch, HADOOP-10101.013.patch, HADOOP-10101.014.patch, 
> HADOOP-10101.015.patch, HADOOP-10101.patch, HADOOP-10101.patch
>
>
> The existing guava version is 11.0.2 which is quite old. This issue tries to 
> update the version to as latest version as possible. 



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Commented] (HADOOP-10101) Update guava dependency to the latest version

2017-03-13 Thread Tsz Wo Nicholas Sze (JIRA)

[ 
https://issues.apache.org/jira/browse/HADOOP-10101?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15923348#comment-15923348
 ] 

Tsz Wo Nicholas Sze commented on HADOOP-10101:
--

> I think it is high time we bump up the dependency however we should look to 
> hiding it too i.e. shading it as needed so that downstream users can use 
> whatever version of guava they want to use. 

I agree with [~hitesh] that we should shade guava.  Then, this JIRA is no 
longer an incompatible change.

> Update guava dependency to the latest version
> -
>
> Key: HADOOP-10101
> URL: https://issues.apache.org/jira/browse/HADOOP-10101
> Project: Hadoop Common
>  Issue Type: Improvement
>Affects Versions: 2.2.0, 2.6.0
>Reporter: Rakesh R
>Assignee: Vinayakumar B
>  Labels: BB2015-05-TBR
> Attachments: HADOOP-10101-002.patch, HADOOP-10101-004.patch, 
> HADOOP-10101-005.patch, HADOOP-10101-006.patch, HADOOP-10101-007.patch, 
> HADOOP-10101-008.patch, HADOOP-10101-009.patch, HADOOP-10101-009.patch, 
> HADOOP-10101-010.patch, HADOOP-10101-010.patch, HADOOP-10101-011.patch, 
> HADOOP-10101.patch, HADOOP-10101.patch
>
>
> The existing guava version is 11.0.2 which is quite old. This 



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Updated] (HADOOP-13459) hadoop-azure runs several test cases repeatedly, causing unnecessarily long running time.

2016-08-02 Thread Tsz Wo Nicholas Sze (JIRA)

 [ 
https://issues.apache.org/jira/browse/HADOOP-13459?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Tsz Wo Nicholas Sze updated HADOOP-13459:
-
Hadoop Flags: Reviewed

It is great to reduce the test run time.

+1 patch looks good.

> hadoop-azure runs several test cases repeatedly, causing unnecessarily long 
> running time.
> -
>
> Key: HADOOP-13459
> URL: https://issues.apache.org/jira/browse/HADOOP-13459
> Project: Hadoop Common
>  Issue Type: Bug
>  Components: fs/azure, test
>Reporter: Chris Nauroth
>Assignee: Chris Nauroth
>Priority: Minor
> Attachments: HADOOP-13459.001.patch
>
>
> Within hadoop-azure, we have the {{NativeAzureFileSystemBaseTest}} abstract 
> class, which defines setup and teardown to handle the Azure storage account 
> and also defines multiple test cases.  This class originally was contributed 
> to provide a layer of indirection for running the same test cases in live 
> mode or mock mode: {{TestNativeAzureFileSystemLive}} and 
> {{TestNativeAzureFileSystemMocked}}.  It appears that since then, we created 
> multiple new test suites that subclassed {{NativeAzureFileSystemBaseTest}} 
> for the benefit of getting the common setup and teardown code, but also with 
> the side effect of running the inherited test cases repeatedly.  This is a 
> significant factor in the overall execution time of the hadoop-azure tests.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Issue Comment Deleted] (HADOOP-5052) Add an example for computing exact digits of Pi

2016-07-07 Thread Tsz Wo Nicholas Sze (JIRA)

 [ 
https://issues.apache.org/jira/browse/HADOOP-5052?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Tsz Wo Nicholas Sze updated HADOOP-5052:

Comment: was deleted

(was: Very useful example visit here for more information. 
(http://bit.ly/1WQKTZ6))

> Add an example for computing exact digits of Pi
> ---
>
> Key: HADOOP-5052
> URL: https://issues.apache.org/jira/browse/HADOOP-5052
> Project: Hadoop Common
>  Issue Type: New Feature
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
> Fix For: 0.21.0
>
> Attachments: 5052_20090115_0.18.patch, 5052_20090117_0.18.patch, 
> 5052_20090127.patch, 5052_20090203.patch, 5052_20090211.patch, 
> 5052_20090219.patch
>
>
> It would be useful to add an example showing how to use Hadoop to do 
> scientific computing.  We should add an example for computing exact digits of 
> Pi.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Updated] (HADOOP-13227) AsyncCallHandler should use an event driven architecture to handle async calls

2016-06-25 Thread Tsz Wo Nicholas Sze (JIRA)

 [ 
https://issues.apache.org/jira/browse/HADOOP-13227?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Tsz Wo Nicholas Sze updated HADOOP-13227:
-
   Resolution: Fixed
Fix Version/s: 2.8.0
   Status: Resolved  (was: Patch Available)

I have committed this.

> AsyncCallHandler should use an event driven architecture to handle async calls
> --
>
> Key: HADOOP-13227
> URL: https://issues.apache.org/jira/browse/HADOOP-13227
> Project: Hadoop Common
>  Issue Type: Improvement
>  Components: io, ipc
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
> Fix For: 2.8.0
>
> Attachments: c13227_20160602.patch, c13227_20160606.patch, 
> c13227_20160607.patch, c13227_20160608.patch, c13227_20160608b.patch, 
> c13227_20160616.patch, c13227_20160621.patch, c13227_20160624.patch
>
>
> This JIRA is to address [Jing's 
> comments|https://issues.apache.org/jira/browse/HADOOP-13226?focusedCommentId=15308630&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-15308630]
>  in HADOOP-13226.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Updated] (HADOOP-13249) RetryInvocationHandler need wrap InterruptedException in IOException when call Thread.sleep

2016-06-25 Thread Tsz Wo Nicholas Sze (JIRA)

 [ 
https://issues.apache.org/jira/browse/HADOOP-13249?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Tsz Wo Nicholas Sze updated HADOOP-13249:
-
Fix Version/s: (was: 2.9.0)
   2.8.0

Merged this to branch-2.8.

> RetryInvocationHandler need wrap InterruptedException in IOException when 
> call Thread.sleep
> ---
>
> Key: HADOOP-13249
> URL: https://issues.apache.org/jira/browse/HADOOP-13249
> Project: Hadoop Common
>  Issue Type: Bug
>  Components: ipc
>Affects Versions: 2.8.0
>Reporter: zhihai xu
>Assignee: zhihai xu
> Fix For: 2.8.0
>
> Attachments: HADOOP-13249.000.patch, HADOOP-13249.001.patch, 
> HADOOP-13249.002.patch
>
>
> RetryInvocationHandler need wrap InterruptedException in IOException when 
> call Thread.sleep. Otherwise InterruptedException can't be handled correctly 
> by other components such as HDFS. 



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Updated] (HADOOP-13227) AsyncCallHandler should use an event driven architecture to handle async calls

2016-06-25 Thread Tsz Wo Nicholas Sze (JIRA)

 [ 
https://issues.apache.org/jira/browse/HADOOP-13227?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Tsz Wo Nicholas Sze updated HADOOP-13227:
-
Hadoop Flags: Reviewed
 Summary: AsyncCallHandler should use an event driven architecture to 
handle async calls  (was: AsyncCallHandler should use a event driven 
architecture to handle async calls)

Thanks Jing for reviewing the patches.

> AsyncCallHandler should use an event driven architecture to handle async calls
> --
>
> Key: HADOOP-13227
> URL: https://issues.apache.org/jira/browse/HADOOP-13227
> Project: Hadoop Common
>  Issue Type: Improvement
>  Components: io, ipc
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
> Attachments: c13227_20160602.patch, c13227_20160606.patch, 
> c13227_20160607.patch, c13227_20160608.patch, c13227_20160608b.patch, 
> c13227_20160616.patch, c13227_20160621.patch, c13227_20160624.patch
>
>
> This JIRA is to address [Jing's 
> comments|https://issues.apache.org/jira/browse/HADOOP-13226?focusedCommentId=15308630&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-15308630]
>  in HADOOP-13226.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Updated] (HADOOP-13227) AsyncCallHandler should use a event driven architecture to handle async calls

2016-06-23 Thread Tsz Wo Nicholas Sze (JIRA)

 [ 
https://issues.apache.org/jira/browse/HADOOP-13227?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Tsz Wo Nicholas Sze updated HADOOP-13227:
-
Attachment: c13227_20160624.patch

c13227_20160624.patch: fixes a compilation problem.

> AsyncCallHandler should use a event driven architecture to handle async calls
> -
>
> Key: HADOOP-13227
> URL: https://issues.apache.org/jira/browse/HADOOP-13227
> Project: Hadoop Common
>  Issue Type: Improvement
>  Components: io, ipc
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
> Attachments: c13227_20160602.patch, c13227_20160606.patch, 
> c13227_20160607.patch, c13227_20160608.patch, c13227_20160608b.patch, 
> c13227_20160616.patch, c13227_20160621.patch, c13227_20160624.patch
>
>
> This JIRA is to address [Jing's 
> comments|https://issues.apache.org/jira/browse/HADOOP-13226?focusedCommentId=15308630&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-15308630]
>  in HADOOP-13226.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Updated] (HADOOP-13227) AsyncCallHandler should use a event driven architecture to handle async calls

2016-06-22 Thread Tsz Wo Nicholas Sze (JIRA)

 [ 
https://issues.apache.org/jira/browse/HADOOP-13227?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Tsz Wo Nicholas Sze updated HADOOP-13227:
-
Attachment: c13227_20160621.patch

c13227_20160621.patch: sync'ed with trunk.

> AsyncCallHandler should use a event driven architecture to handle async calls
> -
>
> Key: HADOOP-13227
> URL: https://issues.apache.org/jira/browse/HADOOP-13227
> Project: Hadoop Common
>  Issue Type: Improvement
>  Components: io, ipc
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
> Attachments: c13227_20160602.patch, c13227_20160606.patch, 
> c13227_20160607.patch, c13227_20160608.patch, c13227_20160608b.patch, 
> c13227_20160616.patch, c13227_20160621.patch
>
>
> This JIRA is to address [Jing's 
> comments|https://issues.apache.org/jira/browse/HADOOP-13226?focusedCommentId=15308630&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-15308630]
>  in HADOOP-13226.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Updated] (HADOOP-13227) AsyncCallHandler should use a event driven architecture to handle async calls

2016-06-16 Thread Tsz Wo Nicholas Sze (JIRA)

 [ 
https://issues.apache.org/jira/browse/HADOOP-13227?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Tsz Wo Nicholas Sze updated HADOOP-13227:
-
Attachment: c13227_20160616.patch

c13227_20160616.patch: sync'ed with trunk.

> AsyncCallHandler should use a event driven architecture to handle async calls
> -
>
> Key: HADOOP-13227
> URL: https://issues.apache.org/jira/browse/HADOOP-13227
> Project: Hadoop Common
>  Issue Type: Improvement
>  Components: io, ipc
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
> Attachments: c13227_20160602.patch, c13227_20160606.patch, 
> c13227_20160607.patch, c13227_20160608.patch, c13227_20160608b.patch, 
> c13227_20160616.patch
>
>
> This JIRA is to address [Jing's 
> comments|https://issues.apache.org/jira/browse/HADOOP-13226?focusedCommentId=15308630&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-15308630]
>  in HADOOP-13226.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Commented] (HADOOP-12910) Add new FileSystem API to support asynchronous method calls

2016-06-09 Thread Tsz Wo Nicholas Sze (JIRA)

[ 
https://issues.apache.org/jira/browse/HADOOP-12910?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15323775#comment-15323775
 ] 

Tsz Wo Nicholas Sze commented on HADOOP-12910:
--

Chaining/callbacks are nonessential in the sense that they can possibly be 
provided by other library but async is not.  Correct me if I am wrong.

I want to support chaining/callbacks but may not be necessarily in the first 
step.  We could first have async and then chaining+callbacks; or first 
async+callbacks and then chaining.  I wish supporting chaining is easy so that 
I will just give you a patch.  On the other hand, I do need the async feature.

I really like to know what's wrong about the AWT-style APIs registering 
listener?  You are right that AWT and the Observer pattern are probably from 
90s or even earlier but ListenableFuture using the same approach was developed 
recently.

The "rocket launching" was a bad joke.  Sorry.

> Add new FileSystem API to support asynchronous method calls
> ---
>
> Key: HADOOP-12910
> URL: https://issues.apache.org/jira/browse/HADOOP-12910
> Project: Hadoop Common
>  Issue Type: New Feature
>  Components: fs
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Xiaobing Zhou
> Attachments: HADOOP-12910-HDFS-9924.000.patch, 
> HADOOP-12910-HDFS-9924.001.patch, HADOOP-12910-HDFS-9924.002.patch
>
>
> Add a new API, namely FutureFileSystem (or AsynchronousFileSystem, if it is a 
> better name).  All the APIs in FutureFileSystem are the same as FileSystem 
> except that the return type is wrapped by Future, e.g.
> {code}
>   //FileSystem
>   public boolean rename(Path src, Path dst) throws IOException;
>   //FutureFileSystem
>   public Future rename(Path src, Path dst) throws IOException;
> {code}
> Note that FutureFileSystem does not extend FileSystem.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Commented] (HADOOP-12910) Add new FileSystem API to support asynchronous method calls

2016-06-09 Thread Tsz Wo Nicholas Sze (JIRA)

[ 
https://issues.apache.org/jira/browse/HADOOP-12910?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15322273#comment-15322273
 ] 

Tsz Wo Nicholas Sze commented on HADOOP-12910:
--

After some thought, copying CompletionStage and CompletableFuture may not be 
straightforward since it requires to copy also the dependent classes such as 
the classes in the java.util.function package.  So returning ListenableFuture 
seems like a better choice.

> There is an additional tactical advantage with ListenableFuture. The current 
> implementation AsyncGetFuture already extends AbstractFuture that implements 
> ListenableFuture. So it seems pretty low hanging. ...

That's correct.  The current code already supports ListenableFuture although we 
keep the return type as Future.

> Add new FileSystem API to support asynchronous method calls
> ---
>
> Key: HADOOP-12910
> URL: https://issues.apache.org/jira/browse/HADOOP-12910
> Project: Hadoop Common
>  Issue Type: New Feature
>  Components: fs
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Xiaobing Zhou
> Attachments: HADOOP-12910-HDFS-9924.000.patch, 
> HADOOP-12910-HDFS-9924.001.patch, HADOOP-12910-HDFS-9924.002.patch
>
>
> Add a new API, namely FutureFileSystem (or AsynchronousFileSystem, if it is a 
> better name).  All the APIs in FutureFileSystem are the same as FileSystem 
> except that the return type is wrapped by Future, e.g.
> {code}
>   //FileSystem
>   public boolean rename(Path src, Path dst) throws IOException;
>   //FutureFileSystem
>   public Future rename(Path src, Path dst) throws IOException;
> {code}
> Note that FutureFileSystem does not extend FileSystem.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Commented] (HADOOP-12910) Add new FileSystem API to support asynchronous method calls

2016-06-09 Thread Tsz Wo Nicholas Sze (JIRA)

[ 
https://issues.apache.org/jira/browse/HADOOP-12910?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=1531#comment-1531
 ] 

Tsz Wo Nicholas Sze commented on HADOOP-12910:
--

Sorry, please forgive my bad wording.  I agree that chaining has its practical 
value.  I only want to emphasize that chaining can be archived by other mean 
once we have async support.

> Add new FileSystem API to support asynchronous method calls
> ---
>
> Key: HADOOP-12910
> URL: https://issues.apache.org/jira/browse/HADOOP-12910
> Project: Hadoop Common
>  Issue Type: New Feature
>  Components: fs
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Xiaobing Zhou
> Attachments: HADOOP-12910-HDFS-9924.000.patch, 
> HADOOP-12910-HDFS-9924.001.patch, HADOOP-12910-HDFS-9924.002.patch
>
>
> Add a new API, namely FutureFileSystem (or AsynchronousFileSystem, if it is a 
> better name).  All the APIs in FutureFileSystem are the same as FileSystem 
> except that the return type is wrapped by Future, e.g.
> {code}
>   //FileSystem
>   public boolean rename(Path src, Path dst) throws IOException;
>   //FutureFileSystem
>   public Future rename(Path src, Path dst) throws IOException;
> {code}
> Note that FutureFileSystem does not extend FileSystem.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Commented] (HADOOP-12910) Add new FileSystem API to support asynchronous method calls

2016-06-08 Thread Tsz Wo Nicholas Sze (JIRA)

[ 
https://issues.apache.org/jira/browse/HADOOP-12910?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15320902#comment-15320902
 ] 

Tsz Wo Nicholas Sze commented on HADOOP-12910:
--

Some other ideas:

# Use ListenableFuture. (exteneral dependency)
# Copy & paste ListenableFuture (the license may not permit)
# Copy & paste CompletionStage and CompletableFuture; see the details below.

#3 in more details: Java 8 
java.util.concurrent.CompletionStage|CompletableFuture are in [public domain 
CC0|http://creativecommons.org/publicdomain/zero/1.0/] so that we may copy them 
to org.apache.hadoop.util.concurrent.CompletionStage|CompletableFuture.  We may 
also need to copy the dependent classes. (Please correct me if I have 
misinterpreted the license.)

In branch-2, define a new class, say HadoopCompletableFuture, which implements 
Future and org.apache.hadoop.util.concurrent.CompletionStage.  The 
implementation of HadoopCompletableFuture is simply using 
org.apache.hadoop.util.concurrent.CompletableFuture.  In trunk, 
HadoopCompletableFuture in addition implements 
java.util.concurrent.CompletionStage.  Then trunk has Java 8 support and is 
backward compatible to branch-2.  In branch-2, it also has the chaining support 
although it does not have the Java 8 language feature such as lambda expression.
{code}
//branch-2
interface HadoopCompletableFuture extends Future, 
org.apache.hadoop.util.concurrent.CompletionStage {
}
{code}
{code}
//trunk
interface HadoopCompletableFuture extends Future, 
org.apache.hadoop.util.concurrent.CompletionStage, 
java.util.concurrent.CompletionStage {
}
{code}
In other words, we copy the Java 8 CompletableFuture related library to 
branch-2 since it is in public domain.  What do you think?


> Add new FileSystem API to support asynchronous method calls
> ---
>
> Key: HADOOP-12910
> URL: https://issues.apache.org/jira/browse/HADOOP-12910
> Project: Hadoop Common
>  Issue Type: New Feature
>  Components: fs
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Xiaobing Zhou
> Attachments: HADOOP-12910-HDFS-9924.000.patch, 
> HADOOP-12910-HDFS-9924.001.patch, HADOOP-12910-HDFS-9924.002.patch
>
>
> Add a new API, namely FutureFileSystem (or AsynchronousFileSystem, if it is a 
> better name).  All the APIs in FutureFileSystem are the same as FileSystem 
> except that the return type is wrapped by Future, e.g.
> {code}
>   //FileSystem
>   public boolean rename(Path src, Path dst) throws IOException;
>   //FutureFileSystem
>   public Future rename(Path src, Path dst) throws IOException;
> {code}
> Note that FutureFileSystem does not extend FileSystem.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Commented] (HADOOP-12910) Add new FileSystem API to support asynchronous method calls

2016-06-08 Thread Tsz Wo Nicholas Sze (JIRA)

[ 
https://issues.apache.org/jira/browse/HADOOP-12910?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15320839#comment-15320839
 ] 

Tsz Wo Nicholas Sze commented on HADOOP-12910:
--

> Why the insistence on doing the async twice? Once for branch-2 and then with 
> a totally different API in branch-3? ...

No, they are the same API.  Branch-2 is a simplified version of trunk.

Features such as chaining are attractive, cool, fashionable and I-like-it, 
however, nonessential.  Today we don't have any async API so that users cannot 
possibly do anything to work around it -- when a blocking call is invoked, the 
caller is blocked.  Once we have an async API as simple as returning Future.  
The caller can possibly use other mean or library to do chaining.  Please 
correct me if you disagree.

> Why invent an API (based on AWT experience with mouse-moved listeners ) 
> rather than take on a proven one whose author is trying to help here and 
> whose API surface is considerably less than the CompletableFuture 
> kitchen-sink?

Honestly, I won't dare to claim that I "invented" the API I proposed.  It is 
just a well-known approach, also known as the Observer pattern.  Let's take a 
look of the 
[ListenableFuture|https://github.com/google/guava/blob/master/guava/src/com/google/common/util/concurrent/ListenableFuture.java]
 interface.  It declares a similar method, which is the only new method added 
to Future.
{code}
//ListenableFuture
void addListener(Runnable listener, Executor executor);
{code}
With this simple method, ListenableFuture (with other library) is able to 
support chaining; see 
[ListenableFutureExplained|https://github.com/google/guava/wiki/ListenableFutureExplained].

Yes, you are right that AWT uses this approach for mouse-moved listeners.  Is 
there anything wrong about it?  If I tell that NASA also uses this approach for 
rocket launching, would it make you feel better?

Thanks.

> Add new FileSystem API to support asynchronous method calls
> ---
>
> Key: HADOOP-12910
> URL: https://issues.apache.org/jira/browse/HADOOP-12910
> Project: Hadoop Common
>  Issue Type: New Feature
>  Components: fs
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Xiaobing Zhou
> Attachments: HADOOP-12910-HDFS-9924.000.patch, 
> HADOOP-12910-HDFS-9924.001.patch, HADOOP-12910-HDFS-9924.002.patch
>
>
> Add a new API, namely FutureFileSystem (or AsynchronousFileSystem, if it is a 
> better name).  All the APIs in FutureFileSystem are the same as FileSystem 
> except that the return type is wrapped by Future, e.g.
> {code}
>   //FileSystem
>   public boolean rename(Path src, Path dst) throws IOException;
>   //FutureFileSystem
>   public Future rename(Path src, Path dst) throws IOException;
> {code}
> Note that FutureFileSystem does not extend FileSystem.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Updated] (HADOOP-13227) AsyncCallHandler should use a event driven architecture to handle async calls

2016-06-07 Thread Tsz Wo Nicholas Sze (JIRA)

 [ 
https://issues.apache.org/jira/browse/HADOOP-13227?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Tsz Wo Nicholas Sze updated HADOOP-13227:
-
Attachment: c13227_20160608b.patch

c13227_20160608b.patch: checkEmpty() should be right after remove()

> AsyncCallHandler should use a event driven architecture to handle async calls
> -
>
> Key: HADOOP-13227
> URL: https://issues.apache.org/jira/browse/HADOOP-13227
> Project: Hadoop Common
>  Issue Type: Improvement
>  Components: io, ipc
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
> Attachments: c13227_20160602.patch, c13227_20160606.patch, 
> c13227_20160607.patch, c13227_20160608.patch, c13227_20160608b.patch
>
>
> This JIRA is to address [Jing's 
> comments|https://issues.apache.org/jira/browse/HADOOP-13226?focusedCommentId=15308630&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-15308630]
>  in HADOOP-13226.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Updated] (HADOOP-13227) AsyncCallHandler should use a event driven architecture to handle async calls

2016-06-07 Thread Tsz Wo Nicholas Sze (JIRA)

 [ 
https://issues.apache.org/jira/browse/HADOOP-13227?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Tsz Wo Nicholas Sze updated HADOOP-13227:
-
Attachment: c13227_20160608.patch

c13227_20160608.patch:
- uses RetryDecision ordering to further simplify newRetryInfo.
- checkCalls should check if the queue is empty at the end.

> AsyncCallHandler should use a event driven architecture to handle async calls
> -
>
> Key: HADOOP-13227
> URL: https://issues.apache.org/jira/browse/HADOOP-13227
> Project: Hadoop Common
>  Issue Type: Improvement
>  Components: io, ipc
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
> Attachments: c13227_20160602.patch, c13227_20160606.patch, 
> c13227_20160607.patch, c13227_20160608.patch
>
>
> This JIRA is to address [Jing's 
> comments|https://issues.apache.org/jira/browse/HADOOP-13226?focusedCommentId=15308630&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-15308630]
>  in HADOOP-13226.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Commented] (HADOOP-12910) Add new FileSystem API to support asynchronous method calls

2016-06-07 Thread Tsz Wo Nicholas Sze (JIRA)

[ 
https://issues.apache.org/jira/browse/HADOOP-12910?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15318065#comment-15318065
 ] 

Tsz Wo Nicholas Sze commented on HADOOP-12910:
--

> One of the key things that Deferred and Finagle's Future both enable is 
> composition. This allows you to chain multiple asynchronous operations in a 
> type-safe fashion. ...

Thanks for the comments.  The dilemma here is that we want to use 
CompletableFuture in trunk but it is unavailable in branch-2.  My suggestion 
indeed is that we only provide minimum async support in branch-2 and have full 
support in trunk.  I originally thought that Future was good enough.  However, 
as you can see in the past comments, people want callbacks so that I proposed 
FutureWithCallback.  You are right that it won't support chaining.

> Add new FileSystem API to support asynchronous method calls
> ---
>
> Key: HADOOP-12910
> URL: https://issues.apache.org/jira/browse/HADOOP-12910
> Project: Hadoop Common
>  Issue Type: New Feature
>  Components: fs
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Xiaobing Zhou
> Attachments: HADOOP-12910-HDFS-9924.000.patch, 
> HADOOP-12910-HDFS-9924.001.patch, HADOOP-12910-HDFS-9924.002.patch
>
>
> Add a new API, namely FutureFileSystem (or AsynchronousFileSystem, if it is a 
> better name).  All the APIs in FutureFileSystem are the same as FileSystem 
> except that the return type is wrapped by Future, e.g.
> {code}
>   //FileSystem
>   public boolean rename(Path src, Path dst) throws IOException;
>   //FutureFileSystem
>   public Future rename(Path src, Path dst) throws IOException;
> {code}
> Note that FutureFileSystem does not extend FileSystem.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Commented] (HADOOP-12910) Add new FileSystem API to support asynchronous method calls

2016-06-07 Thread Tsz Wo Nicholas Sze (JIRA)

[ 
https://issues.apache.org/jira/browse/HADOOP-12910?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15318041#comment-15318041
 ] 

Tsz Wo Nicholas Sze commented on HADOOP-12910:
--

> As to your FutureWithCallback, where does this come from? Have you built any 
> event-driven apps with it? At first blush, it is lacking in vocabulary at 
> least when put against Deferred or CompletableFuture. Thanks.

The idea of Callback and FutureWithCallback is similar to Java Event Model 
which is a well-known model that many and many apps are built with it.  I did 
not intent to use it for replacing Deferred or CompletableFuture.  As mentioned 
previously, I propose using CompletableFuture for trunk.  Unfortunately, 
CompletableFuture is not available in branch-2 so that we need something like 
FutureWithCallback to support callbacks.

> Add new FileSystem API to support asynchronous method calls
> ---
>
> Key: HADOOP-12910
> URL: https://issues.apache.org/jira/browse/HADOOP-12910
> Project: Hadoop Common
>  Issue Type: New Feature
>  Components: fs
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Xiaobing Zhou
> Attachments: HADOOP-12910-HDFS-9924.000.patch, 
> HADOOP-12910-HDFS-9924.001.patch, HADOOP-12910-HDFS-9924.002.patch
>
>
> Add a new API, namely FutureFileSystem (or AsynchronousFileSystem, if it is a 
> better name).  All the APIs in FutureFileSystem are the same as FileSystem 
> except that the return type is wrapped by Future, e.g.
> {code}
>   //FileSystem
>   public boolean rename(Path src, Path dst) throws IOException;
>   //FutureFileSystem
>   public Future rename(Path src, Path dst) throws IOException;
> {code}
> Note that FutureFileSystem does not extend FileSystem.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Commented] (HADOOP-12910) Add new FileSystem API to support asynchronous method calls

2016-06-06 Thread Tsz Wo Nicholas Sze (JIRA)

[ 
https://issues.apache.org/jira/browse/HADOOP-12910?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15317739#comment-15317739
 ] 

Tsz Wo Nicholas Sze commented on HADOOP-12910:
--

It seems we all agree that we should use CompletableFuture (a Java 8 class 
which implements both Future and CompletionStage) with callback support as a 
return type in trunk.  Since we are talking about API, we should actually talk 
about interfaces instead of classes.  Therefore, we should return an 
sub-interface of Future and CompletionStage.

For branch-2, There are two possible ways to use Deferred (or ListenableFuture) 
:
# Using it directly (i.e. import the external jar and use 
com.stumbleupon.async.Deferred in the code).  Then we have an external 
dependency.
# Copy & Paste Deferred to Hadoop, say 
org.apache.hadoop.util.concurrent.Deferred.  Then we can the Deferred 
functionality but our Deferred is incompatible of the 
com.stumbleupon.async.Deferred used in the other projects.  Also, it may be 
harder to support CompletableFuture in trunk since we have to support both 
Deferred and CompletableFuture.

Both choices seem undesirable. Therefore I suggest to create our own interface 
to support callbacks for branch-2 as below.
{code}
  public interface Callback {
void processReturnValue(V returnValue);

void handleException(Exception exception);
  }

  // branch-2 return type
  public interface FutureWithCallback extends Future {
void addCallback(Callback callback);
  }
{code}
For trunk, we have
{code}
  // trunk return type
  public interface CompletableFutureWithCallback
  extends FutureWithCallback, CompletionStage {
  }
{code}
Any comments?

> Add new FileSystem API to support asynchronous method calls
> ---
>
> Key: HADOOP-12910
> URL: https://issues.apache.org/jira/browse/HADOOP-12910
> Project: Hadoop Common
>  Issue Type: New Feature
>  Components: fs
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Xiaobing Zhou
> Attachments: HADOOP-12910-HDFS-9924.000.patch, 
> HADOOP-12910-HDFS-9924.001.patch, HADOOP-12910-HDFS-9924.002.patch
>
>
> Add a new API, namely FutureFileSystem (or AsynchronousFileSystem, if it is a 
> better name).  All the APIs in FutureFileSystem are the same as FileSystem 
> except that the return type is wrapped by Future, e.g.
> {code}
>   //FileSystem
>   public boolean rename(Path src, Path dst) throws IOException;
>   //FutureFileSystem
>   public Future rename(Path src, Path dst) throws IOException;
> {code}
> Note that FutureFileSystem does not extend FileSystem.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Commented] (HADOOP-13227) AsyncCallHandler should use a event driven architecture to handle async calls

2016-06-06 Thread Tsz Wo Nicholas Sze (JIRA)

[ 
https://issues.apache.org/jira/browse/HADOOP-13227?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15317646#comment-15317646
 ] 

Tsz Wo Nicholas Sze commented on HADOOP-13227:
--

> Can we use ConcurrentLinkedQueue's iterator for the scanning here? We will 
> not get any ConcurrentModificationException ...

Honestly, I didn't know that it won't throw ConcurrentModificationException in 
this case.  I learned something here.  Thanks a lot!  Here is a new patch.

c13227_20160607.patch

> AsyncCallHandler should use a event driven architecture to handle async calls
> -
>
> Key: HADOOP-13227
> URL: https://issues.apache.org/jira/browse/HADOOP-13227
> Project: Hadoop Common
>  Issue Type: Improvement
>  Components: io, ipc
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
> Attachments: c13227_20160602.patch, c13227_20160606.patch, 
> c13227_20160607.patch
>
>
> This JIRA is to address [Jing's 
> comments|https://issues.apache.org/jira/browse/HADOOP-13226?focusedCommentId=15308630&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-15308630]
>  in HADOOP-13226.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Updated] (HADOOP-13227) AsyncCallHandler should use a event driven architecture to handle async calls

2016-06-06 Thread Tsz Wo Nicholas Sze (JIRA)

 [ 
https://issues.apache.org/jira/browse/HADOOP-13227?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Tsz Wo Nicholas Sze updated HADOOP-13227:
-
Attachment: c13227_20160607.patch

> AsyncCallHandler should use a event driven architecture to handle async calls
> -
>
> Key: HADOOP-13227
> URL: https://issues.apache.org/jira/browse/HADOOP-13227
> Project: Hadoop Common
>  Issue Type: Improvement
>  Components: io, ipc
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
> Attachments: c13227_20160602.patch, c13227_20160606.patch, 
> c13227_20160607.patch
>
>
> This JIRA is to address [Jing's 
> comments|https://issues.apache.org/jira/browse/HADOOP-13226?focusedCommentId=15308630&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-15308630]
>  in HADOOP-13226.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Updated] (HADOOP-13227) AsyncCallHandler should use a event driven architecture to handle async calls

2016-06-06 Thread Tsz Wo Nicholas Sze (JIRA)

 [ 
https://issues.apache.org/jira/browse/HADOOP-13227?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Tsz Wo Nicholas Sze updated HADOOP-13227:
-
Attachment: c13227_20160606.patch

> In RetryInfo#newRetryInfo, looks like failover, fail, and retry are mutual 
> exclusive? ...

It is correct that the end result is mutually exclusive.  However, we need to 
loop all of the actions in order to determine which one to keep.  Indeed, we 
may combine the failover and fail in RetryInfo to a single action.  Let me 
change it.

> ... consider directly using ConcurrentLinkedQueue which utilizes an efficient 
> non-block algorithm. 

It is a good idea.
 
> In checkCalls, do you think we can avoid the poll+offer operations for a 
> not-done-yet call?

I think it is hard to avoid.  We don't want to read the non-head elements from 
the queue since it is an O( n) operation, where n is the size of the queue.  
Poll and offer indeed are cheap for linked queue.  Let me know if you have an 
idea to avoid poll+offer.

Here is a new patch:

c13227_20160606.patch

> AsyncCallHandler should use a event driven architecture to handle async calls
> -
>
> Key: HADOOP-13227
> URL: https://issues.apache.org/jira/browse/HADOOP-13227
> Project: Hadoop Common
>  Issue Type: Improvement
>  Components: io, ipc
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
> Attachments: c13227_20160602.patch, c13227_20160606.patch
>
>
> This JIRA is to address [Jing's 
> comments|https://issues.apache.org/jira/browse/HADOOP-13226?focusedCommentId=15308630&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-15308630]
>  in HADOOP-13226.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Updated] (HADOOP-13226) Support async call retry and failover

2016-06-06 Thread Tsz Wo Nicholas Sze (JIRA)

 [ 
https://issues.apache.org/jira/browse/HADOOP-13226?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Tsz Wo Nicholas Sze updated HADOOP-13226:
-
Fix Version/s: (was: HDFS-9924)
   2.8.0

> Support async call retry and failover
> -
>
> Key: HADOOP-13226
> URL: https://issues.apache.org/jira/browse/HADOOP-13226
> Project: Hadoop Common
>  Issue Type: New Feature
>  Components: io, ipc
>Reporter: Xiaobing Zhou
>Assignee: Tsz Wo Nicholas Sze
> Fix For: 2.8.0
>
> Attachments: h10433_20160524.patch, h10433_20160525.patch, 
> h10433_20160525b.patch, h10433_20160527.patch, h10433_20160528.patch, 
> h10433_20160528c.patch
>
>
> In current Async DFS implementation, file system calls are invoked and 
> returns Future immediately to clients. Clients call Future#get to retrieve 
> final results. Future#get internally invokes a chain of callbacks residing in 
> ClientNamenodeProtocolTranslatorPB, ProtobufRpcEngine and ipc.Client. The 
> callback path bypasses the original retry layer/logic designed for 
> synchronous DFS. This proposes refactoring to make retry also works for Async 
> DFS.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Updated] (HADOOP-12957) Limit the number of outstanding async calls

2016-06-06 Thread Tsz Wo Nicholas Sze (JIRA)

 [ 
https://issues.apache.org/jira/browse/HADOOP-12957?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Tsz Wo Nicholas Sze updated HADOOP-12957:
-
Fix Version/s: (was: HDFS-9924)
   2.8.0

> Limit the number of outstanding async calls
> ---
>
> Key: HADOOP-12957
> URL: https://issues.apache.org/jira/browse/HADOOP-12957
> Project: Hadoop Common
>  Issue Type: Improvement
>  Components: ipc
>Reporter: Xiaobing Zhou
>Assignee: Xiaobing Zhou
> Fix For: 2.8.0
>
> Attachments: HADOOP-12957-HADOOP-12909.000.patch, 
> HADOOP-12957-combo.000.patch, HADOOP-12957.001.patch, HADOOP-12957.002.patch, 
> HADOOP-12957.003.patch, HADOOP-12957.004.patch, HADOOP-12957.005.patch, 
> HADOOP-12957.006.patch, HADOOP-12957.007.patch, HADOOP-12957.008.patch, 
> HADOOP-12957.009.patch, HADOOP-12957.010.patch, HADOOP-12957.011.patch
>
>
> In async RPC, if the callers don't read replies fast enough, the buffer 
> storing replies could be used up. This is to propose limiting the number of 
> outstanding async calls to eliminate the issue.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Updated] (HADOOP-13168) Support Future.get with timeout in ipc async calls

2016-06-06 Thread Tsz Wo Nicholas Sze (JIRA)

 [ 
https://issues.apache.org/jira/browse/HADOOP-13168?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Tsz Wo Nicholas Sze updated HADOOP-13168:
-
Fix Version/s: (was: HDFS-9924)
   2.8.0

> Support Future.get with timeout in ipc async calls
> --
>
> Key: HADOOP-13168
> URL: https://issues.apache.org/jira/browse/HADOOP-13168
> Project: Hadoop Common
>  Issue Type: Improvement
>  Components: ipc
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
> Fix For: 2.8.0
>
> Attachments: c13168_20160517.patch, c13168_20160518.patch, 
> c13168_20160519.patch
>
>
> Currently, the Future returned by ipc async call only support Future.get() 
> but not Future.get(timeout, unit).  We should support the latter as well.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Updated] (HADOOP-13227) AsyncCallHandler should use a event driven architecture to handle async calls

2016-06-03 Thread Tsz Wo Nicholas Sze (JIRA)

 [ 
https://issues.apache.org/jira/browse/HADOOP-13227?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Tsz Wo Nicholas Sze updated HADOOP-13227:
-
Status: Patch Available  (was: In Progress)

> AsyncCallHandler should use a event driven architecture to handle async calls
> -
>
> Key: HADOOP-13227
> URL: https://issues.apache.org/jira/browse/HADOOP-13227
> Project: Hadoop Common
>  Issue Type: Improvement
>  Components: io, ipc
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
> Attachments: c13227_20160602.patch
>
>
> This JIRA is to address [Jing's 
> comments|https://issues.apache.org/jira/browse/HADOOP-13226?focusedCommentId=15308630&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-15308630]
>  in HADOOP-13226.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Work started] (HADOOP-13227) AsyncCallHandler should use a event driven architecture to handle async calls

2016-06-02 Thread Tsz Wo Nicholas Sze (JIRA)

 [ 
https://issues.apache.org/jira/browse/HADOOP-13227?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Work on HADOOP-13227 started by Tsz Wo Nicholas Sze.

> AsyncCallHandler should use a event driven architecture to handle async calls
> -
>
> Key: HADOOP-13227
> URL: https://issues.apache.org/jira/browse/HADOOP-13227
> Project: Hadoop Common
>  Issue Type: Improvement
>  Components: io, ipc
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
> Attachments: c13227_20160602.patch
>
>
> This JIRA is to address [Jing's 
> comments|https://issues.apache.org/jira/browse/HADOOP-13226?focusedCommentId=15308630&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-15308630]
>  in HADOOP-13226.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Work started] (HADOOP-13227) AsyncCallHandler should use a event driven architecture to handle async calls

2016-06-02 Thread Tsz Wo Nicholas Sze (JIRA)

 [ 
https://issues.apache.org/jira/browse/HADOOP-13227?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Work on HADOOP-13227 started by Tsz Wo Nicholas Sze.

> AsyncCallHandler should use a event driven architecture to handle async calls
> -
>
> Key: HADOOP-13227
> URL: https://issues.apache.org/jira/browse/HADOOP-13227
> Project: Hadoop Common
>  Issue Type: Improvement
>  Components: io, ipc
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
> Attachments: c13227_20160602.patch
>
>
> This JIRA is to address [Jing's 
> comments|https://issues.apache.org/jira/browse/HADOOP-13226?focusedCommentId=15308630&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-15308630]
>  in HADOOP-13226.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Updated] (HADOOP-13227) AsyncCallHandler should use a event driven architecture to handle async calls

2016-06-02 Thread Tsz Wo Nicholas Sze (JIRA)

 [ 
https://issues.apache.org/jira/browse/HADOOP-13227?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Tsz Wo Nicholas Sze updated HADOOP-13227:
-
Attachment: c13227_20160602.patch

c13227_20160602.patch: uses an event driven architecture.

After the patch, the running time of TestAsyncHDFSWithHA is decreased from 
around a minute to around 6 seconds in my machine since the unnecessary waits 
are eliminated.

> AsyncCallHandler should use a event driven architecture to handle async calls
> -
>
> Key: HADOOP-13227
> URL: https://issues.apache.org/jira/browse/HADOOP-13227
> Project: Hadoop Common
>  Issue Type: Improvement
>  Components: io, ipc
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
> Attachments: c13227_20160602.patch
>
>
> This JIRA is to address [Jing's 
> comments|https://issues.apache.org/jira/browse/HADOOP-13226?focusedCommentId=15308630&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-15308630]
>  in HADOOP-13226.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Work stopped] (HADOOP-13227) AsyncCallHandler should use a event driven architecture to handle async calls

2016-06-02 Thread Tsz Wo Nicholas Sze (JIRA)

 [ 
https://issues.apache.org/jira/browse/HADOOP-13227?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Work on HADOOP-13227 stopped by Tsz Wo Nicholas Sze.

> AsyncCallHandler should use a event driven architecture to handle async calls
> -
>
> Key: HADOOP-13227
> URL: https://issues.apache.org/jira/browse/HADOOP-13227
> Project: Hadoop Common
>  Issue Type: Improvement
>  Components: io, ipc
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
> Attachments: c13227_20160602.patch
>
>
> This JIRA is to address [Jing's 
> comments|https://issues.apache.org/jira/browse/HADOOP-13226?focusedCommentId=15308630&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-15308630]
>  in HADOOP-13226.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Updated] (HADOOP-13227) AsyncCallHandler should use a event driven architecture to handle async calls

2016-05-31 Thread Tsz Wo Nicholas Sze (JIRA)

 [ 
https://issues.apache.org/jira/browse/HADOOP-13227?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Tsz Wo Nicholas Sze updated HADOOP-13227:
-
Description: This JIRA is to address [Jing 
comments|https://issues.apache.org/jira/browse/HADOOP-13226?focusedCommentId=15308630&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-15308630]
 in HADOOP-13226.  (was: This JIRA is to address Jing comments in HADOOP-13226.)

> AsyncCallHandler should use a event driven architecture to handle async calls
> -
>
> Key: HADOOP-13227
> URL: https://issues.apache.org/jira/browse/HADOOP-13227
> Project: Hadoop Common
>  Issue Type: Improvement
>  Components: io, ipc
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
>
> This JIRA is to address [Jing 
> comments|https://issues.apache.org/jira/browse/HADOOP-13226?focusedCommentId=15308630&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-15308630]
>  in HADOOP-13226.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Updated] (HADOOP-13227) AsyncCallHandler should use a event driven architecture to handle async calls

2016-05-31 Thread Tsz Wo Nicholas Sze (JIRA)

 [ 
https://issues.apache.org/jira/browse/HADOOP-13227?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Tsz Wo Nicholas Sze updated HADOOP-13227:
-
Description: This JIRA is to address [Jing's 
comments|https://issues.apache.org/jira/browse/HADOOP-13226?focusedCommentId=15308630&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-15308630]
 in HADOOP-13226.  (was: This JIRA is to address [Jing 
comments|https://issues.apache.org/jira/browse/HADOOP-13226?focusedCommentId=15308630&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-15308630]
 in HADOOP-13226.)

> AsyncCallHandler should use a event driven architecture to handle async calls
> -
>
> Key: HADOOP-13227
> URL: https://issues.apache.org/jira/browse/HADOOP-13227
> Project: Hadoop Common
>  Issue Type: Improvement
>  Components: io, ipc
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
>
> This JIRA is to address [Jing's 
> comments|https://issues.apache.org/jira/browse/HADOOP-13226?focusedCommentId=15308630&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-15308630]
>  in HADOOP-13226.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Updated] (HADOOP-13227) AsyncCallHandler should use a event driven architecture to handle async calls

2016-05-31 Thread Tsz Wo Nicholas Sze (JIRA)

 [ 
https://issues.apache.org/jira/browse/HADOOP-13227?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Tsz Wo Nicholas Sze updated HADOOP-13227:
-
Description: This JIRA is to address Jing comments in HADOOP-13226.

> AsyncCallHandler should use a event driven architecture to handle async calls
> -
>
> Key: HADOOP-13227
> URL: https://issues.apache.org/jira/browse/HADOOP-13227
> Project: Hadoop Common
>  Issue Type: Improvement
>  Components: io, ipc
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
>
> This JIRA is to address Jing comments in HADOOP-13226.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Issue Comment Deleted] (HADOOP-13227) AsyncCallHandler should use a event driven architecture to handle async calls

2016-05-31 Thread Tsz Wo Nicholas Sze (JIRA)

 [ 
https://issues.apache.org/jira/browse/HADOOP-13227?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Tsz Wo Nicholas Sze updated HADOOP-13227:
-
Comment: was deleted

(was: This JIRA is to address [Jing 
comments|https://issues.apache.org/jira/browse/HADOOP-13226?focusedCommentId=15308630&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-15308630]
 in HADOOP-13226.)

> AsyncCallHandler should use a event driven architecture to handle async calls
> -
>
> Key: HADOOP-13227
> URL: https://issues.apache.org/jira/browse/HADOOP-13227
> Project: Hadoop Common
>  Issue Type: Improvement
>  Components: io, ipc
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
>




--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Updated] (HADOOP-13227) AsyncCallHandler should use a event driven architecture to handle async calls

2016-05-31 Thread Tsz Wo Nicholas Sze (JIRA)

 [ 
https://issues.apache.org/jira/browse/HADOOP-13227?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Tsz Wo Nicholas Sze updated HADOOP-13227:
-

This JIRA is to address [Jing 
comments|https://issues.apache.org/jira/browse/HADOOP-13226?focusedCommentId=15308630&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-15308630]
 in HADOOP-13226.

> AsyncCallHandler should use a event driven architecture to handle async calls
> -
>
> Key: HADOOP-13227
> URL: https://issues.apache.org/jira/browse/HADOOP-13227
> Project: Hadoop Common
>  Issue Type: Improvement
>  Components: io, ipc
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
>




--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Created] (HADOOP-13227) AsyncCallHandler should use a event driven architecture to handle async calls

2016-05-31 Thread Tsz Wo Nicholas Sze (JIRA)
Tsz Wo Nicholas Sze created HADOOP-13227:


 Summary: AsyncCallHandler should use a event driven architecture 
to handle async calls
 Key: HADOOP-13227
 URL: https://issues.apache.org/jira/browse/HADOOP-13227
 Project: Hadoop Common
  Issue Type: Improvement
  Components: io, ipc
Reporter: Tsz Wo Nicholas Sze
Assignee: Tsz Wo Nicholas Sze






--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Updated] (HADOOP-13226) Support async call retry and failover

2016-05-31 Thread Tsz Wo Nicholas Sze (JIRA)

 [ 
https://issues.apache.org/jira/browse/HADOOP-13226?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Tsz Wo Nicholas Sze updated HADOOP-13226:
-
   Resolution: Fixed
Fix Version/s: 2.8.0
   Status: Resolved  (was: Patch Available)

Thanks Jing for reviewing the patches and the great ideas!

I have committed this.

> Support async call retry and failover
> -
>
> Key: HADOOP-13226
> URL: https://issues.apache.org/jira/browse/HADOOP-13226
> Project: Hadoop Common
>  Issue Type: New Feature
>  Components: io, ipc
>Reporter: Xiaobing Zhou
>Assignee: Tsz Wo Nicholas Sze
> Fix For: 2.8.0
>
> Attachments: h10433_20160524.patch, h10433_20160525.patch, 
> h10433_20160525b.patch, h10433_20160527.patch, h10433_20160528.patch, 
> h10433_20160528c.patch
>
>
> In current Async DFS implementation, file system calls are invoked and 
> returns Future immediately to clients. Clients call Future#get to retrieve 
> final results. Future#get internally invokes a chain of callbacks residing in 
> ClientNamenodeProtocolTranslatorPB, ProtobufRpcEngine and ipc.Client. The 
> callback path bypasses the original retry layer/logic designed for 
> synchronous DFS. This proposes refactoring to make retry also works for Async 
> DFS.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Updated] (HADOOP-13146) Refactor RetryInvocationHandler

2016-05-31 Thread Tsz Wo Nicholas Sze (JIRA)

 [ 
https://issues.apache.org/jira/browse/HADOOP-13146?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Tsz Wo Nicholas Sze updated HADOOP-13146:
-
Fix Version/s: (was: 2.9.0)
   2.8.0

Merged to 2.8.

> Refactor RetryInvocationHandler
> ---
>
> Key: HADOOP-13146
> URL: https://issues.apache.org/jira/browse/HADOOP-13146
> Project: Hadoop Common
>  Issue Type: Improvement
>  Components: io
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
>Priority: Minor
> Fix For: 2.8.0
>
> Attachments: c13146_20160513.patch, c13146_20160513b.patch, 
> c13146_20160514.patch, c13146_20160514b.patch, c13146_20160516.patch
>
>
> - The exception handling is quite long.  It is better to refactor it to a 
> separated method.
> - The failover logic and synchronization can be moved to a new inner class.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Updated] (HADOOP-13226) Support async call retry and failover

2016-05-31 Thread Tsz Wo Nicholas Sze (JIRA)

 [ 
https://issues.apache.org/jira/browse/HADOOP-13226?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Tsz Wo Nicholas Sze updated HADOOP-13226:
-
Summary: Support async call retry and failover  (was: Make retry also works 
well for Async DFS)

> Support async call retry and failover
> -
>
> Key: HADOOP-13226
> URL: https://issues.apache.org/jira/browse/HADOOP-13226
> Project: Hadoop Common
>  Issue Type: New Feature
>  Components: io, ipc
>Reporter: Xiaobing Zhou
>Assignee: Tsz Wo Nicholas Sze
> Attachments: h10433_20160524.patch, h10433_20160525.patch, 
> h10433_20160525b.patch, h10433_20160527.patch, h10433_20160528.patch, 
> h10433_20160528c.patch
>
>
> In current Async DFS implementation, file system calls are invoked and 
> returns Future immediately to clients. Clients call Future#get to retrieve 
> final results. Future#get internally invokes a chain of callbacks residing in 
> ClientNamenodeProtocolTranslatorPB, ProtobufRpcEngine and ipc.Client. The 
> callback path bypasses the original retry layer/logic designed for 
> synchronous DFS. This proposes refactoring to make retry also works for Async 
> DFS.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Moved] (HADOOP-13226) Make retry also works well for Async DFS

2016-05-31 Thread Tsz Wo Nicholas Sze (JIRA)

 [ 
https://issues.apache.org/jira/browse/HADOOP-13226?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Tsz Wo Nicholas Sze moved HDFS-10433 to HADOOP-13226:
-

Component/s: (was: hdfs)
 ipc
 io
Key: HADOOP-13226  (was: HDFS-10433)
Project: Hadoop Common  (was: Hadoop HDFS)

> Make retry also works well for Async DFS
> 
>
> Key: HADOOP-13226
> URL: https://issues.apache.org/jira/browse/HADOOP-13226
> Project: Hadoop Common
>  Issue Type: New Feature
>  Components: io, ipc
>Reporter: Xiaobing Zhou
>Assignee: Tsz Wo Nicholas Sze
> Attachments: h10433_20160524.patch, h10433_20160525.patch, 
> h10433_20160525b.patch, h10433_20160527.patch, h10433_20160528.patch, 
> h10433_20160528c.patch
>
>
> In current Async DFS implementation, file system calls are invoked and 
> returns Future immediately to clients. Clients call Future#get to retrieve 
> final results. Future#get internally invokes a chain of callbacks residing in 
> ClientNamenodeProtocolTranslatorPB, ProtobufRpcEngine and ipc.Client. The 
> callback path bypasses the original retry layer/logic designed for 
> synchronous DFS. This proposes refactoring to make retry also works for Async 
> DFS.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Commented] (HADOOP-12910) Add new FileSystem API to support asynchronous method calls

2016-05-29 Thread Tsz Wo Nicholas Sze (JIRA)

[ 
https://issues.apache.org/jira/browse/HADOOP-12910?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15306059#comment-15306059
 ] 

Tsz Wo Nicholas Sze commented on HADOOP-12910:
--

It seems that people really want Future with callback.  I will think about how 
to do it.  Thanks.

> Add new FileSystem API to support asynchronous method calls
> ---
>
> Key: HADOOP-12910
> URL: https://issues.apache.org/jira/browse/HADOOP-12910
> Project: Hadoop Common
>  Issue Type: New Feature
>  Components: fs
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Xiaobing Zhou
> Attachments: HADOOP-12910-HDFS-9924.000.patch, 
> HADOOP-12910-HDFS-9924.001.patch, HADOOP-12910-HDFS-9924.002.patch
>
>
> Add a new API, namely FutureFileSystem (or AsynchronousFileSystem, if it is a 
> better name).  All the APIs in FutureFileSystem are the same as FileSystem 
> except that the return type is wrapped by Future, e.g.
> {code}
>   //FileSystem
>   public boolean rename(Path src, Path dst) throws IOException;
>   //FutureFileSystem
>   public Future rename(Path src, Path dst) throws IOException;
> {code}
> Note that FutureFileSystem does not extend FileSystem.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Commented] (HADOOP-12910) Add new FileSystem API to support asynchronous method calls

2016-05-29 Thread Tsz Wo Nicholas Sze (JIRA)

[ 
https://issues.apache.org/jira/browse/HADOOP-12910?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15306058#comment-15306058
 ] 

Tsz Wo Nicholas Sze commented on HADOOP-12910:
--

It would be less work if we could copy/paste of Deferred.  However, below 
quoted from http://opentsdb.net/faq.html says that ASF does not allow it.
{quote}
Why does OpenTSDB use the LGPL?
...
- The LGPL is perfectly compatible with the ASF2 license. Many people are 
misled to believe that there is an incompatibility because the Apache Software 
Foundation (ASF) decided to not allow inclusion of LGPL'ed code in its own 
projects. This choice only applies to the projects managed by the ASF itself 
and doesn't stem from any license incompability.
{quote}


> Add new FileSystem API to support asynchronous method calls
> ---
>
> Key: HADOOP-12910
> URL: https://issues.apache.org/jira/browse/HADOOP-12910
> Project: Hadoop Common
>  Issue Type: New Feature
>  Components: fs
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Xiaobing Zhou
> Attachments: HADOOP-12910-HDFS-9924.000.patch, 
> HADOOP-12910-HDFS-9924.001.patch, HADOOP-12910-HDFS-9924.002.patch
>
>
> Add a new API, namely FutureFileSystem (or AsynchronousFileSystem, if it is a 
> better name).  All the APIs in FutureFileSystem are the same as FileSystem 
> except that the return type is wrapped by Future, e.g.
> {code}
>   //FileSystem
>   public boolean rename(Path src, Path dst) throws IOException;
>   //FutureFileSystem
>   public Future rename(Path src, Path dst) throws IOException;
> {code}
> Note that FutureFileSystem does not extend FileSystem.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Commented] (HADOOP-12910) Add new FileSystem API to support asynchronous method calls

2016-05-26 Thread Tsz Wo Nicholas Sze (JIRA)

[ 
https://issues.apache.org/jira/browse/HADOOP-12910?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15303008#comment-15303008
 ] 

Tsz Wo Nicholas Sze commented on HADOOP-12910:
--

> Can we introduce our own ListenableFuture class? Or use callbacks instead of 
> Future? Listener or callback is very useful for writing event-driven programs.

It seems like a good idea to have our own Future sub-interface.  

> Use Deferred, ...

I guess it may not be a good idea since it introduces an external dependency at 
API level.

> Target this for 3.0 and use CompletableFuture. ...

Why making the entire async feature unavailable in branch 2?  CompletableFuture 
is nice-to-have but not must-have.

I suggest returning Future (or a sub-interface to support callbacks) in branch 
2 and CompletableFuture (or our own implementation of CompletableFuture) in 
trunk.  In this way, trunk is backward compatible to branch 2 since 
CompletableFuture implements Future.

> Add new FileSystem API to support asynchronous method calls
> ---
>
> Key: HADOOP-12910
> URL: https://issues.apache.org/jira/browse/HADOOP-12910
> Project: Hadoop Common
>  Issue Type: New Feature
>  Components: fs
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Xiaobing Zhou
> Attachments: HADOOP-12910-HDFS-9924.000.patch, 
> HADOOP-12910-HDFS-9924.001.patch, HADOOP-12910-HDFS-9924.002.patch
>
>
> Add a new API, namely FutureFileSystem (or AsynchronousFileSystem, if it is a 
> better name).  All the APIs in FutureFileSystem are the same as FileSystem 
> except that the return type is wrapped by Future, e.g.
> {code}
>   //FileSystem
>   public boolean rename(Path src, Path dst) throws IOException;
>   //FutureFileSystem
>   public Future rename(Path src, Path dst) throws IOException;
> {code}
> Note that FutureFileSystem does not extend FileSystem.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Commented] (HADOOP-12910) Add new FileSystem API to support asynchronous method calls

2016-05-26 Thread Tsz Wo Nicholas Sze (JIRA)

[ 
https://issues.apache.org/jira/browse/HADOOP-12910?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15302850#comment-15302850
 ] 

Tsz Wo Nicholas Sze commented on HADOOP-12910:
--

Okay, let's replay it.

> ... . It's not fair to push the burden of supporting multiple APIs onto our 
> downstreams, ...

We are not going to support multiple APIs. Once we have decided the async API, 
the unstable API can be removed. That is the meaning of "unstable".

The down streams are intelligent people. They can decide whether they want to 
use the unstable API. It is even more unfair if we delay to provide any async 
API to the down streams. No?

[~andrew.wang], is it your intention to slow down the async hdfs development? I 
hope not.

> Add new FileSystem API to support asynchronous method calls
> ---
>
> Key: HADOOP-12910
> URL: https://issues.apache.org/jira/browse/HADOOP-12910
> Project: Hadoop Common
>  Issue Type: New Feature
>  Components: fs
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Xiaobing Zhou
> Attachments: HADOOP-12910-HDFS-9924.000.patch, 
> HADOOP-12910-HDFS-9924.001.patch, HADOOP-12910-HDFS-9924.002.patch
>
>
> Add a new API, namely FutureFileSystem (or AsynchronousFileSystem, if it is a 
> better name).  All the APIs in FutureFileSystem are the same as FileSystem 
> except that the return type is wrapped by Future, e.g.
> {code}
>   //FileSystem
>   public boolean rename(Path src, Path dst) throws IOException;
>   //FutureFileSystem
>   public Future rename(Path src, Path dst) throws IOException;
> {code}
> Note that FutureFileSystem does not extend FileSystem.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Updated] (HADOOP-13168) Support Future.get with timeout in ipc async calls

2016-05-19 Thread Tsz Wo Nicholas Sze (JIRA)

 [ 
https://issues.apache.org/jira/browse/HADOOP-13168?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Tsz Wo Nicholas Sze updated HADOOP-13168:
-
   Resolution: Fixed
 Hadoop Flags: Reviewed
Fix Version/s: 2.8.0
   Status: Resolved  (was: Patch Available)

Thanks Xiaobing and Jing for reviewing the patches.

I have committed this.

> Support Future.get with timeout in ipc async calls
> --
>
> Key: HADOOP-13168
> URL: https://issues.apache.org/jira/browse/HADOOP-13168
> Project: Hadoop Common
>  Issue Type: Improvement
>  Components: ipc
>Reporter: Tsz Wo Nicholas Sze
>Assignee: Tsz Wo Nicholas Sze
> Fix For: 2.8.0
>
> Attachments: c13168_20160517.patch, c13168_20160518.patch, 
> c13168_20160519.patch
>
>
> Currently, the Future returned by ipc async call only support Future.get() 
> but not Future.get(timeout, unit).  We should support the latter as well.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



  1   2   3   4   5   6   7   8   9   10   >