Uploaded image for project: 'Flink'
  1. Flink
  2. FLINK-18503

bug occurs when `HeapReducingState#add` method handles null

    XMLWordPrintableJSON

Details

    Description

      In our production environment, there are advertising billing jobs, which are keyBy according to advertiserId. Calculate the cost of each advertiser in the specified window, that is, use ReducingFunction to sum the price after keyBy by advertiser.

      But it is found that the results calculated using FsStateBackend and RocksDBStateBackend are different. The calculation result of FsStateBackend is wrong, and the calculation result of RocksDBStateBackend is correct.

      After reading the source code, HeapReducingState#add code:code link

       

      public void add(V value) throws IOException {
      
         if (value == null) {
            clear();
            return;
         }
      
         try {
            stateTable.transform(currentNamespace, value, reduceTransformation);
         } catch (Exception e) {
            throw new IOException("Exception while applying ReduceFunction in reducing state", e);
         }
      }
      

      If value==null, the clear method deletes the data of the current <key,namespace> from the StateTable. ReducingFunction will only be executed if value!=null.

      Why is there a bug?

      For a job that calculates cost, if price != null, the price is added to result; if price == null, then result is unchanged.

      The ResultFunction method handles the case of price == null, our ResultFunction is as follows:

       

      ReduceFunction<Long> sumFunction = new ReduceFunction<Long>() {
         @Override
         public Long reduce(Long previousState, Long newValue) throws Exception {
            // if newValue ==null,
            // consider newValue to be 0 and return previousState directly
            if (newValue == null) {
               return previousState;
            }
            return previousState + newValue;
         }
      };
      

       

      However, when HeapReducingState#add finds that the input value == null, it directly executes the clear method, and does not execute the user-defined ResultFunction at all.

      For example: if the input prices are 17, null, and 11, the price saved in the state is 17 when you enter 17, the price is cleared when you enter null, and the price is 11 when you enter 11, so the result is wrong.

      Fortunately, the calculation result of RocksDBStateBackend is correct. The RocksDBReducingState#add method does not perform special treatment for null. RocksDBReducingState#add code is as follows:code link

       

      public void add(V value) throws Exception {
         byte[] key = getKeyBytes();
         V oldValue = getInternal(key);
         V newValue = oldValue == null ? value : reduceFunction.reduce(oldValue, value);
         updateInternal(key, newValue);
      }
      

      Flink UT can reproduce this bug

      StateBackendTestBase#testReducingStateAddAndGet can reproduce this bug.

      Need to be modified as follows:

      1. udf
        ReduceFunction<Long> sumFunction = new ReduceFunction<Long>() {
           @Override
           public Long reduce(Long previousState, Long newValue) throws Exception {
              // if newValue ==null,
              // consider newValue to be 0 and return previousState directly
              if (newValue == null) {
                 return previousState;
              }
              return previousState + newValue;
           }
        };
        
        final ReducingStateDescriptor<Long> stateDescr =
           new ReducingStateDescriptor<>("my-state", sumFunction, Long.class);
      1. add element
      keyedBackend.setCurrentKey("def");
      assertNull(state.get());
      state.add(17L);
      state.add(null);//new code
      state.add(11L);
      assertEquals(28L, state.get().longValue());

      My code repository commit link

      The UT execution output of RocksDBStateBackendTest is as follows:

       

      The UT execution output of FileStateBackendTest&MemoryStateBackendTest is as follows:

      java.lang.AssertionError: 
      Expected :28
      Actual   :11

      The above phenomenon shows that the HeapReducingState#add method has a bug. Regardless of which state backend you choose, the semantics provided by the Flink engine should be consistent and should not output different calculation results.

      My solution

      Remove the processing logic of value == null in HeapReducingState#add. Result: All UTs of FileStateBackendTest can be passed.

      Similar bug

      HeapFoldingState#add & HeapAggregatingState#add

      Question

      HeapReducingState#add When designing, why does the designer handle the null case specially? I think the case of null should be handled by the user-defined ReducingFunction.

       

       

      Attachments

        1. image-2020-07-07-02-20-03-420.png
          151 kB
          Rui Fan
        2. image-2020-07-07-02-20-57-299.png
          231 kB
          Rui Fan

        Activity

          People

            Unassigned Unassigned
            fanrui Rui Fan
            Votes:
            0 Vote for this issue
            Watchers:
            5 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: