Hi Airflow Dev's,

TLDR Summary: Please express your feedback for AIP-69, if no objections I'd 
propose another round of VOTE.

Thanks for the responses and questions to AIP-69 so far. I hosted a Q&A session 
today with Ash, Jarek, Vikram, Daniel, Vincent attending. We discussed about 
the background and the "tactical Airflow 2.10+" vs. "Airflow 3" timeline and 
risks.

Main discussion points were:
- All agree on Remote executor being a valid requirement and AIP with 
sufficient detail. PoC PR also available
- Jens proposes to raise the feature of Remote Executor to be in 2.10 line as 
the need in Bosch to stabilize is quite high. Waiting for Airflow 3 would make 
a lot of delay.
- Changes in Airflow Core are minimal but dependency to AIP-44 and AIP-61 are 
existing
- In Airflow 3 a lot of logic needs to be re-written due to AIP-72 which all 
agree is a great way forward
- Doubts were raised if an early general availability of Remote Executor 
Feature will distract community for support and off Airflow 3 development
- Doubts were raised also in the area that the API for remote execution has 
security implications as a public facing API can be attacked

From the meeting main consent was elaborated in a direction of:
- Remote Executor will be marked experimental, not contained in default release
- Even if installed, remote endpoint will be disabled by default to minimize 
risk of exposure
- We would release the package only with a version suffix "pre0" such that an 
user must explicitly install a pre-release version as manual install
- Support and maintenance in Airflow 2.10++ will end with the feature being 
available in Airflow 3 to reduce double maintenance and as motivation to migrate

Alternatively Bosch could develop the package internally and not contribute 
this feature. It was also briefly discussed if Bosch could maintain and provide 
to Pypi - but after the meeting and with a bit of thought this might be complex 
if Bosch would release SW outside of Apache foundation (legal, tax whatever 
discussions - I am not a lawyer).

With this summary I'd raise another round of feedback. If no major discussions 
or concerns shared I'd raise another round of VOTEs.
(+ still if anybody wants to contribute and join forces, does not need to be a 
pure Bosch party 😃 )

--> https://cwiki.apache.org/confluence/display/AIRFLOW/AIP-69+Remote+Executor
--> https://github.com/apache/airflow/pull/40224

Mit freundlichen Grüßen / Best regards

Jens Scheffler

Alliance: Enabler - Tech Lead (XC-AS/EAE-ADA-T)
Robert Bosch GmbH | Hessbruehlstraße 21 | 70565 Stuttgart-Vaihingen | GERMANY | 
http://www.bosch.com/
Tel. +49 711 811-91508 | Mobil +49 160 90417410 | jens.scheff...@de.bosch.com

Sitz: Stuttgart, Registergericht: Amtsgericht Stuttgart, HRB 14000;
Aufsichtsratsvorsitzender: Prof. Dr. Stefan Asenkerschbaumer;
Geschäftsführung: Dr. Stefan Hartung, Dr. Christian Fischer, Dr. Markus 
Forschner,
Stefan Grosch, Dr. Markus Heyn, Dr. Frank Meyer, Dr. Tanja Rückert

-----Original Message-----
From: Scheffler Jens (XC-AS/EAE-ADA-T) <jens.scheff...@de.bosch.com.INVALID>
Sent: Thursday, June 13, 2024 11:54 PM
To: dev@airflow.apache.org
Subject: RE: [DISCUSS] AIP-69 Remote Executor

Hi Airflow Dev's,

After todays Airflow 3 planning and the "sneak preview" from Ash' AIP-72 I took 
some time today to update the AIP-69 Remote Executor with more technical 
details as I tried a PoC implementation.

The AIP document is now updated and I call a second round for feedback / 
discussion in 
https://cwiki.apache.org/confluence/display/AIRFLOW/AIP-69+Remote+Executor

Also as requested I shared my PoC code (which actually is now able to make the 
first task being called via HTTP) - I hope this gives an insight about the 
direction I am thinking - but also here feedback welcome: 
https://github.com/apache/airflow/pull/40224

Direct feedback to the email before: As I tried to look into task execution cut 
I saw a lot of Spaghetti and am convinced now that the AIP-69 must in todays 
world build on top of AIP-44, should not re-invent the wheel and will in future 
greatly benefit of AIP-72. I intend that we align across these three AIPs so 
that the pieces are fitting together. But this also means that not all targets 
can be met in a first release and dependencies very probably can only be cut 
off after AIP-72.

Implementation target will be that most of the logic is made into the provider 
package and as you can see in the Draft PR only minimal changes are needed in 
the core.

Mit freundlichen Grüßen / Best regards

Jens Scheffler

Alliance: Enabler - Tech Lead (XC-AS/EAE-ADA-T) Robert Bosch GmbH | 
Hessbruehlstraße 21 | 70565 Stuttgart-Vaihingen | GERMANY | 
http://www.bosch.com/ Tel. +49 711 811-91508 | Mobil +49 160 90417410 | 
jens.scheff...@de.bosch.com

Sitz: Stuttgart, Registergericht: Amtsgericht Stuttgart, HRB 14000;
Aufsichtsratsvorsitzender: Prof. Dr. Stefan Asenkerschbaumer;
Geschäftsführung: Dr. Stefan Hartung, Dr. Christian Fischer, Dr. Markus 
Forschner, Stefan Grosch, Dr. Markus Heyn, Dr. Frank Meyer, Dr. Tanja Rückert

-----Original Message-----
From: Bolke de Bruin <bdbr...@gmail.com>
Sent: Sunday, May 19, 2024 1:14 PM
To: dev@airflow.apache.org
Subject: Re: [VOTE] AIP-69 Remote Executor

Hi Jens,

I've read your proposal and I think I understand where you want to take it.
To me the gist is:

"Allow running airflow agents behind corporate firewalls that use a pull model 
to run tasks and do not accept incoming connections".

