Review Board 1.7.22


Fix SQOOP-1080

Review Request #11896 - Created June 15, 2013 and updated

Venkat Ranganathan
Reviewers
Sqoop
sqoop-sqoop2
The reason we have this failure is because the hosts running the jenkins CI runs are all set to Etc/UTC timezone which never has daylight saving time  (For example see https://builds.apache.org/computer/ubuntu6/systemInfo) .   To fix this, I have denormalized  the time to UTC time from local time ( as the time we read in is normalized) and print the time in UTC.   This will mean that we will have same partitions irrespective of whether a timezone observes DST or not. 
Ran the complete tests with various timezones to check this
connector/connector-generic-jdbc/src/main/java/org/apache/sqoop/connector/jdbc/GenericJdbcImportPartitioner.java
Revision eef18f2 New Change
[20] 19 lines
[+20]
20
import java.math.BigDecimal;
20
import java.math.BigDecimal;
21
import java.sql.Date;
21
import java.sql.Date;
22
import java.sql.Time;
22
import java.sql.Time;
23
import java.sql.Timestamp;
23
import java.sql.Timestamp;
24
import java.sql.Types;
24
import java.sql.Types;

    
   
25
import java.text.ParseException;

    
   
26
import java.text.SimpleDateFormat;
25
import java.util.LinkedList;
27
import java.util.LinkedList;
26
import java.util.List;
28
import java.util.List;

    
   
29
import java.util.TimeZone;
27

    
   
30

   
28
import org.apache.sqoop.common.SqoopException;
31
import org.apache.sqoop.common.SqoopException;
29
import org.apache.sqoop.connector.jdbc.configuration.ConnectionConfiguration;
32
import org.apache.sqoop.connector.jdbc.configuration.ConnectionConfiguration;
30
import org.apache.sqoop.connector.jdbc.configuration.ImportJobConfiguration;
33
import org.apache.sqoop.connector.jdbc.configuration.ImportJobConfiguration;
31
import org.apache.sqoop.job.etl.Partition;
34
import org.apache.sqoop.job.etl.Partition;
[+20] [20] 72 lines
[+20] [+] public class GenericJdbcImportPartitioner extends Partitioner<ConnectionConfiguration, ImportJobConfiguration> {
104
  protected List<Partition> partitionDateTimeColumn() {
107
  protected List<Partition> partitionDateTimeColumn() {
105
    List<Partition> partitions = new LinkedList<Partition>();
108
    List<Partition> partitions = new LinkedList<Partition>();
106

    
   
109

   
107
    long minDateValue = 0;
110
    long minDateValue = 0;
108
    long maxDateValue = 0;
111
    long maxDateValue = 0;
109

    
   
112
    SimpleDateFormat sdf = null;
110
    switch(partitionColumnType) {
113
    switch(partitionColumnType) {
111
      case Types.DATE:
114
      case Types.DATE:

    
   
115
        sdf = new SimpleDateFormat("yyyy-MM-dd");
112
        minDateValue = Date.valueOf(partitionMinValue).getTime();
116
        minDateValue = Date.valueOf(partitionMinValue).getTime();
113
        maxDateValue = Date.valueOf(partitionMaxValue).getTime();
117
        maxDateValue = Date.valueOf(partitionMaxValue).getTime();
114
        break;
118
        break;
115
      case Types.TIME:
119
      case Types.TIME:

    
   
120
        sdf = new SimpleDateFormat("HH:mm:ss");
116
        minDateValue = Time.valueOf(partitionMinValue).getTime();
121
        minDateValue = Time.valueOf(partitionMinValue).getTime();
117
        maxDateValue = Time.valueOf(partitionMaxValue).getTime();
122
        maxDateValue = Time.valueOf(partitionMaxValue).getTime();
118
        break;
123
        break;
119
      case Types.TIMESTAMP:
124
      case Types.TIMESTAMP:

    
   
125
        sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSS");
120
        minDateValue = Timestamp.valueOf(partitionMinValue).getTime();
126
        minDateValue = Timestamp.valueOf(partitionMinValue).getTime();
121
        maxDateValue = Timestamp.valueOf(partitionMaxValue).getTime();
127
        maxDateValue = Timestamp.valueOf(partitionMaxValue).getTime();
122
        break;
128
        break;
123
    }
129
    }

    
   
130

   

    
   
131
    long tzOffset = TimeZone.getDefault().getRawOffset();

    
   
132

   

    
   
133
    minDateValue += tzOffset;

    
   
134
    maxDateValue += tzOffset;

    
   
135

   

    
   
136
    sdf.setTimeZone(TimeZone.getTimeZone("GMT"));

    
   
137

   
124
    long interval =  (maxDateValue - minDateValue) / numberPartitions;
138
    long interval =  (maxDateValue - minDateValue) / numberPartitions;
125
    long remainder = (maxDateValue - minDateValue) % numberPartitions;
139
    long remainder = (maxDateValue - minDateValue) % numberPartitions;
126

    
   
140

   
127
    if (interval == 0) {
141
    if (interval == 0) {
128
      numberPartitions = (int)remainder;
142
      numberPartitions = (int)remainder;
129
    }
143
    }

    
   
144

   
130
    long lowerBound;
145
    long lowerBound;
131
    long upperBound = minDateValue;
146
    long upperBound = minDateValue;
132

    
   
147

   
133
    Object objLB = null;
148
    Object objLB = null;
134
    Object objUB = null;
149
    Object objUB = null;
[+20] [20] 9 lines
[+20] public class GenericJdbcImportPartitioner extends Partitioner<ConnectionConfiguration, ImportJobConfiguration> {
144
          objUB = new Date(upperBound);
159
          objUB = new Date(upperBound);
145
          break;
160
          break;
146
        case Types.TIME:
161
        case Types.TIME:
147
          objLB = new Time(lowerBound);
162
          objLB = new Time(lowerBound);
148
          objUB = new Time(upperBound);
163
          objUB = new Time(upperBound);

    
   
164

   
149
          break;
165
          break;
150
        case Types.TIMESTAMP:
166
        case Types.TIMESTAMP:
151
          objLB = new Timestamp(lowerBound);
167
          objLB = new Timestamp(lowerBound);
152
          objUB = new Timestamp(upperBound);
168
          objUB = new Timestamp(upperBound);
153
          break;
169
          break;
154
      }
170
      }
155

    
   
171

   
156
      GenericJdbcImportPartition partition = new GenericJdbcImportPartition();
172
      GenericJdbcImportPartition partition = new GenericJdbcImportPartition();
157
      partition.setConditions(
173
      partition.setConditions(
158
          constructDateConditions(objLB, objUB, false));
174
          constructDateConditions(sdf, objLB, objUB, false));
159
      partitions.add(partition);
175
      partitions.add(partition);
160
    }
176
    }

    
   
177

   
161
    switch(partitionColumnType) {
178
    switch(partitionColumnType) {
162
      case Types.DATE:
179
      case Types.DATE:
163
        objLB = new Date(upperBound);
180
        objLB = new Date(upperBound);
164
        objUB = new Date(maxDateValue);
181
        objUB = new Date(maxDateValue);
165
        break;
182
        break;
[+20] [20] 4 lines
[+20] public class GenericJdbcImportPartitioner extends Partitioner<ConnectionConfiguration, ImportJobConfiguration> {
170
      case Types.TIMESTAMP:
187
      case Types.TIMESTAMP:
171
        objLB = new Timestamp(upperBound);
188
        objLB = new Timestamp(upperBound);
172
        objUB = new Timestamp(maxDateValue);
189
        objUB = new Timestamp(maxDateValue);
173
        break;
190
        break;
174
    }
191
    }

    
   
192

   

    
   
193

   
175
    GenericJdbcImportPartition partition = new GenericJdbcImportPartition();
194
    GenericJdbcImportPartition partition = new GenericJdbcImportPartition();
176
    partition.setConditions(
195
    partition.setConditions(
177
        constructDateConditions(objLB, objUB, true));
196
        constructDateConditions(sdf, objLB, objUB, true));
178
    partitions.add(partition);
197
    partitions.add(partition);
179
    return partitions;
198
    return partitions;
180
  }
199
  }
181

    
   
200

   
182
  protected List<Partition> partitionTextColumn() {
201
  protected List<Partition> partitionTextColumn() {
[+20] [20] 276 lines
[+20] [+] protected String constructConditions(Object value) {
459
      .append(value)
478
      .append(value)
460
      .toString()
479
      .toString()
461
     ;
480
     ;
462
  }
481
  }
463

    
   
482

   
464
  protected String constructDateConditions(
483
  protected String constructDateConditions(SimpleDateFormat sdf,
465
      Object lowerBound, Object upperBound, boolean lastOne) {
484
      Object lowerBound, Object upperBound, boolean lastOne) {
466
    StringBuilder conditions = new StringBuilder();
485
    StringBuilder conditions = new StringBuilder();
467
    conditions.append('\'').append(lowerBound.toString()).append('\'');
486
    conditions.append('\'').append(sdf.format((java.util.Date)lowerBound)).append('\'');
468
    conditions.append(" <= ");
487
    conditions.append(" <= ");
469
    conditions.append(partitionColumnName);
488
    conditions.append(partitionColumnName);
470
    conditions.append(" AND ");
489
    conditions.append(" AND ");
471
    conditions.append(partitionColumnName);
490
    conditions.append(partitionColumnName);
472
    conditions.append(lastOne ? " <= " : " < ");
491
    conditions.append(lastOne ? " <= " : " < ");
473
    conditions.append('\'').append(upperBound.toString()).append('\'');
492
    conditions.append('\'').append(sdf.format((java.util.Date)upperBound)).append('\'');
474
    return conditions.toString();
493
    return conditions.toString();
475
  }
494
  }
476

    
   
495

   
477
  protected String constructTextConditions(String prefix,
496
  protected String constructTextConditions(String prefix,
478
      Object lowerBound, Object upperBound, boolean lastOne) {
497
      Object lowerBound, Object upperBound, boolean lastOne) {
[+20] [20] 73 lines
connector/connector-generic-jdbc/src/test/java/org/apache/sqoop/connector/jdbc/TestImportPartitioner.java
Revision 522a515 New Change
 
  1. connector/connector-generic-jdbc/src/main/java/org/apache/sqoop/connector/jdbc/GenericJdbcImportPartitioner.java: Loading...
  2. connector/connector-generic-jdbc/src/test/java/org/apache/sqoop/connector/jdbc/TestImportPartitioner.java: Loading...