You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by mjsax <gi...@git.apache.org> on 2015/08/04 13:03:02 UTC

[GitHub] flink pull request: [FLINK-2457] Integrate Tuple0

GitHub user mjsax opened a pull request:

    https://github.com/apache/flink/pull/983

    [FLINK-2457] Integrate Tuple0

     - extended TypeExtractor to handle Tuple0
     - extended .getTupleClass(int) to hanlde arity zero
     - simplified Receiver.createTuple(int)

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/mjsax/flink flink-2457-tuple0

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/flink/pull/983.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #983
    
----
commit 55685a61fa4ad341b6f7ee0c62fba389ad128933
Author: mjsax <mj...@informatik.hu-berlin.de>
Date:   2015-08-02T19:38:31Z

    [FLINK-2457] Integrate Tuple0
     - extended TypeExtractor to handle Tuple0
     - extended .getTupleClass(int) to hanlde arity zero
     - simplified Receiver.createTuple(int)

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2457] Integrate Tuple0

Posted by mjsax <gi...@git.apache.org>.
Github user mjsax commented on a diff in the pull request:

    https://github.com/apache/flink/pull/983#discussion_r36190666
  
    --- Diff: flink-java/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java ---
    @@ -391,6 +392,10 @@ protected TypeExtractor() {
     				curT = typeToClass(curT).getGenericSuperclass();
     			}
     			
    +			if(typeToClass(curT).equals(Tuple0.class)) {
    --- End diff --
    
    Yes it can. But in this case we are not done and do not return here. The test for `if (curT instanceof Class<?>)` is done below... This `if` is an early return that is only valid if the type is `Tuple0`. Or do I miss something?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2457] Integrate Tuple0

Posted by twalthr <gi...@git.apache.org>.
Github user twalthr commented on a diff in the pull request:

    https://github.com/apache/flink/pull/983#discussion_r36189782
  
    --- Diff: flink-java/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java ---
    @@ -391,6 +392,10 @@ protected TypeExtractor() {
     				curT = typeToClass(curT).getGenericSuperclass();
     			}
     			
    +			if(typeToClass(curT).equals(Tuple0.class)) {
    --- End diff --
    
    `curT` could be `TypeVariable` or someting else at this point. I think it's better to use `if (curT instanceof Class<?> && curT == Tuple0.class)`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2457] Integrate Tuple0

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on the pull request:

    https://github.com/apache/flink/pull/983#issuecomment-127764809
  
    I tried to write a DataSet program that processes Tuple0 records and fails during execution. However, this is surprisingly hard. In fact, I didn't manage to break the system.
    
    The reason why we argued that Tuple0 is unsafe is because de/serialization does not read/write anything and the byte stream is not forwarded. Hence you could "read" a million Tuple0 objects without advancing the stream. This is only relevant for DataSet that consist completely of types that have this behavior, because as soon there is another type with proper de/serialization the stream is advanced. 
    
    I was surprised when @mjsax said, that a job that shuffled Tuple0 records (which means de/serialization) worked. I verified that and my guess would be that the network stack writes the number of records into its network buffer and stops deserializing after all records are deserialized (not when the byte stream is EOF). Hence, network serialization works even if no bytes are shipped. @uce or @StephanEwen might confirm this.
    
    Another reason to de/serialize data is for sorting, grouping, joining, or crossing. Most transformation that require a key (there are some inconsistency wrt to key handling in the API) do not allow Tuple0 keys and fail with a good error message when the program is constructed (not at runtime). The API does not allow to sort, join, or group a data set on a Tuple0. I thought that using a cross transformation would let a program fail while executing because it does not require a key and should serialize data, however it worked. 
    
    The system seems to be more robust than I expected. Nonetheless, I am still a bit skeptical about this change and would like to learn why network transfer and crossing worked so well.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2457] Integrate Tuple0

Posted by mjsax <gi...@git.apache.org>.
Github user mjsax commented on the pull request:

    https://github.com/apache/flink/pull/983#issuecomment-128387669
  
    - changed `LinkedList` to `ArrayList` in **all** `TupleXBuilder` classes
    - removed WS formatting
    - changed `Tuple0` **and** `Tuple0Serializer` to singleton


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2457] Integrate Tuple0