I agree with the other commenters that the current proposal is not strong 
enough. Particularly, because it does not establish a relationship with other 
proposals and established ways of working. For example, you mention that the 
agent should communicate over HTTPS. If you would connect this to AIP-44, which 
imho only covers part of what you are doing, I would at least expect you to 
mention GRPC and how it would extend the message format of AIP-44. 
Architecturally I would like to see a discussion why it would be embedded in 
the webserver (spoiler: I don't think it should).

As Jarek is mentioning, you are not clear whether you expect tasks to be able 
to communicate with the central service or this is done by the agent on behalf 
of the tasks. In a corporate environment I would expect the latter, which puts 
it a lot closer to future isolation.

In addition, you are not documenting how an agent is deciding whether it can 
run a certain task and how it establishes a 'lock' to make sure no race 
condition happens. How do you recover from failures? Did you consider something 
like using Temporal here?

So while I understand that some things are left open to implementation, now 
there are too many questions still open and those influence the direction.

Cheers
Bolke

On Sun, 19 May 2024 at 00:37, Jarek Potiuk <ja...@potiuk.com> wrote:

> > P.S.: I'd like to take your feedback serious, the AIP process
> > description
> in Confluence just tells: "Once you or someone else feels like there’s
> a rough consensus on the idea and there’s no strong opposition, you
> can move your proposal to the Vote phase." - neither this nor the
> structure template mention that a technical spec or PR must be
> provided prior vote. If you feel that an AIP should include this then
> I assume the contribution docs need to be adjusted.
>
> Just one - comment here (I am at Pycon US now). I think it's hard to
> describe in detail the requirements because it will be more of a
> case-by-case, but I think you should simply compare this with other
> AIPs where a lot of details have been worked out before voting - often
> with code snippets, or detailed description of the APIs or communication 
> involved.
> It's just difficult to understand what exactly changes it will bring -
> will it use AIP-44 or not? - not clear, will it change how the
> heartbeat will work ? How are we going to handle retries, what the API
> proposal will be in general (snippets?). Even if it is not a final
> version, but first "pass" of how roughly all those things will look
> like - even if not working, what changes it will impose on each
> component is usually what you will find in recently created AIPs. I
> think a LOT depends on the implementation choices here - including
> whether we **really** want to do it in 2.* or maybe that is simply
> something that should be Airflow 3 - because we will find out that
> what is needed to implement is basically already available in what we
> will have in Airflow 3 or because we will be able to implement is in a
> much simpler way in Airflow 3. Or maybe Airflow 3 will implement all the 
> needs there simply "via" task isolation.
>
> And I know it means "delay" - (and for example waiting for the
> detailed Task Isolation proposal that Ash works on and that will
> undoubtedly contain a lot of details) - but this is also exactly the
> reason why i put on hold
> AIP-67 "Multi-team" - even if I had it much more detailed and knew
> what it means for most of the internals, because I think we should put
> it in perspective of Airflow 3 vision and decide: do we want it in Airflow 2 ?
> Maybe differently in Airflow 2 and Airflow 3 or Airflow 2 only? I do
> not know.
>
> In the case of this AIP, the thing is that it is too high level to
> understand how it maps into changes to the components. How things will
> change and which components are impacted here ? I have no idea. will
> we change "local task" execution and heartbeat and status updates /
> retry/ execution semantics that are currently all run (and very
> tightly coupled with how "local task" implementation works"). Actually
> one reason why Airflow 3 was a "thing" was that we need to remove a
> lot of coupling from "local task" and "Airflow DB" to make it easier
> and less of a maintenance burden (and breaking compatibility of how
> tasks work). And this is why
> AIP-44 went in the other direction - don't change anything, just wrap
> DB calls in gRPc (but keep compatibility on how tasks work and
> interact with Airflow).
>
> And to be honest - from the current AIP it's entirely unclear whether
> it is closer to AIP-44 or future isolation where the API will be completely 
> new.
> So I simply do not know what I am voting on.
>
> Maybe to put it in short terms - I see AIP voting not voting on "Idea"
> but voting on "Implementation proposal" - and for me, the current
> status is that it is an "Idea" but a lot of Implementation proposal
> needs to be hashed out.
>
> J.
>
> On Sat, May 18, 2024 at 2:45 PM Scheffler Jens (XC-AS/EAE-ADA-T)
> <jens.scheff...@de.bosch.com.invalid> wrote:
>
> > Hi Vikram, Jarek, Ash,
> >
> > Thanks for your fast feedback. So I understand you ask for more details.
> > Then I will cancel the vote if you think the AIP is not ready.
> >
> > I would have expected with my earlier call for discussion to receive
> > more feedback on what you expect missing. Please add comments to
> > confluence or post a message to me. That would help making the AIP
> > to the level of
> detail
> > you expect.
> > A PR is not ready helping to discover, it is rather a skeleton but a
> > lot of pieces are not decided. I would have planned to tackle this
> > during implementation. And then - once code is mature - there are
> > PRs planned of course where also serious code feedback will be needed.
> > But my intent was not to spent 100's of hours in a detailed concept
> > then to get a push-back later. That is why I called for vote. If you
> > can also please express the "do this" or "don't this" in the AIP
> > feedback that
> will
> > help that I am not wasting my time hitting the wall later. Also I
> > was seeking for contribution, so if you have ideas or
> > recommendation, please keep them coming.
> >
> > Immediate feedback to some of your questions:
> > - @ash: What do you expect in a level of detail in a PR to review
> > before accepting the AIP? Are you looking for certain patterns,
> > modules which
> are
> > used?
> > - @jarek: PoC can be made but I fear that is already 50% of effort
> > to a MVP. That is why I was seeking for feedback if this would be
> > the right
> way
> > before spending efforts.
> > - @jarek: AIP-44 would certainly be leveraged. I have no plan to
> replicate
> > existing API and I know that it will be a challenge to de-couple the
> > task execution. Do you expect me to evaluate all details before a
> > vote? I
> would
> > have planned a pragmatic approach, once some dependency missing I
> > would maybe have contributed to AIP-44 efforts to close gaps. But
> > AIP-44 does
> not
> > make a remote scheduling/execution possible as it does not include
> > the remote worker and executor component. That is to be added in AIP-69.
> > - @jarek/@ash: You briefly talked about the "task isolation" but are
> there
> > any concepts you can share which will help? I understood you are
> _thinking_
> > about it and there will be papers _soon_? If there is anything that
> > contributes, please share. I have no glassball.
> > @jarek: Lifecyle of task and adoption etc.: Yes this is something to
> > be addressed and is a must to be included. Mainly it would be around
> > a heartbeat logic. I am sure there will be something to be done and
> > resiliency will be covered. If you have requirements or ideas please
> > contribute. Do you see this as a must have in the AIP before going
> > to
> vote?
> > Or is it just important it is covered?
> > @jarek: API promises on exactly once: My plan is to use and rely on
> > DB locks and transactions from the API. Assuming that something can
> > go wrong also between API and remote side I would add an additional
> > confirmation
> in
> > the heartbeat when the task was accepted and is starting on remote
> > (to cover the use case that a task is assigned and DB transaction
> > committed
> and
> > response is not reaching the remote because of... line breaks down).
> > Als certainly if a remote worker is "lost" and does not heartbeat
> > for a (to
> be
> > configured) timeout then a task must be re-assigned or assumed to be
> failed.
> >
> > P.S.: I'd like to take your feedback serious, the AIP process
> > description in Confluence just tells: "Once you or someone else
> > feels like there’s a rough consensus on the idea and there’s no
> > strong opposition, you can
> move
> > your proposal to the Vote phase." - neither this nor the structure
> template
> > mention that a technical spec or PR must be provided prior vote. If
> > you feel that an AIP should include this then I assume the
> > contribution docs need to be adjusted.
> >
> > Mit freundlichen Grüßen / Best regards
> >
> > Jens Scheffler
> >
> > Alliance: Enabler - Tech Lead (XC-AS/EAE-ADA-T) Robert Bosch GmbH |
> > Hessbruehlstraße 21 | 70565 Stuttgart-Vaihingen | GERMANY |
> > http://www.bosch.com/ Tel. +49 711 811-91508 | Mobil +49 160 90417410 |
> > jens.scheff...@de.bosch.com
> >
> > Sitz: Stuttgart, Registergericht: Amtsgericht Stuttgart, HRB 14000;
> > Aufsichtsratsvorsitzender: Prof. Dr. Stefan Asenkerschbaumer;
> > Geschäftsführung: Dr. Stefan Hartung, Dr. Christian Fischer, Dr.
> > Markus Forschner, Stefan Grosch, Dr. Markus Heyn, Dr. Frank Meyer,
> > Dr. Tanja Rückert
> >
> > -----Original Message-----
> > From: Vikram Koka <vik...@astronomer.io.INVALID>
> > Sent: Saturday, May 18, 2024 6:57 PM
> > To: dev@airflow.apache.org
> > Subject: Re: [VOTE] AIP-69 Remote Executor
> >
> > I agree with Jarek and Ash on this.
> >
> > I believe that the AIP as written documents the “what” and the “why”
> > very well, but is too light on the “how”.
> >
> > I very much would like to see this AIP become reality as well, but I
> > believe that we need some foundational elements such as API-44 and
> > the “task context” concept to take this AIP to fruition with enough
> > functionality to be meaningful.
> >
> > It’s entire possible that you are proposing something in between
> > that’s feasible in the short-term, but that’s not clear to me yet.
> >
> > Vikram
> >
> >
> > On Sat, May 18, 2024 at 8:46 AM Jarek Potiuk <ja...@potiuk.com> wrote:
> >
> > > Agree with Ash. I also have a feeling that this is a very generic
> > > description and some POC describing the approach we would like to
> > > take here is needed to be able to vote on it. Feels a bit too
> > > early to vote
> > on it.
> > >
> > > A lot of internals of the current (Airflow 2) way Airflow handling
> > > of tasks running in executor are about database communication and
> > > if you look closely, decoupling those internals to make it works
> > > with the current executor API might be either very difficult or
> > > complex if we stick to the current task <> airflow communication
> > > patterns. In some ways, you already get "Remote Executor" by
> > > simply completing AIP-44 (which precisely provides an HTTP-based.
> > > interface between tasks and scheduler in a very similar fashion as
> > > AIP-69 proposal, but without changing the communication patterns.
> > >
> > > As I see it it generally looks like AIP-69 is either the same as
> > > AIP-44 or the same as the future "Airflow 3" task isolation we've
> > > been discussing about and Ash is working on. Both AIP-44 and the
> > > future Task Isolation aim to solve pretty much the same problem
> > > but AIP-44 in a very "backwards compatible" way and limited change
> > > on how to achieve
> > "remote execution"
> > > without actually making a lot of ripple effect on all other components.
> > >
> > > So I would really love to understand if AIP-69 is really something
> > > in-between and how - but the "lightweitness" of description makes
> > > it difficult to understand how different AIP-69 is from those two
> > > (of course we should see the future task isolation AIP as well to
> > > understand it better and know what kind of back-compatibilities it
> > > will
> > involve in Airflow 3).
> > >
> > > I think at the very minimum we should see the proposal of how the
> > > API will look like between the task and executor (including the
> > > whole life-cycle of tasks), the way how we are going to implement
> > > all the complexity involved with task adoption, edge cases of
> > > scheduling, execution semantic promises the API will hold
> > > (exactly-once, at-most-once, at least once) - something that comes
> > > as given for celery, the queuing mechanism and technologies used
> > > (how do we handle distributed case where you have to manage
> > > multiple workers running and
> > how the tasks will be distributed etc. etc.
> > >
> > > For me the AIP currently is mostly documenting a "wishlist" but
> > > the implementation details on which of those wishes we implement,
> > > and which not, and how is very much absent.
> > >
> > > J.
> > >
> > > On Sat, May 18, 2024 at 10:41 AM Ash Berlin-Taylor
> > > <a...@apache.org>
> > wrote:
> > >
> > > > Can we have a link to the pr please? The AIP doc itself is still
> > > > light on what changes are actually needed
> > > >
> > > > On 18 May 2024 14:56:57 BST, Aritra Basu
> > > > <aritrabasu1...@gmail.com>
> > > wrote:
> > > > >+1 (non-binding)
> > > > >The proposal was a good read, would love to see it come up and
> > > > >would
> > > love
> > > > >to help out if you need a helping hand.
> > > > >
> > > > >--
> > > > >Regards,
> > > > >Aritra Basu
> > > > >
> > > > >On Sat, May 18, 2024, 7:15 PM Christian Schilling
> > > > ><christian.lellm...@googlemail.com.invalid> wrote:
> > > > >
> > > > >> Hi Jens,
> > > > >>
> > > > >> Thank you very much for the proposal!
> > > > >> This would be cool to have such a feature in Airflow.
> > > > >>
> > > > >> +1 non-binding
> > > > >>
> > > > >> Best,
> > > > >>
> > > > >> Chris
> > > > >>
> > > > >> Scheffler Jens (XC-AS/EAE-ADA-T) <jens.scheff...@de.bosch.com
> > > .invalid>
> > > > >> schrieb am Sa., 18. Mai 2024, 15:40:
> > > > >>
> > > > >> > Hi all,
> > > > >> >
> > > > >> >
> > > > >> >
> > > > >> > Discussion thread:
> > > > >> >
> > > > >> > https://eur03.safelinks.protection.outlook.com/?url=https%253
> > > > >> > A%2F
> > > > >> > %2Flists.apache.org%2Fthread%2F8hlltm9brdxqyf8jyw1syrfb11hl
> > > > >> > 52k5
> > > > >> > &data=05%7C02%7CJens.Scheffler%40de.bosch.com%7C4300b2481a7
> > > > >> > 34dd
> > > > >> > aadac08dc775b921e%7C0ae51e1907c84e4bbb6d648ee58410f4%7C0%7C
> > > > >> > 0%7C
> > > > >> > 638516482373030237%7CUnknown%7CTWFpbGZsb3d8eyJWIjoiMC4wLjAw
> > > > >> > MDAi
> > > > >> > LCJQIjoiV2luMzIiLCJBTiI6Ik1haWwiLCJXVCI6Mn0%3D%7C0%7C%7C%7C
> > > > >> > &sda
> > > > >> > ta=E4blX6HbfvW73CkOWMxQoc5XSYDiIToAcLqbh9zVFE8%3D&reserved=
> > > > >> > 0
> > > > >> >
> > > > >> >
> > > > >> >
> > > > >> > I would like to officially call for a vote to add a Remote
> > > > >> > Executor feature to Airflow – via a provider package. All
> > > > >> > details are
> > > > documented
> > > > >> in:
> > > > >> >
> > > > >>
> > > >
> > > https://eur03.safelinks.protection.outlook.com/?url=https%3A%2F%2F
> > > cwik%2F&data=05%7C02%7CJens.Scheffler%40de.bosch.com%7Cefe0146942d
> > > 64664a65008dc77f4fea3%7C0ae51e1907c84e4bbb6d648ee58410f4%7C0%7C0%7
> > > C638517141326404301%7CUnknown%7CTWFpbGZsb3d8eyJWIjoiMC4wLjAwMDAiLC
> > > JQIjoiV2luMzIiLCJBTiI6Ik1haWwiLCJXVCI6Mn0%3D%7C0%7C%7C%7C&sdata=xR
> > > X1nVkoAwqviz8prAEmFjF4JUYxW65z%2FZvuZChpDuE%3D&reserved=0
> > > i.apache.org%2Fconfluence%2Fdisplay%2FAIRFLOW%2FAIP-69%2BRemote%2B
> > > Exec
> > > utor&data=05%7C02%7CJens.Scheffler%40de.bosch.com%7C4300b2481a734d
> > > daad
> > > ac08dc775b921e%7C0ae51e1907c84e4bbb6d648ee58410f4%7C0%7C0%7C638516
> > > 4823
> > > 73039443%7CUnknown%7CTWFpbGZsb3d8eyJWIjoiMC4wLjAwMDAiLCJQIjoiV2luM
> > > zIiL
> > > CJBTiI6Ik1haWwiLCJXVCI6Mn0%3D%7C0%7C%7C%7C&sdata=Hbvz5Ct1PE52pUDFD
> > > AeHO
> > > SYncOZ%2BOFi9eZQilaxeu9E%3D&reserved=0
> > > > >> >
> > > > >> >
> > > > >> >
> > > > >> > Since the first draft was raised and the discussion thread
> > > > >> > went
> > > > along, I
> > > > >> > integrated a bit of feedback and now also added some
> > > > >> > technical
> > > > details as
> > > > >> > proposed development. After successful vote I’d raise a
> > > > >> > draft PR as
> > > > PoC.
> > > > >> > As a big wave of emails was posted by Jarek after I dropped
> > > > >> > this AIP
> > > > I’d
> > > > >> > like to highlight that I propose to make this a tactical
> > > > implementation
> > > > >> > which might be a base for some discussion how to distribute
> > > > >> > work in
> > > a
> > > > >> > future Airflow 3.0. I would assume if interfaces and
> > > > >> > structures
> > > > change,
> > > > >> > rework will be needed and it is fully accepted that
> > > > >> > breaking changes
> > > > need
> > > > >> > to be applied if moving to Airflow 3.
> > > > >> >
> > > > >> >
> > > > >> >
> > > > >> > Looking forward to releasing this for Airflow 2.10 (but
> > > > >> > depending
> > > how
> > > > >> fast
> > > > >> > I can make it and also depending on if somebody wants to
> > > > >> > join
> > > forces).
> > > > >> >
> > > > >> >
> > > > >> >
> > > > >> > Consider this my +1 binding vote.
> > > > >> >
> > > > >> >
> > > > >> >
> > > > >> > The vote will last until 6:00 PM GMT/UTC on May 23, 2024,
> > > > >> > and until
> > > at
> > > > >> > least 3 binding votes have been cast. I have it a bit
> > > > >> > longer as
> > > usual
> > > > >> > because of a public holiday in some countries.
> > > > >> >
> > > > >> >
> > > > >> >
> > > > >> > Please vote accordingly:
> > > > >> >
> > > > >> >
> > > > >> >
> > > > >> > [ ] + 1 approve
> > > > >> >
> > > > >> > [ ] + 0 no opinion
> > > > >> >
> > > > >> > [ ] - 1 disapprove with the reason
> > > > >> >
> > > > >> >
> > > > >> >
> > > > >> > Only votes from PMC members and committers are binding, but
> > > > >> > other
> > > > members
> > > > >> > of the community are encouraged to check the AIP and vote
> > > > >> > with "(non-binding)".
> > > > >> >
> > > > >> > Mit freundlichen Grüßen / Best regards
> > > > >> >
> > > > >> > Jens Scheffler
> > > > >> >
> > > > >> > Alliance: Enabler - Tech Lead (XC-AS/EAE-ADA-T) Robert
> > > > >> > Bosch GmbH | Hessbruehlstraße 21 | 70565
> > > > >> > Stuttgart-Vaihingen
> > > |
> > > > >> > GERMANY | http://www.bosch.com/
> > > > >> > Tel. +49 711 811-91508 | Mobil +49 160 90417410 |
> > > > >> > jens.scheff...@de.bosch.com<mailto:Jens.Scheffler@de.bosch.
> > > > >> > com>
> > > > >> >
> > > > >> > Sitz: Stuttgart, Registergericht: Amtsgericht Stuttgart,
> > > > >> > HRB 14000;
> > > > >> > Aufsichtsratsvorsitzender: Prof. Dr. Stefan
> > > > >> > Asenkerschbaumer;
> > > > >> > Geschäftsführung: Dr. Stefan Hartung, Dr. Christian Fischer, Dr.
> > > > Markus
> > > > >> > Forschner,
> > > > >> > Stefan Grosch, Dr. Markus Heyn, Dr. Frank Meyer, Dr. Tanja
> > > > >> > Rückert ​
> > > > >> >
> > > > >>
> > > >
> > >
> >
>


--

--
Bolke de Bruin
bdbr...@gmail.com
B KKKKKKKKKKKKKKKKKKKKKKKKKKKKKKKKKKKKKKKKKKKKKKKKKKKKKKKKKKKKKKKKKKKKCB  [  
X  ܚX KK[XZ[
 ] ][  X  ܚX PZ\   ˘\X K ܙ B  ܈Y][ۘ[  [X[  K[XZ[
 ] Z[Z\   ˘\X K ܙ B

Reply via email to