Re: Remove airflow from pypi

2018-11-27 Thread George Leslie-Waksman
rror('Please install package apache-airflow instead > > of airflow') > > > > RuntimeError: Please install package apache-airflow instead of airflow > > > > > > > > ---- > > > > Command "python

Re: Remove airflow from pypi

2018-11-24 Thread George Leslie-Waksman
It's probably a good idea to put something at "airflow", even if it just fails to install and tells people to install apache-airflow instead. If not, there's a risk someone squats the name airflow and puts up something malicious. --George On Fri, Nov 23, 2018 at 11:44 AM Driesprong, Fokko wrote:

Re: error handling flow in DAG

2018-10-10 Thread George Leslie-Waksman
This is a great use case for the all_success and one_failed trigger rules. If we have "--S-->" be a dependency where the downstream has all_success trigger rule, and "--F-->" be a dependency where the downstream has one_failed as the trigger rule, you can do what you want with a DAG of the form:

Re: Pinning dependencies for Apache Airflow

2018-10-10 Thread George Leslie-Waksman
duation > > > > from Incubation fwiw - it's an unavoidable artefact of the > open-source > > > > world we develop in. > > > > > > > > > > > > > > https://emea01.safelinks.protection.outlook.com/?url=https%3A%2F%2Flibraries.i

Re: Pinning dependencies for Apache Airflow

2018-10-08 Thread George Leslie-Waksman
do much of the work for this, if it can help avoid pinning all of the depends at the installation level. --George Leslie-Waksman On Sun, Oct 7, 2018 at 1:26 PM Maxime Beauchemin wrote: > > pip-tools can definitely help here to ship a reference [locked] > `requirements.txt` that can be

Re: execution_date - can we stop the confusion?

2018-09-27 Thread George Leslie-Waksman
> run by Airflow. > > > > What is actually desired here? > > - The right bound of the schedule interval? > > - The time the DagRun was created? > > - The time that any Tasks associated with a DagRun were first considered > > by the scheduler? > > - T

Re: execution_date - can we stop the confusion?

2018-09-26 Thread George Leslie-Waksman
This comes up a lot. I've seen it on this mailing list multiple times and it's something that I have to explicitly call out to every single person that I've helped train up on Airflow. If we take a moment to set aside why things are the way they are, what the documentation says, and how experience

Re: Guidelines on Contrib vs Non-contrib

2018-09-17 Thread George Leslie-Waksman
Given we have a plugin system, could we alternatively move away from keeping non-core supported code outside of the core project/repo? It would hugely decrease the surface area of the main repository and testing infrastructure to get most of the contrib code out to its own place. Further, it woul

Re: It's very hard to become a committer on the project

2018-09-17 Thread George Leslie-Waksman
Are there Apache rules preventing us from switching to GitHub Issues? That seems like it might better fit much of Airflow's user base. On Sun, Sep 16, 2018, 9:21 AM Jeff Payne wrote: > I agree that Jira could be better utilized. I read the original > conversation on the mailing list about how

Re: Plan to change type of dag_id from String to Number?

2018-08-16 Thread George Leslie-Waksman
These performance characteristics are metadata database backend dependent as well. If there are benchmarks, I would hope we look at them across sqlite, mysql, postgresql, and any other supported backends before we take action. On Thu, Aug 9, 2018 at 12:41 PM Maxime Beauchemin < maximebeauche...@gm

Re: [VOTE] Airflow 1.10.0rc3

2018-08-08 Thread George Leslie-Waksman
u upgrading to 1.9 first? And see if that helps? > > -ash > > > On 8 Aug 2018, at 00:07, George Leslie-Waksman <mailto:waks...@gmail.com>> wrote: > > > > We just tried to upgrade a 1.8.1 install to 1.10rc3 and ran into a > critical > > error on ale

Re: [VOTE] Airflow 1.10.0rc3

2018-08-07 Thread George Leslie-Waksman
We just tried to upgrade a 1.8.1 install to 1.10rc3 and ran into a critical error on alembic migration execution. I have captured the issue in JIRA: https://issues.apache.org/jira/browse/AIRFLOW-2870 I would consider this a critical blocker for release because it hard blocks upgrading. George On

Re: Catchup By default = False vs LatestOnlyOperator

2018-07-23 Thread George Leslie-Waksman
pdate production services with it as soon as it > is available; we might occasionally want to re-run historical days, in > which case we want to run the same DAG but without interacting with live > production services at all. > > On Mon, Jul 23, 2018 at 2:18 PM, George Leslie-Waksman

Re: Catchup By default = False vs LatestOnlyOperator

2018-07-23 Thread George Leslie-Waksman
As the author of LatestOnlyOperator, the goal was as a stopgap until catchup=False landed. There are some (very) fringe use cases where you might still want LatestOnlyOperator but in almost all cases what you want is probably catchup=False. The situations where LatestOnlyOperator is still useful

Re: Sep Airflow Bay Area Meetup @ Google

2018-07-20 Thread George Leslie-Waksman
cheduler tuning talk simmering in the back of my mind. Thanks, George Leslie-Waksman On Fri, Jul 20, 2018 at 3:06 AM Naik Kaxil wrote: > Hi Feng, > > Will the session be recorded? Will love if it is. :) > > On 19/07/2018, 22:26, "Feng Lu" wrote: > > Hi

Re: [DISCUSS] AIP - Time for Airflow Improvement Proposals?

2018-07-10 Thread George Leslie-Waksman
+1 On Tue, Jul 10, 2018 at 11:50 AM Jakob Homan wrote: > Lots of Apache projects use ?IPs - Whatever Improvement Proposal - to > document and gather consensus on large changes to the code base. Some > examples: >* Kafka Improvement Proposals (KIP) - > > https://cwiki.apache.org/confluence/d

Re: Concurrency Settings for Celery Executor

2018-06-20 Thread George Leslie-Waksman
"celeryd_concurrency" and "parallelism" serve different purposes "celeryd_concurrency" determines how many worker subprocesses will be spun up on an airflow worker instance (how many concurrent tasks per machine) "parallelism" determines how many tasks airflow will schedule at once; running and q

Re: Managed Apache Airflow Service on Google Cloud Platform

2018-05-07 Thread George Leslie-Waksman
Wow! That's awesome! On Thu, May 3, 2018 at 12:28 PM Chris Riccomini wrote: > Awesome work, Feng! Very cool. :) > > On Tue, May 1, 2018 at 10:10 PM, Feng Lu > wrote: > > > Thank you all and looking forward to the many collaborations to come! > > > > On Tue, May 1, 2018, 17:37 Alex Tronchin-Jame

Re: What are the advantages of plugins, not sure I see any?

2018-03-29 Thread George Leslie-Waksman
aybe the benefit > only kicks in for other airflow abstractions. > > > On Mar 29, 2018, at 7:15 PM, George Leslie-Waksman < > geo...@cloverhealth.com.INVALID> wrote: > > > > We also import our operators and sensors directly. > > > > However, executors and some

Re: What are the advantages of plugins, not sure I see any?

2018-03-29 Thread George Leslie-Waksman
We also import our operators and sensors directly. However, executors and some other pieces are a little bit harder to deal with as non-plugins On Thu, Mar 29, 2018 at 3:56 PM Kyle Hamlin wrote: > Hello, > > I just got done writing a few plugins, and the process has left me > wondering what the

Re: Very looong py files

2018-03-14 Thread George Leslie-Waksman
This has been something that I've wanted to see for a while but as Fokko mentions, it can be hard for very large files that see lots of PRs. Probably easiest is to pull them apart one class at a time to minimize merge conflict risks. Additionally, in some cases, there are interdependencies that ma