Posted by mjsax <gi...@git.apache.org>.
Github user mjsax commented on the pull request:

    https://github.com/apache/flink/pull/983#issuecomment-128339056
  
    Done. Ready to get merged?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2457] Integrate Tuple0

Posted by mjsax <gi...@git.apache.org>.
Github user mjsax commented on the pull request:

    https://github.com/apache/flink/pull/983#issuecomment-127623565
  
    @twalthr thanks for the hint to `TypeInfoParser`. I will have a lock. But your statement about "introduce multiple bugs", seems to be a little bit to general... I did not break any tests... If there is untested code that I might have broken, please let me know. For this case, we should extend test coverage for those classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2457] Integrate Tuple0

Posted by StephanEwen <gi...@git.apache.org>.
Github user StephanEwen commented on the pull request:

    https://github.com/apache/flink/pull/983#issuecomment-127594920
  
    I think this patch needs to include the proper serializers and comparators for the tuple0. Otherwise it now allows users to use the type, but the runtime may fail on it, which is a dangerous inconsistent state.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2457] Integrate Tuple0

Posted by StephanEwen <gi...@git.apache.org>.
Github user StephanEwen commented on the pull request:

    https://github.com/apache/flink/pull/983#issuecomment-128396895
  
    Two comments on the singleton change:
      - The Tuple0.class.newInstance() will now probably fail (due to the private constructor)
      - Java serialization still creates copies. Need to overwrite the `readResolve()` method to create proper singletons.
      - It is probably okay allow Tuple0 instances to be created. We could simply providing a static shareable instance in the class for convenience (soft-singleton style).
    
    @twalthr Are the TypeExctractor related changes good?
    
    Thanks for reverting the whitespace changes. I did actually not expect that, it was more a comment for future pull requests...


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2457] Integrate Tuple0

Posted by mjsax <gi...@git.apache.org>.
Github user mjsax commented on the pull request:

    https://github.com/apache/flink/pull/983#issuecomment-128407712
  
    Ups. That was not intended at all... Does not make any sense. Sorry. I didn't realize that I edited the wrong class because Tuple0Serializer is singleton, too... Just fixed it.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2457] Integrate Tuple0

Posted by mjsax <gi...@git.apache.org>.
Github user mjsax commented on a diff in the pull request:

    https://github.com/apache/flink/pull/983#discussion_r36410500
  
    --- Diff: flink-java/src/main/java/org/apache/flink/api/java/tuple/builder/Tuple0Builder.java ---
    @@ -0,0 +1,46 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * 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 KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +
    +// --------------------------------------------------------------
    +//  THIS IS A GENERATED SOURCE FILE. DO NOT EDIT!
    +//  GENERATED FROM org.apache.flink.api.java.tuple.TupleGenerator.
    +// --------------------------------------------------------------
    +
    +
    +package org.apache.flink.api.java.tuple.builder;
    +
    +import java.util.LinkedList;
    +import java.util.List;
    +
    +import org.apache.flink.api.java.tuple.Tuple0;
    +
    +public class Tuple0Builder {
    +
    +	private List<Tuple0> tuples = new LinkedList<Tuple0>();
    --- End diff --
    
    I copied from Tuple1Builder, ...Tuple25Builder. Will introduce ArrayList for all other, too.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2457] Integrate Tuple0

Posted by twalthr <gi...@git.apache.org>.
Github user twalthr commented on the pull request:

    https://github.com/apache/flink/pull/983#issuecomment-130320695
  
    Your are right Tuple0 support for the TupleGenerator is not important. Actually, I meant the change of `import java.util.LinkedList;` to `import java.util.ArrayList;` otherwise these changes get lost if the TupleGenerator is executed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2457] Integrate Tuple0

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/983#discussion_r36362299
  
    --- Diff: flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializer.java ---
    @@ -97,48 +97,63 @@ public T copy(T from) {
     		}
     		return target;
     	}
    -	
    +
     	@Override
     	public T copy(T from, T reuse) {
     		for (int i = 0; i < arity; i++) {
     			Object copy = fieldSerializers[i].copy(from.getField(i), reuse.getField(i));
     			reuse.setField(copy, i);
     		}
    -		
    +
     		return reuse;
     	}
     
     	@Override
     	public void serialize(T value, DataOutputView target) throws IOException {
    -		for (int i = 0; i < arity; i++) {
    -			Object o = value.getField(i);
    -			try {
    -				fieldSerializers[i].serialize(o, target);
    -			} catch (NullPointerException npex) {
    -				throw new NullFieldException(i, npex);
    +		if (arity > 0) {
    --- End diff --
    
    IMO, it would be better if the TupleTypeInfo returns a special Tuple0Serializer instead of doing this check for each record during execution.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2457] Integrate Tuple0

Posted by StephanEwen <gi...@git.apache.org>.
Github user StephanEwen commented on the pull request:

    https://github.com/apache/flink/pull/983#issuecomment-127614323
  
    Try the example I wrote. I am pretty sure that one will not work.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2457] Integrate Tuple0

Posted by mjsax <gi...@git.apache.org>.
Github user mjsax commented on the pull request:

    https://github.com/apache/flink/pull/983#issuecomment-130323723
  
    I did the following changes:
     - added Tuple0 to `TupleGenerator.modifyTupleType()`
     - changes `LinkedList` to `ArrayList` in TupleGenerator to create code for `TupleX.java`
     - added methods `toString()`, `equals()`, and `hashCode()` to `Tuple0`
    (Of course, I run TupleGenerator to test it.)
    
    This PR should be ready for merging now.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2457] Integrate Tuple0

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/flink/pull/983


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2457] Integrate Tuple0

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on the pull request:

    https://github.com/apache/flink/pull/983#issuecomment-127693034
  
    The empty file is exactly the problem. 
    What happens if you read that file with a TypeSerializerInpitFormat typed for Tuple0? It will not return the same amount of Tuple0 objects as were "written" to it. That's the problem with de/serializers that do not read/write anything. This problem can occur anywhere in Flink's engine where data is serialized.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2457] Integrate Tuple0

Posted by mjsax <gi...@git.apache.org>.
Github user mjsax commented on the pull request:

    https://github.com/apache/flink/pull/983#issuecomment-128405666
  
    You are right about `newInstance`. I removed private constructor. Soft singleton is used now. Also added `readResolve()` (something new I learned today :) ).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2457] Integrate Tuple0

Posted by mjsax <gi...@git.apache.org>.
Github user mjsax commented on a diff in the pull request:

    https://github.com/apache/flink/pull/983#discussion_r36410627
  
    --- Diff: flink-staging/flink-language-binding/flink-language-binding-generic/src/main/java/org/apache/flink/languagebinding/api/java/common/streaming/Receiver.java ---
    @@ -346,61 +349,12 @@ public Tuple deserialize() {
     	}
     
     	public static Tuple createTuple(int size) {
    -		switch (size) {
    -			case 0:
    -				return new Tuple0();
    -			case 1:
    -				return new Tuple1();
    -			case 2:
    -				return new Tuple2();
    -			case 3:
    -				return new Tuple3();
    -			case 4:
    -				return new Tuple4();
    -			case 5:
    -				return new Tuple5();
    -			case 6:
    -				return new Tuple6();
    -			case 7:
    -				return new Tuple7();
    -			case 8:
    -				return new Tuple8();
    -			case 9:
    -				return new Tuple9();
    -			case 10:
    -				return new Tuple10();
    -			case 11:
    -				return new Tuple11();
    -			case 12:
    -				return new Tuple12();
    -			case 13:
    -				return new Tuple13();
    -			case 14:
    -				return new Tuple14();
    -			case 15:
    -				return new Tuple15();
    -			case 16:
    -				return new Tuple16();
    -			case 17:
    -				return new Tuple17();
    -			case 18:
    -				return new Tuple18();
    -			case 19:
    -				return new Tuple19();
    -			case 20:
    -				return new Tuple20();
    -			case 21:
    -				return new Tuple21();
    -			case 22:
    -				return new Tuple22();
    -			case 23:
    -				return new Tuple23();
    -			case 24:
    -				return new Tuple24();
    -			case 25:
    -				return new Tuple25();
    -			default:
    -				throw new IllegalArgumentException("Tuple size not supported: " + size);
    +		try {
    +			return Tuple.getTupleClass(size).newInstance();
    --- End diff --
    
    No need for this. `.getTupleClass()` does the check already.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2457] Integrate Tuple0

Posted by StephanEwen <gi...@git.apache.org>.
Github user StephanEwen commented on the pull request:

    https://github.com/apache/flink/pull/983#issuecomment-127949704
  
    Quoting from above:
    
    In many cases, the problem that Tuple0 does not consume anything, will be caught by the fact that other data that is stored or shipped with the tuple will advance the stream. However, relying on that is super dangerous. As soon as you hit a case where that is not given, it will not work any more.
    
    Not being able to find a counter example does not mean it is correct. 
    
    The network stack works, because it ships metadata bytes per element. With the TypeSerializerInput/Output format example, the output writes for 10 elements an empty file, the input reads either none or infinitely many. There is at least one counterexample.
    
    What is so bad about creating a `Tuple0Serializer` that writes a dummy byte per tuple?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2457] Integrate Tuple0

Posted by StephanEwen <gi...@git.apache.org>.
Github user StephanEwen commented on the pull request:

    https://github.com/apache/flink/pull/983#issuecomment-127598734
  
    Are you sending the Tuple0 as part of another job, or alone?
    
    Try the following: Job that generates n Tuple0, write it with a TypeSerializerOutputFormat. Then read it back in with a TypeSerializerInputFormat.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2457] Integrate Tuple0

Posted by mjsax <gi...@git.apache.org>.
Github user mjsax commented on the pull request:

    https://github.com/apache/flink/pull/983#issuecomment-127597013
  
    I double checked with two different programs and it worked... Not sure if I am missing something. Please see dev mailing list. Otherwise, please help me to come up with an example that breaks serialization such that I can use this example to fix the code. I did not try comparators though: not sure for what this would be helpful? There is no data in Tuple0 so there is nothing to compare with. IMHO, all Tuple0 object are equal to each other. I don't see any use case to group on Tuple0, thus, comparators are not required from my point of view and should not be supported.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2457] Integrate Tuple0

Posted by StephanEwen <gi...@git.apache.org>.
Github user StephanEwen commented on the pull request:

    https://github.com/apache/flink/pull/983#issuecomment-127614982
  
    In many cases, the problem that Tuple0 does not consume anything, will be caught by the fact that other data that is stored or shipped with the tuple will advance the stream.
    
    However, relying on that is super dangerous. As soon as you hit a case where that is not given, it will not work any more.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2457] Integrate Tuple0

Posted by twalthr <gi...@git.apache.org>.
Github user twalthr commented on the pull request:

    https://github.com/apache/flink/pull/983#issuecomment-127620260
  
    I think this PR will introduce multiple bugs.
    @mjsax please also have a look into the `TypeInfoParser`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2457] Integrate Tuple0

Posted by StephanEwen <gi...@git.apache.org>.
Github user StephanEwen commented on a diff in the pull request:

    https://github.com/apache/flink/pull/983#discussion_r36408488
  
    --- Diff: flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/Tuple0Serializer.java ---
    @@ -0,0 +1,86 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
    + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
    + * 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 KIND, either express or implied. See the License for the
    + * specific language governing permissions and limitations under the License.
    + */
    +package org.apache.flink.api.java.typeutils.runtime;
    +
    +import java.io.IOException;
    +
    +import org.apache.flink.api.common.typeutils.TypeSerializer;
    +import org.apache.flink.api.java.tuple.Tuple0;
    +import org.apache.flink.core.memory.DataInputView;
    +import org.apache.flink.core.memory.DataOutputView;
    +
    +public class Tuple0Serializer extends TupleSerializer<Tuple0> {
    +	private static final long serialVersionUID = 1278813169022975971L;
    +
    +	public Tuple0Serializer() {
    +		super(Tuple0.class, new TypeSerializer<?>[0]);
    +	}
    +
    +	@Override
    +	public Tuple0Serializer duplicate() {
    +		return new Tuple0Serializer();
    --- End diff --
    
    No need to create a new instance, since this is stateless.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2457] Integrate Tuple0

Posted by twalthr <gi...@git.apache.org>.
Github user twalthr commented on the pull request:

    https://github.com/apache/flink/pull/983#issuecomment-127633696
  
    Sorry, I was just thinking loadly. You are right ;)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2457] Integrate Tuple0

Posted by StephanEwen <gi...@git.apache.org>.
Github user StephanEwen commented on a diff in the pull request:

    https://github.com/apache/flink/pull/983#discussion_r36408352
  
    --- Diff: flink-java/src/main/java/org/apache/flink/api/java/tuple/builder/Tuple0Builder.java ---
    @@ -0,0 +1,46 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * 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 KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +
    +// --------------------------------------------------------------
    +//  THIS IS A GENERATED SOURCE FILE. DO NOT EDIT!
    +//  GENERATED FROM org.apache.flink.api.java.tuple.TupleGenerator.
    +// --------------------------------------------------------------
    +
    +
    +package org.apache.flink.api.java.tuple.builder;
    +
    +import java.util.LinkedList;
    +import java.util.List;
    +
    +import org.apache.flink.api.java.tuple.Tuple0;
    +
    +public class Tuple0Builder {
    +
    +	private List<Tuple0> tuples = new LinkedList<Tuple0>();
    --- End diff --
    
    ArrayLists are almost always way superior in performance than LinkedLists.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2457] Integrate Tuple0

Posted by mjsax <gi...@git.apache.org>.
Github user mjsax commented on the pull request:

    https://github.com/apache/flink/pull/983#issuecomment-127605738
  
    I used different dop for source/consumer and double check that multiple TMs are involved to make sure data is really going over the network...
    
    Batch example:
    ```
    public class TestEmptyTupleBatch {
    
    	public static void main(String[] args) throws Exception {
    		final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
    
    		DataSource<Tuple0> input = env.fromElements(
    				new Tuple0(), new Tuple0(), new Tuple0(), new Tuple0(), new Tuple0(),
    				new Tuple0(), new Tuple0(), new Tuple0(), new Tuple0(), new Tuple0(),
    				new Tuple0(), new Tuple0(), new Tuple0(), new Tuple0(), new Tuple0(),
    				new Tuple0(), new Tuple0(), new Tuple0(), new Tuple0(), new Tuple0());
    
    		input.map(new Counter()).setParallelism(4).print();
    	}
    
    	public static class Counter implements MapFunction<Tuple0, Integer> {
    		private static final long serialVersionUID = 5518823010274195186L;
    
    		int counter = 0;
    		@Override
    		public Integer map(Tuple0 value) throws Exception {
    			System.out.println("mjsax: " + (++counter) + " " + value.getField(0));
    			return new Integer(value.getArity());
    		}
    	}
    }
    ```
    
    Streaming Example:
    ```
    public class TestEmptyTuple {
    
    	public static void main(String[] args) throws Exception {
    		final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
    
    		DataStream<Tuple0> input = env.addSource(new Empty()).setParallelism(4);
    		input.shuffle().transform("verify", null, new Counter()).setParallelism(1);
    
    		env.execute();
    	}
    
    	public static class Empty implements ParallelSourceFunction<Tuple0> {
    		private static final long serialVersionUID = 1350902748274781033L;
    		private volatile boolean isRunning = true;
    
    		@Override
    		public void run(org.apache.flink.streaming.api.functions.source.SourceFunction.SourceContext<Tuple0> ctx)
    						throws Exception {
    			while (this.isRunning) {
    				Thread.sleep(1000);
    				ctx.collect(new Tuple0());
    			}
    		}
    
    		@Override
    		public void cancel() {
    			this.isRunning = false;
    		}
    	}
    
    	public static class Counter implements OneInputStreamOperator<Tuple0,Object> {
    		private static final long serialVersionUID = 5518823010274195186L;
    
    		private int counter = 0;
    		@Override
    		public void processElement(StreamRecord<Tuple0> element) throws Exception {
    			System.out.println("Tuple0: " + (++counter));
    		}
    
    		// 
    	}
    
    }
    ```
    



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2457] Integrate Tuple0

Posted by StephanEwen <gi...@git.apache.org>.
Github user StephanEwen commented on the pull request:

    https://github.com/apache/flink/pull/983#issuecomment-128406575
  
    Was it on purpose to add `readResolve()` to the serializer, rather than the tuple?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2457] Integrate Tuple0

Posted by StephanEwen <gi...@git.apache.org>.
Github user StephanEwen commented on the pull request:

    https://github.com/apache/flink/pull/983#issuecomment-131150367
  
    Looks good, will merge this.
    
    Will make slight adjustments in the merge (for code style consistency), like naming the single instance uppercase `INSTANCE`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2457] Integrate Tuple0

Posted by twalthr <gi...@git.apache.org>.
Github user twalthr commented on the pull request:

    https://github.com/apache/flink/pull/983#issuecomment-130251096
  
    The typeutil classes look good. I see that you have modified the TupleXXBuilders, have you modified them by hand or by running the `TupleGenerator`? I can't see the modified `TupleGenerator` in your PR.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2457] Integrate Tuple0

Posted by StephanEwen <gi...@git.apache.org>.
Github user StephanEwen commented on a diff in the pull request:

    https://github.com/apache/flink/pull/983#discussion_r36411464
  
    --- Diff: flink-staging/flink-language-binding/flink-language-binding-generic/src/main/java/org/apache/flink/languagebinding/api/java/common/streaming/Receiver.java ---
    @@ -346,61 +349,12 @@ public Tuple deserialize() {
     	}
     
     	public static Tuple createTuple(int size) {
    -		switch (size) {
    -			case 0:
    -				return new Tuple0();
    -			case 1:
    -				return new Tuple1();
    -			case 2:
    -				return new Tuple2();
    -			case 3:
    -				return new Tuple3();
    -			case 4:
    -				return new Tuple4();
    -			case 5:
    -				return new Tuple5();
    -			case 6:
    -				return new Tuple6();
    -			case 7:
    -				return new Tuple7();
    -			case 8:
    -				return new Tuple8();
    -			case 9:
    -				return new Tuple9();
    -			case 10:
    -				return new Tuple10();
    -			case 11:
    -				return new Tuple11();
    -			case 12:
    -				return new Tuple12();
    -			case 13:
    -				return new Tuple13();
    -			case 14:
    -				return new Tuple14();
    -			case 15:
    -				return new Tuple15();
    -			case 16:
    -				return new Tuple16();
    -			case 17:
    -				return new Tuple17();
    -			case 18:
    -				return new Tuple18();
    -			case 19:
    -				return new Tuple19();
    -			case 20:
    -				return new Tuple20();
    -			case 21:
    -				return new Tuple21();
    -			case 22:
    -				return new Tuple22();
    -			case 23:
    -				return new Tuple23();
    -			case 24:
    -				return new Tuple24();
    -			case 25:
    -				return new Tuple25();
    -			default:
    -				throw new IllegalArgumentException("Tuple size not supported: " + size);
    +		try {
    +			return Tuple.getTupleClass(size).newInstance();
    --- End diff --
    
    Yep, you're right...


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2457] Integrate Tuple0

Posted by mjsax <gi...@git.apache.org>.
Github user mjsax commented on the pull request:

    https://github.com/apache/flink/pull/983#issuecomment-127680821
  
    I just tried using `TypeSerializerOutputFormat` for a DataSet sink. There is no error. The system only creates an empty file. How can I force the usage of a file-channel (instead of network or in-memory chaining) between two mappers?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2457] Integrate Tuple0

Posted by StephanEwen <gi...@git.apache.org>.
Github user StephanEwen commented on a diff in the pull request:

    https://github.com/apache/flink/pull/983#discussion_r36408368
  
    --- Diff: flink-java/src/main/java/org/apache/flink/api/java/tuple/builder/Tuple0Builder.java ---
    @@ -0,0 +1,46 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * 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 KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +
    +// --------------------------------------------------------------
    +//  THIS IS A GENERATED SOURCE FILE. DO NOT EDIT!
    +//  GENERATED FROM org.apache.flink.api.java.tuple.TupleGenerator.
    +// --------------------------------------------------------------
    +
    +
    +package org.apache.flink.api.java.tuple.builder;
    +
    +import java.util.LinkedList;
    +import java.util.List;
    +
    +import org.apache.flink.api.java.tuple.Tuple0;
    +
    +public class Tuple0Builder {
    +
    +	private List<Tuple0> tuples = new LinkedList<Tuple0>();
    --- End diff --
    
    And memory efficiency...


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2457] Integrate Tuple0

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/983#discussion_r36362631
  
    --- Diff: flink-java/src/test/java/org/apache/flink/api/java/type/extractor/TypeExtractorTest.java ---
    @@ -200,12 +201,12 @@ public void testTupleWithBasicTypes() throws Exception {
     	@Test
     	public void testTupleWithTuples() {
     		// use getFlatMapReturnTypes()
    -		RichFlatMapFunction<?, ?> function = new RichFlatMapFunction<Tuple3<Tuple1<String>, Tuple1<Integer>, Tuple2<Long, Long>>, Tuple3<Tuple1<String>, Tuple1<Integer>, Tuple2<Long, Long>>>() {
    --- End diff --
    
    Can you add new tests instead of changing existing ones?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2457] Integrate Tuple0

Posted by mjsax <gi...@git.apache.org>.
Github user mjsax commented on the pull request:

    https://github.com/apache/flink/pull/983#issuecomment-127951144
  
    There is nothing bad about it. I just would like to have a breaking example...


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2457] Integrate Tuple0

Posted by mjsax <gi...@git.apache.org>.
Github user mjsax commented on a diff in the pull request:

    https://github.com/apache/flink/pull/983#discussion_r36195314
  
    --- Diff: flink-java/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java ---
    @@ -391,6 +392,10 @@ protected TypeExtractor() {
     				curT = typeToClass(curT).getGenericSuperclass();
     			}
     			
    +			if(typeToClass(curT).equals(Tuple0.class)) {
    --- End diff --
    
    I see. Makes sense. I will update it.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2457] Integrate Tuple0

Posted by fhueske <gi...@git.apache.org>.
Github user fhueske commented on the pull request:

    https://github.com/apache/flink/pull/983#issuecomment-128171745
  
    Thanks for updating the PR!
    Apart from two comments, it looks good to me.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2457] Integrate Tuple0

Posted by mjsax <gi...@git.apache.org>.
Github user mjsax commented on the pull request:

    https://github.com/apache/flink/pull/983#issuecomment-130211866
  
    Any news about this PR?  @twalthr : Are you going to review it again?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2457] Integrate Tuple0

Posted by mjsax <gi...@git.apache.org>.
Github user mjsax commented on the pull request:

    https://github.com/apache/flink/pull/983#issuecomment-130321100
  
    I did those manually, too. I am just reworking the code to fix this up.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2457] Integrate Tuple0

Posted by StephanEwen <gi...@git.apache.org>.
Github user StephanEwen commented on a diff in the pull request:

    https://github.com/apache/flink/pull/983#discussion_r36408610
  
    --- Diff: flink-staging/flink-language-binding/flink-language-binding-generic/src/main/java/org/apache/flink/languagebinding/api/java/common/streaming/Receiver.java ---
    @@ -346,61 +349,12 @@ public Tuple deserialize() {
     	}
     
     	public static Tuple createTuple(int size) {
    -		switch (size) {
    -			case 0:
    -				return new Tuple0();
    -			case 1:
    -				return new Tuple1();
    -			case 2:
    -				return new Tuple2();
    -			case 3:
    -				return new Tuple3();
    -			case 4:
    -				return new Tuple4();
    -			case 5:
    -				return new Tuple5();
    -			case 6:
    -				return new Tuple6();
    -			case 7:
    -				return new Tuple7();
    -			case 8:
    -				return new Tuple8();
    -			case 9:
    -				return new Tuple9();
    -			case 10:
    -				return new Tuple10();
    -			case 11:
    -				return new Tuple11();
    -			case 12:
    -				return new Tuple12();
    -			case 13:
    -				return new Tuple13();
    -			case 14:
    -				return new Tuple14();
    -			case 15:
    -				return new Tuple15();
    -			case 16:
    -				return new Tuple16();
    -			case 17:
    -				return new Tuple17();
    -			case 18:
    -				return new Tuple18();
    -			case 19:
    -				return new Tuple19();
    -			case 20:
    -				return new Tuple20();
    -			case 21:
    -				return new Tuple21();
    -			case 22:
    -				return new Tuple22();
    -			case 23:
    -				return new Tuple23();
    -			case 24:
    -				return new Tuple24();
    -			case 25:
    -				return new Tuple25();
    -			default:
    -				throw new IllegalArgumentException("Tuple size not supported: " + size);
    +		try {
    +			return Tuple.getTupleClass(size).newInstance();
    --- End diff --
    
    This should have a size check and give a proper error message.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2457] Integrate Tuple0

Posted by mjsax <gi...@git.apache.org>.
Github user mjsax commented on the pull request:

    https://github.com/apache/flink/pull/983#issuecomment-127992002
  
    Added serialization and extended tests. I tested the serialization part by writing Tuple0 as result of one batch program to disc and consumed those Tuple0 recored in a second program successfully. Please give feedback.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2457] Integrate Tuple0

Posted by mjsax <gi...@git.apache.org>.
Github user mjsax commented on the pull request:

    https://github.com/apache/flink/pull/983#issuecomment-130298853
  
    I modified by hand. Was not aware of `TupleGenerator`. Just had a look into it. Not sure if `Tuple0` can be included appropriately. For example, it is no generic class; it is implemented as soft singleton. Extending `TupleGenerator` would result in special handling of Tuple0 in every place. Thus, adding it manually and not generate the source code for it seems better to me.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2457] Integrate Tuple0

Posted by twalthr <gi...@git.apache.org>.
Github user twalthr commented on a diff in the pull request:

    https://github.com/apache/flink/pull/983#discussion_r36193152
  
    --- Diff: flink-java/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java ---
    @@ -391,6 +392,10 @@ protected TypeExtractor() {
     				curT = typeToClass(curT).getGenericSuperclass();
     			}
     			
    +			if(typeToClass(curT).equals(Tuple0.class)) {
    --- End diff --
    
    No, calling `typeToClass()` with a TypeVar will throw an `IllegalArgumentException`. `typeToClass()` should only be used if  `isClassType()` is true.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2457] Integrate Tuple0

Posted by StephanEwen <gi...@git.apache.org>.
Github user StephanEwen commented on the pull request:

    https://github.com/apache/flink/pull/983#issuecomment-130316822
  
    Agree, Tuple0 should probably not go through the tuple generator.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2457] Integrate Tuple0

Posted by mjsax <gi...@git.apache.org>.
Github user mjsax commented on the pull request:

    https://github.com/apache/flink/pull/983#issuecomment-127905311
  
    Thanks for conforming my observations and even dive deeper into it. What should the next step be?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-2457] Integrate Tuple0

Posted by StephanEwen <gi...@git.apache.org>.
Github user StephanEwen commented on the pull request:

    https://github.com/apache/flink/pull/983#issuecomment-128350696
  
    Some comments inline, other than that, two issues with this pull request:
      - A lot of whitespace reformatting. We explicitly ask not to do this. Some IDEs do it automatically, but you can deactivate it. It makes diffs dangerously convoluted.
      - Tuple0 can be treated as a singleton, since it has no state. Any reason not to do this?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---