[jira] [Commented] (AIRFLOW-2670) SSHOperator's timeout parameter doesn't affect SSHook timeoot

2018-09-02 Thread Apache Spark (JIRA)


[ 
https://issues.apache.org/jira/browse/AIRFLOW-2670?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16601299#comment-16601299
 ] 

Apache Spark commented on AIRFLOW-2670:
---

User 'Noremac201' has created a pull request for this issue:
https://github.com/apache/incubator-airflow/pull/3553

> SSHOperator's timeout parameter doesn't affect SSHook timeoot
> -
>
> Key: AIRFLOW-2670
> URL: https://issues.apache.org/jira/browse/AIRFLOW-2670
> Project: Apache Airflow
>  Issue Type: Improvement
>  Components: contrib
>Affects Versions: 2.0.0
>Reporter: jin zhang
>Priority: Major
> Fix For: 2.0.0
>
>
> when I use SSHOperator, SSHOperator's timeout parameter can't set in SSHHook 
> and it's just effect exce_command. 
> old version:
> self.ssh_hook = SSHHook(ssh_conn_id=self.ssh_conn_id)
> I change it to :
> self.ssh_hook = SSHHook(ssh_conn_id=self.ssh_conn_id, timeout=self.timeout)



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


[jira] [Commented] (AIRFLOW-2670) SSHOperator's timeout parameter doesn't affect SSHook timeoot

2018-07-31 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/AIRFLOW-2670?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16564247#comment-16564247
 ] 

ASF GitHub Bot commented on AIRFLOW-2670:
-

Fokko closed pull request #3666: [AIRFLOW-2670] Update SSH Operator's Hook to 
respect timeout
URL: https://github.com/apache/incubator-airflow/pull/3666
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git a/airflow/contrib/operators/ssh_operator.py 
b/airflow/contrib/operators/ssh_operator.py
index 2e890f463e..747ad04ff0 100644
--- a/airflow/contrib/operators/ssh_operator.py
+++ b/airflow/contrib/operators/ssh_operator.py
@@ -69,16 +69,17 @@ def __init__(self,
 def execute(self, context):
 try:
 if self.ssh_conn_id and not self.ssh_hook:
-self.ssh_hook = SSHHook(ssh_conn_id=self.ssh_conn_id)
+self.ssh_hook = SSHHook(ssh_conn_id=self.ssh_conn_id,
+timeout=self.timeout)
 
 if not self.ssh_hook:
-raise AirflowException("can not operate without ssh_hook or 
ssh_conn_id")
+raise AirflowException("Cannot operate without ssh_hook or 
ssh_conn_id.")
 
 if self.remote_host is not None:
 self.ssh_hook.remote_host = self.remote_host
 
 if not self.command:
-raise AirflowException("no command specified so nothing to 
execute here.")
+raise AirflowException("SSH command not specified. Aborting.")
 
 with self.ssh_hook.get_conn() as ssh_client:
 # Auto apply tty when its required in case of sudo
diff --git a/tests/contrib/operators/test_ssh_operator.py 
b/tests/contrib/operators/test_ssh_operator.py
index b97ba84a01..7ddd24b2ac 100644
--- a/tests/contrib/operators/test_ssh_operator.py
+++ b/tests/contrib/operators/test_ssh_operator.py
@@ -7,9 +7,9 @@
 # to you under the Apache License, Version 2.0 (the
 # "License"); you may not use this file except in compliance
 # with the License.  You may obtain a copy of the License at
-# 
+#
 #   http://www.apache.org/licenses/LICENSE-2.0
-# 
+#
 # Unless required by applicable law or agreed to in writing,
 # software distributed under the License is distributed on an
 # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
@@ -58,6 +58,23 @@ def setUp(self):
 self.hook = hook
 self.dag = dag
 
+def test_hook_created_correctly(self):
+TIMEOUT = 20
+SSH_ID = "ssh_default"
+task = SSHOperator(
+task_id="test",
+command="echo -n airflow",
+dag=self.dag,
+timeout=TIMEOUT,
+ssh_conn_id="ssh_default"
+)
+self.assertIsNotNone(task)
+
+task.execute(None)
+
+self.assertEquals(TIMEOUT, task.ssh_hook.timeout)
+self.assertEquals(SSH_ID, task.ssh_hook.ssh_conn_id)
+
 def test_json_command_execution(self):
 configuration.conf.set("core", "enable_xcom_pickling", "False")
 task = SSHOperator(


 


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> SSHOperator's timeout parameter doesn't affect SSHook timeoot
> -
>
> Key: AIRFLOW-2670
> URL: https://issues.apache.org/jira/browse/AIRFLOW-2670
> Project: Apache Airflow
>  Issue Type: Improvement
>  Components: contrib
>Affects Versions: Airflow 2.0
>Reporter: jin zhang
>Priority: Major
>
> when I use SSHOperator, SSHOperator's timeout parameter can't set in SSHHook 
> and it's just effect exce_command. 
> old version:
> self.ssh_hook = SSHHook(ssh_conn_id=self.ssh_conn_id)
> I change it to :
> self.ssh_hook = SSHHook(ssh_conn_id=self.ssh_conn_id, timeout=self.timeout)



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


[jira] [Commented] (AIRFLOW-2670) SSHOperator's timeout parameter doesn't affect SSHook timeoot

2018-07-31 Thread ASF subversion and git services (JIRA)


[ 
https://issues.apache.org/jira/browse/AIRFLOW-2670?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16564248#comment-16564248
 ] 

ASF subversion and git services commented on AIRFLOW-2670:
--

Commit 3b35d360f6ff8694b6fb4387901c182ca39160b5 in incubator-airflow's branch 
refs/heads/master from [~noremac201]
[ https://gitbox.apache.org/repos/asf?p=incubator-airflow.git;h=3b35d36 ]

[AIRFLOW-2670] Update SSH Operator's Hook to respect timeout (#3666)



> SSHOperator's timeout parameter doesn't affect SSHook timeoot
> -
>
> Key: AIRFLOW-2670
> URL: https://issues.apache.org/jira/browse/AIRFLOW-2670
> Project: Apache Airflow
>  Issue Type: Improvement
>  Components: contrib
>Affects Versions: Airflow 2.0
>Reporter: jin zhang
>Priority: Major
>
> when I use SSHOperator, SSHOperator's timeout parameter can't set in SSHHook 
> and it's just effect exce_command. 
> old version:
> self.ssh_hook = SSHHook(ssh_conn_id=self.ssh_conn_id)
> I change it to :
> self.ssh_hook = SSHHook(ssh_conn_id=self.ssh_conn_id, timeout=self.timeout)



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


[jira] [Commented] (AIRFLOW-2670) SSHOperator's timeout parameter doesn't affect SSHook timeoot

2018-07-31 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/AIRFLOW-2670?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16564246#comment-16564246
 ] 

ASF GitHub Bot commented on AIRFLOW-2670:
-

Fokko commented on issue #3666: [AIRFLOW-2670] Update SSH Operator's Hook to 
respect timeout
URL: 
https://github.com/apache/incubator-airflow/pull/3666#issuecomment-409338606
 
 
   Nice one @Noremac201 Thanks


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> SSHOperator's timeout parameter doesn't affect SSHook timeoot
> -
>
> Key: AIRFLOW-2670
> URL: https://issues.apache.org/jira/browse/AIRFLOW-2670
> Project: Apache Airflow
>  Issue Type: Improvement
>  Components: contrib
>Affects Versions: Airflow 2.0
>Reporter: jin zhang
>Priority: Major
>
> when I use SSHOperator, SSHOperator's timeout parameter can't set in SSHHook 
> and it's just effect exce_command. 
> old version:
> self.ssh_hook = SSHHook(ssh_conn_id=self.ssh_conn_id)
> I change it to :
> self.ssh_hook = SSHHook(ssh_conn_id=self.ssh_conn_id, timeout=self.timeout)



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


[jira] [Commented] (AIRFLOW-2670) SSHOperator's timeout parameter doesn't affect SSHook timeoot

2018-07-30 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/AIRFLOW-2670?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16562663#comment-16562663
 ] 

ASF GitHub Bot commented on AIRFLOW-2670:
-

codecov-io commented on issue #3666: [AIRFLOW-2670] Update SSH Operator's Hook 
to respect timeout
URL: 
https://github.com/apache/incubator-airflow/pull/3666#issuecomment-409045376
 
 
   # 
[Codecov](https://codecov.io/gh/apache/incubator-airflow/pull/3666?src=pr=h1)
 Report
   > Merging 
[#3666](https://codecov.io/gh/apache/incubator-airflow/pull/3666?src=pr=desc)
 into 
[master](https://codecov.io/gh/apache/incubator-airflow/commit/dfa7b26ddaca80ee8fd9915ee9f6eac50fac77f6?src=pr=desc)
 will **not change** coverage.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree 
graph](https://codecov.io/gh/apache/incubator-airflow/pull/3666/graphs/tree.svg?height=150=650=WdLKlKHOAU=pr)](https://codecov.io/gh/apache/incubator-airflow/pull/3666?src=pr=tree)
   
   ```diff
   @@   Coverage Diff   @@
   ##   master#3666   +/-   ##
   ===
 Coverage   77.51%   77.51%   
   ===
 Files 205  205   
 Lines   1575115751   
   ===
 Hits1221012210   
 Misses   3541 3541
   ```
   
   
   
   --
   
   [Continue to review full report at 
Codecov](https://codecov.io/gh/apache/incubator-airflow/pull/3666?src=pr=continue).
   > **Legend** - [Click here to learn 
more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute  (impact)`, `ø = not affected`, `? = missing data`
   > Powered by 
[Codecov](https://codecov.io/gh/apache/incubator-airflow/pull/3666?src=pr=footer).
 Last update 
[dfa7b26...42b907c](https://codecov.io/gh/apache/incubator-airflow/pull/3666?src=pr=lastupdated).
 Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> SSHOperator's timeout parameter doesn't affect SSHook timeoot
> -
>
> Key: AIRFLOW-2670
> URL: https://issues.apache.org/jira/browse/AIRFLOW-2670
> Project: Apache Airflow
>  Issue Type: Improvement
>  Components: contrib
>Affects Versions: Airflow 2.0
>Reporter: jin zhang
>Priority: Major
>
> when I use SSHOperator, SSHOperator's timeout parameter can't set in SSHHook 
> and it's just effect exce_command. 
> old version:
> self.ssh_hook = SSHHook(ssh_conn_id=self.ssh_conn_id)
> I change it to :
> self.ssh_hook = SSHHook(ssh_conn_id=self.ssh_conn_id, timeout=self.timeout)



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


[jira] [Commented] (AIRFLOW-2670) SSHOperator's timeout parameter doesn't affect SSHook timeoot

2018-07-30 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/AIRFLOW-2670?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16562541#comment-16562541
 ] 

ASF GitHub Bot commented on AIRFLOW-2670:
-

Noremac201 opened a new pull request #3666: [AIRFLOW-2670] Update SSH 
Operator's Hook to respect timeout
URL: https://github.com/apache/incubator-airflow/pull/3666
 
 
   ### JIRA
   - [x] My PR addresses the following [Airflow 
JIRA](https://issues.apache.org/jira/browse/AIRFLOW/) issues and references 
them in the PR title. For example, "\[AIRFLOW-XXX\] My Airflow PR"
   - https://issues.apache.org/jira/browse/AIRFLOW-2670
   
   ### Description
   - [x] Here are some details about my PR, including screenshots of any UI 
changes:
   
   Previously the SSH operator was not respecting the passed in timeout to the 
operator. Changed the Operator to pass the timeout to hook, as well as add a 
test to make sure the hook is being created correctly.
   
   Extension of #3553, mistakenly closed after I thought it was fixed elsewhere.
   
   ### Commits
   - [x] My commits all reference JIRA issues in their subject lines, and I 
have squashed multiple commits if they address the same issue. In addition, my 
commits follow the guidelines from "[How to write a good git commit 
message](http://chris.beams.io/posts/git-commit/)":
   1. Subject is separated from body by a blank line
   2. Subject is limited to 50 characters
   3. Subject does not end with a period
   4. Subject uses the imperative mood ("add", not "adding")
   5. Body wraps at 72 characters
   6. Body explains "what" and "why", not "how"
   
   
   ### Code Quality
   - [x] Passes `git diff upstream/master -u -- "*.py" | flake8 --diff`


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> SSHOperator's timeout parameter doesn't affect SSHook timeoot
> -
>
> Key: AIRFLOW-2670
> URL: https://issues.apache.org/jira/browse/AIRFLOW-2670
> Project: Apache Airflow
>  Issue Type: Improvement
>  Components: contrib
>Affects Versions: Airflow 2.0
>Reporter: jin zhang
>Priority: Major
>
> when I use SSHOperator, SSHOperator's timeout parameter can't set in SSHHook 
> and it's just effect exce_command. 
> old version:
> self.ssh_hook = SSHHook(ssh_conn_id=self.ssh_conn_id)
> I change it to :
> self.ssh_hook = SSHHook(ssh_conn_id=self.ssh_conn_id, timeout=self.timeout)



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