Re: [VOTE] Migrate to Github as primary repo (a.k.a. Gitbox)

2018-03-14 Thread George Leslie-Waksman
+1 (non-binding) On Mon, Mar 12, 2018 at 11:51 AM Shah Altaf wrote: > +1 (non binding) > > @Beau Barker - correct me if I'm wrong - I believe that issues will still > remain on Jira. Github will be used just for source control and pull > requests. In the Kylin example, the commit messages cont

Re: Apache Airflow @ QCon AI

2018-01-30 Thread George Leslie-Waksman
I can't because I have a kid due right in the middle of QCon but I'm sure that I could recruit one of my co-workers, if you'd like. --George On Sun, Jan 28, 2018 at 10:34 AM Sid Anand wrote: > Anyone interested in giving a 10-minute intro to Apache Airflow at > https://qcon.ai/ > > I have a mic

Re: Q1 Airflow Bay Area Meetup

2018-01-30 Thread George Leslie-Waksman
t; > > > > > B. > > > > > > > > Op 9 jan. 2018 5:36 a.m. schreef "Ananth Durai" >: > > > > > > > > I can give a talk about all the hacks we did to scale Airflow Local > > > > Executor and im

Re: AttributeError: 'NoneType' object has no attribute 'tzinfo'

2018-01-16 Thread George Leslie-Waksman
Seems like a reasonable PR to me. On Tue, Jan 16, 2018 at 3:20 PM Ruslan Dautkhanov wrote: > Fix: > > vi "/opt/airflow/airflow-20180116/src/apache-airflow/airflow/models.py" > :2951 > changed > self.timezone = self.default_args['start_date'].tzinfo > to > if self.default_

Re: Documentation error

2018-01-16 Thread George Leslie-Waksman
Yes, that documentation appears to be inconsistent. Would you mind opening an issue in jira: https://issues.apache.org/jira/projects/AIRFLOW/issues and submitting a PR to fix it? Thanks, George On Wed, Jan 10, 2018 at 5:18 PM Kewei Shang wrote: > Hi, > > May I ask if the following dag definiti

Re: Airflow 1.10

2018-01-16 Thread George Leslie-Waksman
+1 On Tue, Jan 16, 2018 at 11:08 AM Joy Gao wrote: > The new FAB UI does not modify the existing API (i.e. www2/api/ will be a > copy of www/api/), and the endpoints are registered as blueprints to the > flask app the same way as before, so it is fully backward compatible. > > Although FAB offer

Re: Airflow 2.0

2018-01-08 Thread George Leslie-Waksman
> > Dropping stuff would lessen the burden of maintenance. There is a lot of > > cruft that is not used. This reduces the surface for bugs and makes it > > easier to do the optimizations you refer too. However, I’m not married > to a > > 2.0 release now, but it does look

Re: Backwards compability - what do we mean? when? how long?

2018-01-08 Thread George Leslie-Waksman
I'm glad to see us start having this discussion. Thank you, Ash. 1) +1 to using SemVer. 2) I'm a strong Yes to Hooks and Operators, moderate on log output, and otherwise pretty weak in terms of the guarantees we should provide. That said, I think some of this is made more problematic for us by A

Re: Q1 Airflow Bay Area Meetup

2018-01-08 Thread George Leslie-Waksman
+1 and would love to hear about other folks running Airflow in GCP. Would strongly prefer March (but I'm just one person) On Mon, Jan 8, 2018 at 2:30 PM Feng Lu wrote: > +1 Joy! > > Would prefer April if possible, I can talk about running Airflow in GCP if > there's sufficient interest. > > On

Re: Airflow 2.0

2017-12-18 Thread George Leslie-Waksman
I really do not think we should drop sqlite support. We use sqlite for local testing/development; I used it when doing my initial evaluation of Airflow; and there are regular comments in this group about people using sqlite locally for their workflows. It feels like a critical feature for a lot of

Re: Enabling flake8

2017-11-27 Thread George Leslie-Waksman
+1; thank you Bolke! On Mon, Nov 27, 2017 at 7:48 AM Driesprong, Fokko wrote: > Hi Bolke, > > Great initiative. This should also help improve the quality of the PR's. > Let me know if you need any help. > > Cheers, Fokko > > 2017-11-27 16:47 GMT+01:00 Bolke de Bruin : > > > Hi All, > > > > I jus

Re: Making Airflow Timezone aware

2017-11-15 Thread George Leslie-Waksman
Really happy to hear this moving forward. Thanks Bolke! On Tue, Nov 14, 2017 at 7:44 AM Bolke de Bruin wrote: > See inline answers below. > > Verstuurd vanaf mijn iPad > > > Op 14 nov. 2017 om 16:33 heeft Heistermann, Till < > till.heisterm...@blue-yonder.com> het volgende geschreven: > > > > Hi

Re: Apache Airflow Feature request

2017-10-27 Thread George Leslie-Waksman
Hello Saiprasad, I'm glad to hear about your interest in Airflow. The focus of Airflow development is on better support for configuration as code based workflow development. For a lot of Airflow users, NOT using a drag+drop UI to configure pipelines is one of the major selling points. One could c

Re: Moving on to Lyft

2017-10-27 Thread George Leslie-Waksman
Congrats! On Thu, Oct 26, 2017 at 2:43 PM Daniel Imberman wrote: > Congratulations, Max! > > On Thu, Oct 26, 2017 at 9:57 AM Arthur Wiedmer > wrote: > > > Congratulations, Max! > > > > > > > > On Thu, Oct 26, 2017 at 8:25 AM, Chris Riccomini > > wrote: > > > > > Congrats, Max! > > > > > > On T

Re: Bad Request CSRF

2017-08-17 Thread George Leslie-Waksman
time-to-time and we also > have a single sign-on system. > > On Wed, Aug 16, 2017 at 10:29 AM, George Leslie-Waksman < > geo...@cloverhealth.com.invalid> wrote: > > > I have further tracked the issue to our new single-sign-on system. > Airflow > > is fine. Pleas

Re: Bad Request CSRF

2017-08-16 Thread George Leslie-Waksman
I have further tracked the issue to our new single-sign-on system. Airflow is fine. Please disregard. On Wed, Aug 16, 2017 at 9:15 AM Chris Riccomini wrote: > Try > > pip install --upgrade flask-wtf > > On Tue, Aug 15, 2017 at 4:10 PM, George Leslie-Waksman < > geo...@clo

Bad Request CSRF

2017-08-15 Thread George Leslie-Waksman
I'm seeing "Bad Request \n CSRF token missing or incorrect." when attempting to clear things from the Task Instance interface in Airflow 1.8.1. Is anyone else seeing this or is this more likely something on our end? --George

Re: Task partitioning using Airflow

2017-08-09 Thread George Leslie-Waksman
Airflow is best for situations where you want to run different tasks that depend on each other or process data that arrives over time. If your goal is to take a large dataset, split it up, and process chunks of it, there are probably other tools better suited to your purpose. Off the top of my hea

Re: [VOTE] Release Airflow 1.8.2 based on Airflow 1.8.2 RC4

2017-08-09 Thread George Leslie-Waksman
We've been having a lot of problems with 1.8.1 because of a bug in the DagRun deadlock detection, reported under AIRFLOW-1420 and AIRFLOW-1473. I have a bugfix PR out: https://github.com/apache/incubator-airflow/pull/2506 I would love to see this bugfix make it in before work starts on new featur

Re: Completed tasks not being marked as completed

2017-08-04 Thread George Leslie-Waksman
Pretty sure (not 100%) what is happening is: 1. Scheduler bugs result in task getting scheduled twice 2. Worker 1 grabs task 3. Worker 2 grabs task 4. Worker 1 starts task 5. Worker 2 starts task 6. Worker 2 sees that Worker 1 has started and plans to abort 7. Worker 1 finishe

Re: Tasks stay queued when they fail in celery

2017-08-04 Thread George Leslie-Waksman
We've seen this before as well, it's a bug in the Celery Executor that has a bunch of different manifestations. There is at least one open issue relating to this bug: https://issues.apache.org/jira/browse/AIRFLOW-1463 I have been working on a fix but it's likely to be a few more days before I hav

Re: [VOTE] Release Airflow 1.8.2 based on Airflow 1.8.2 RC2

2017-07-25 Thread George Leslie-Waksman
m> wrote: > [AIRFLOW-1296] is part of 1.8.2. > > Is this a dealbreaker for 1.8.2? > > Max > > On Tue, Jul 25, 2017 at 2:40 PM, George Leslie-Waksman < > geo...@cloverhealth.com.invalid> wrote: > > > I hope that it's not too late for me to chime i

Re: chrome tab cpu usage

2017-07-25 Thread George Leslie-Waksman
We have also observed this. Since upgrading to Airflow 1.8.1, I frequently find Chrome to be eating ~500% CPU. Closing the tab with Airflow open drop CPU back to normal. On Fri, Jul 14, 2017 at 9:06 AM Adrian Bridgett wrote: > For the last few weeks chrome starts to use 100% when the airflow ta

Re: [VOTE] Release Airflow 1.8.2 based on Airflow 1.8.2 RC2

2017-07-25 Thread George Leslie-Waksman
I hope that it's not too late for me to chime in but there is a breaking change in the behavior of LatestOnlyOperator. The change was introduced in PR: https://github.com/apache/incubator-airflow/pull/2365 Change: 333e0b3 [AIRFLOW-1296] Propagate SKIPPED to all downstream tasks Prior to this chan

Anyone else going to SustainOSS

2017-06-07 Thread George Leslie-Waksman
I'm planning to attend Sustain: https://sustainoss.org/ Sustain is in SF and bills itself as "A one day conversation for Open Source Software sustainers". Is there anyone else that's planning to attend and would like to meet up? --George

Re: How to handle this case "Another instance is running, skipping" ?

2017-05-26 Thread George Leslie-Waksman
Airflow v1.7.1.3 > I am running two AWS instances. > Both are running celery worker and one is running webserver+scheduler (so, > only one scheduler). > > My scheduler is not set to restart periodically (should I ? how often to > restart it ?) > > -Jason > > > On T

Re: How to handle this case "Another instance is running, skipping" ?

2017-05-25 Thread George Leslie-Waksman
That is the expected behavior. What happened is two worker processes each grabbed a request of TaskB1, but one of them noticed, and left it to the other one. There are a handful of reasons it might be showing up in your logs. Which version of Airflow are you running? Is your scheduler set to rest

Re: Too many task instances

2017-05-17 Thread George Leslie-Waksman
We're sitting at over 2.4M task instances in our metadata db without much trouble. Have you seen substantial performance degradation or are you just worried about the future possibility? On Wed, Apr 19, 2017 at 12:23 PM Maxime Beauchemin < maximebeauche...@gmail.com> wrote: > You can archive the

Airflow Meetup 1Q17 Talk Videos

2017-02-06 Thread George Leslie-Waksman
Video of the meetup talks and subsequent Q&As is now on YouTube: https://www.youtube.com/watch?v=P0GYZXR0YP4

Airflow Meetup Slides and testing sample code

2017-01-13 Thread George Leslie-Waksman
Since there seemed to be a lot of interest in how we're using pytest fixtures to test dags and postgres, I have thrown together a minimal repo with some code samples that show how we do some of our testing: https://github.com/CloverHealth/aftest_examples I have also included the slides from my tal

Re: Last minute open meetup speaking slot

2017-01-10 Thread George Leslie-Waksman
running in production at > Airbnb. > > Best, > Arthur. > > On Jan 10, 2017 8:03 PM, "George Leslie-Waksman" > wrote: > > 20 minutes. If you want to fill in, that would be great. > > Regards, > --George > > On Tue, Jan 10, 2017 at 4:56 PM Arthur

Re: Last minute open meetup speaking slot

2017-01-10 Thread George Leslie-Waksman
t Airbnb on top of Airflow. > > Best, > Arthur > > On Tue, Jan 10, 2017 at 4:36 PM, George Leslie-Waksman < > geo...@cloverhealth.com.invalid> wrote: > > > One of the speakers for tomorrow's meetup has come down with a cold. > > > > Is there anyone that

Last minute open meetup speaking slot

2017-01-10 Thread George Leslie-Waksman
One of the speakers for tomorrow's meetup has come down with a cold. Is there anyone that would like to claim the third time slot? If not, we'll have extra time for Q&A, updates, and general meeting-up. --George

Re: Refactoring Connection

2017-01-09 Thread George Leslie-Waksman
Could registering new types be handled through the plugin infrastructure? On Mon, Jan 9, 2017 at 5:14 AM Alex Van Boxel wrote: > I was actually going to propose something different with entry-points, but > your requirement beat me to it (but that's ok :-). Actually I think with > this mechanism

Re: Q1 Meetup

2017-01-05 Thread George Leslie-Waksman
Will do. On Wed, Jan 4, 2017 at 4:00 PM siddharth anand wrote: > Dan, Kevin, > Please send George your updated talk titles and abstracts. George, pls > update the meetup once you have them. > > > https://www.meetup.com/Bay-Area-Apache-Airflow-Incubating-Meetup/events/235259523/ > > -s >

Re: Subsequent Airflow Meetup: 2017/01/11

2017-01-04 Thread George Leslie-Waksman
k. Thanks, > > Kevin Mandich > > On Tue, Jan 3, 2017 at 5:40 AM, Dan Davydov .invalid> > wrote: > > > Confirmed. > > > > On Sun, Jan 1, 2017 at 9:16 PM, George Leslie-Waksman < > > geo...@cloverhealth.com.invalid> wrote: > > > > > Sorr

Re: Python 3

2017-01-01 Thread George Leslie-Waksman
We are using Airflow with Python3, PostgreSQL and Redis; it works just fine. On Thu, Dec 1, 2016 at 10:30 AM Maycock, Luke < luke.mayc...@affiliate.oliverwyman.com> wrote: > Thanks Dennis. We will perhaps try Redis as there has been a ticket to > make librabbitmq Python 3 compatible since 2012. >

Re: New trigger_rule

2017-01-01 Thread George Leslie-Waksman
The existing trigger_rule code is a little inflexible but you can cobble together what you want using the DummyOperator and various trigger rules. Say you have Tasks A, B, C, D, E and you want another task F to run when they are all_done but without failure: dummy1 = DummyOperator(..., trigger_ru

Re: Airflow 2.0

2017-01-01 Thread George Leslie-Waksman
There is a bit of a numbering question that we seem to be assuming an answer to when we talk about 2.0; does 2.0 *necessarily* follow 1.9, or will there be a 1.10, perhaps 1.11, etc. A lot of the things that folks are proposing for 2.0 wouldn't really break the world for people on 1.8, so maybe th

Re: Subsequent Airflow Meetup: 2017/01/11

2017-01-01 Thread George Leslie-Waksman
to production. > > > > > > > > We currently use Airflow to build models from our telemetry data > which > > > are > > > > then used for scoring in our near-real-time pipeline. I'd like to > talk > > > > about some of the DAGs we've set up to do t

Subsequent Airflow Meetup: 2017/01/11

2016-10-31 Thread George Leslie-Waksman
folks sometime after Thanksgiving and before the December holidays. --George Leslie-Waksman

Re: String formatting

2016-10-26 Thread George Leslie-Waksman
There are a few layers of things going on here. The first, and the reason for the landscape warning, is that logger performs string formatting lazily and doing the formatting explicitly results in extra processing for log levels that are being ignored. The second is that the logger can be configu

Re: 1.7.0: sqlalchemy version

2016-10-26 Thread George Leslie-Waksman
We are using airflow with SQLAlchemy==1.1.1 without any problems. On Wed, Oct 19, 2016 at 2:07 AM harish singh wrote: > Hi everyone, > > We have been using airflow for roughly about 7 months. Pretty smooth :) > Thanks! > > Today when I build my airflow container, I found a weird issue. > > This

Wiki write access

2016-10-16 Thread George Leslie-Waksman
Could I get write access to the wiki for the username "gwax"? Thanks, --George Leslie-Waksman

Re: Next Airflow meet-up

2016-10-13 Thread George Leslie-Waksman
Can someone add "gwax" to the editor list for Confluence so I can add a Clover Health as the host of the subsequent meetup? I assume early December is too soon; how do folks feel about mid-January? --George Leslie-Waksman On Wed, Oct 12, 2016 at 12:22 AM Alex Van Boxel wrote: >

Re: Next Airflow meet-up

2016-10-06 Thread George Leslie-Waksman
t; > > On Sat, Oct 1, 2016 at 8:51 AM siddharth anand > wrote: > > > > > SGTM. > > > > > > -s > > > > > > On Fri, Sep 30, 2016 at 10:18 PM, George Leslie-Waksman < > > > geo...@cloverhealth.com> wrote: > > > >

Re: UI changes? Include screenshots!

2016-10-06 Thread George Leslie-Waksman
Is this noted in the how to submit a PR documentation / PR template? On Sat, Oct 1, 2016 at 1:31 AM siddharth anand wrote: > For all PRs with UI changes, it is a requirement that screenshots be > included in the PR. Please help reinforce this practice. > > > -s >

Re: Airflow Latency Between Task Scheduling and task Execution

2016-10-03 Thread George Leslie-Waksman
If you want to run faster and you're only using one machine, I would recommend using the LocalExecutor instead of the CeleryExecutor. On the more general topic, Airflow is NOT designed for processing real time data. It is a workflow tool for batch processing. The sweet spot for Airflow is running

Re: Setting Connections via Env Vars

2016-10-01 Thread George Leslie-Waksman
r having multiple ways to configure the same thing but, for the time being, it's how we use Airflow. --George Leslie-Waksman On Fri, Sep 30, 2016 at 11:47 PM siddharth anand wrote: > I see. Thanks for the explanation. > > Then, wouldn't you like Airflow's connections views t

Re: Split/Chunk operator

2016-09-30 Thread George Leslie-Waksman
Airflow isn't really designed for tasks to pass data to each other. There are limited capabilities for doing so but Airflow is primarily a workflow management tool, not a dataflow tool. --George Leslie-Waksman (Clover Health) On Sun, Sep 4, 2016 at 6:01 AM Mohsen Bande wrote: > it

Re: Airflow Releases

2016-09-30 Thread George Leslie-Waksman
. --George Leslie-Waksman (Clover Health) On Fri, Sep 30, 2016 at 12:56 PM Bolke de Bruin wrote: > We will switch to master beginning next week. I think it would be nice to > sync on git commit hashes that we are considering to be “master”. > > We have some operational concerns so fr

Re: Invitation: Airflow Contributors & Roadmapping Meeting @ Thu Oct 6, 2016 10am - 12pm (PDT) (gurer.kira...@airbnb.com)

2016-09-30 Thread George Leslie-Waksman
Great, I'll be there. --George Leslie-Waksman (Clover Health) On Fri, Sep 30, 2016 at 10:09 PM Gurer Kiratli wrote: > It's on Thursday October the 6th. Sorry for the typo. > > Sent from my iPhone > > On Sep 30, 2016, at 10:00 PM, George Leslie-Waksman < > geo...

Re: Next Airflow meet-up

2016-09-30 Thread George Leslie-Waksman
t; > Hey guys, > > Sure Airbnb can always host, but I think it might be nice to switch around. > Clover is welcome to do it this quarter. We can always be a fallback. > > Cheers, > > Gurer > > On Fri, Sep 30, 2016 at 7:27 AM, George Leslie-Waksman < > geo...@clove

Re: Setting Connections via Env Vars

2016-09-30 Thread George Leslie-Waksman
. I would be happy to discuss things further, if you need more details. Certainly we could work around, and certainly there are things we don't like about the current system, but we only have so many places that we can devote resources at any given point in time. --George Leslie-Waksman (C

Re: Invitation: Airflow Contributors & Roadmapping Meeting @ Thu Oct 6, 2016 10am - 12pm (PDT) (gurer.kira...@airbnb.com)

2016-09-30 Thread George Leslie-Waksman
I would like to attend but I am confused about the date: The message says October 7th but the calendar Invite says October 6th. Could someone please confirm the date so that I can make arrangements to attend? Thank you, --George Leslie-Waksman (Clover Health) On Tue, Sep 27, 2016 at 8:55 AM

Re: Next Airflow meet-up

2016-09-30 Thread George Leslie-Waksman
Clover Health would be happy to host a meetup, be it in Q4 or next year. --George Leslie-Waksman On Thu, Sep 29, 2016 at 4:07 PM siddharth anand wrote: > Gurer? Let us know what you think. > > Q4 tends to be a busy time, so we should get the next one set up at the > earliest. I&

How to get PRs reviewed?

2016-09-19 Thread George Leslie-Waksman
I have two PRs that have been sitting, open, ready for review / merge (one for over a month): https://github.com/apache/incubator-airflow/pull/1738 https://github.com/apache/incubator-airflow/pull/1752 Am I submitting incorrectly? How do I get things reviewed? I'd rather not maintain my own for

Re: [jira] [Commented] (AIRFLOW-62) XCom push not working reliably

2016-09-02 Thread George Leslie-Waksman
If you're using 1.7.1.3, there's a bug (fixed in master) involving a race condition that can occasionally result in XCom values being cleared. The short version is: 1) task get's scheduled; 2) worker A grabs task; 3) worker B grabs task; 4) worker A clears the XCom, completes task, and pushes XCom

Re: DagRun date "off by one"

2016-08-08 Thread George Leslie-Waksman
This has been one of the bigger gotchas for people on my team. Once people "get it" they get it but every single person in my company has assumed that the behavior was the opposite of what it is. While I understand that there are arguments in favor of the scheduling being the way that it is, it fe

Re: High res Airflow image?

2016-06-23 Thread George Leslie-Waksman
For fun, I've started making an svg of the logo. It won't be exactly the same as the original but it should be fairly close. I hope to get a PR with it out later today. On Wed, Jun 22, 2016 at 8:22 AM Chris Riccomini wrote: > Hey Bence, > > The best I was able to come up with was a fairly large

Re: UTC everywhere requirement

2016-06-06 Thread George Leslie-Waksman
There are three common ways of handling timezones: 1) use timezone aware datetime objects, 2) use timezone naive datetime objects and make sure any system running your software has its clock set toUTC, or 3) be very, very sad because have potentially inconsistent times in your logs, scheduling, etc

Re: S3 connection

2016-05-24 Thread George Leslie-Waksman
We ran into this issue as well. If you set the environment variable to anything random, it'll get ignored and control will pass through to .aws/credentials We used "n/a" It's kind of annoying that the s3 connection is a) required, and b) poorly supported as an env var. On Tue, May 24, 2016 at 8: