aboutsummaryrefslogtreecommitdiffstats
path: root/src/main/java/parallelai/spyglass/jdbc/db/DBOutputFormat.java
blob: 116697018c1107ccdcca9a327fabb8dc400a9bf9 (plain)
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
/*
 * Copyright (c) 2009 Concurrent, Inc.
 *
 * This work has been released into the public domain
 * by the copyright holder. This applies worldwide.
 *
 * In case this is not legally possible:
 * The copyright holder grants any entity the right
 * to use this work for any purpose, without any
 * conditions, unless such conditions are required by law.
 */
/**
 * 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 parallelai.spyglass.jdbc.db;

import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.mapred.JobConf;
import org.apache.hadoop.mapred.OutputFormat;
import org.apache.hadoop.mapred.RecordWriter;
import org.apache.hadoop.mapred.Reporter;
import org.apache.hadoop.util.Progressable;
import org.apache.hadoop.util.StringUtils;

import com.jcraft.jsch.Logger;

import java.io.IOException;
import java.sql.Connection;
import java.sql.PreparedStatement;
import java.sql.SQLException;
import java.util.Collections;
import java.util.HashSet;
import java.util.Set;

/**
 * A OutputFormat that sends the reduce output to a SQL table. <p/> {@link DBOutputFormat} accepts
 * &lt;key,value&gt; pairs, where key has a type extending DBWritable. Returned {@link RecordWriter}
 * writes <b>only the key</b> to the database with a batch SQL query.
 */
public class DBOutputFormat<K extends DBWritable, V> implements OutputFormat<K, V> {
    private static final Log LOG = LogFactory.getLog(DBOutputFormat.class);

    /** A RecordWriter that writes the reduce output to a SQL table */
    protected class DBRecordWriter implements RecordWriter<K, V> {
        private Connection connection;
        private PreparedStatement insertStatement;
        private PreparedStatement updateStatement;
        private final int statementsBeforeExecute;

        private long statementsAdded = 0;
        private long insertStatementsCurrent = 0;
        private long updateStatementsCurrent = 0;

        protected DBRecordWriter(Connection connection, PreparedStatement insertStatement,
            PreparedStatement updateStatement, int statementsBeforeExecute) {
            this.connection = connection;
            this.insertStatement = insertStatement;
            this.updateStatement = updateStatement;
            this.statementsBeforeExecute = statementsBeforeExecute;
        }

        /** {@inheritDoc} */
        public void close(Reporter reporter) throws IOException {
            executeBatch();

            try {
                if (insertStatement != null) { insertStatement.close(); }

                if (updateStatement != null) { updateStatement.close(); }

                connection.commit();
            } catch (SQLException exception) {
                rollBack();

                createThrowMessage("unable to commit batch", 0, exception);
            } finally {
                try {
                    connection.close();
                } catch (SQLException exception) {
                    throw new IOException("unable to close connection", exception);
                }
            }
        }

        private void executeBatch() throws IOException {
            try {
                if (insertStatementsCurrent != 0) {
                    LOG.info(
                        "executing insert batch " + createBatchMessage(insertStatementsCurrent));

                    insertStatement.executeBatch();
                }

                insertStatementsCurrent = 0;
            } catch (SQLException exception) {
                rollBack();

                createThrowMessage("unable to execute insert batch", insertStatementsCurrent, exception);
            }

            try {
                if (updateStatementsCurrent != 0) {
                    LOG.info(
                        "executing update batch " + createBatchMessage(updateStatementsCurrent));

                    int[] result = updateStatement.executeBatch();

                    int count = 0;

                    for (int value : result) { count += value; }

                    if (count != updateStatementsCurrent) {
                        throw new IOException(
                            "update did not update same number of statements executed in batch, batch: "
                            + updateStatementsCurrent + " updated: " + count);
                    }
                }

                updateStatementsCurrent = 0;
            } catch (SQLException exception) {
            	
            	String message = exception.getMessage();
            	if (message.indexOf("Duplicate Key") >= 0) {
            		LOG.warn("In exception block. Bypass exception becuase of Insert/Update.");
            	} else {
                    rollBack();

                    createThrowMessage("unable to execute update batch", updateStatementsCurrent, exception);
            	}
            }
        }

        private void rollBack() {
            try {
                connection.rollback();
            } catch (SQLException sqlException) {
                LOG.warn(StringUtils.stringifyException(sqlException));
            }
        }

        private String createBatchMessage(long currentStatements) {
            return String
                .format("[totstmts: %d][crntstmts: %d][batch: %d]", statementsAdded, currentStatements, statementsBeforeExecute);
        }

        private void createThrowMessage(String stateMessage, long currentStatements,
            SQLException exception) throws IOException {
            String message = exception.getMessage();

            message = message.substring(0, Math.min(75, message.length()));

            int messageLength = exception.getMessage().length();
            String batchMessage = createBatchMessage(currentStatements);
            String template = "%s [msglength: %d]%s %s";
            String errorMessage =
                String.format(template, stateMessage, messageLength, batchMessage, message);

            LOG.error(errorMessage, exception.getNextException());

            throw new IOException(errorMessage, exception.getNextException());
        }

        /** {@inheritDoc} */
        public synchronized void write(K key, V value) throws IOException {
            try {
                if (value == null) {
                    key.write(insertStatement);
                    insertStatement.addBatch();
                    insertStatementsCurrent++;
                } else {
                    key.write(updateStatement);
                    updateStatement.addBatch();
                    updateStatementsCurrent++;
                }
            } catch (SQLException exception) {
                throw new IOException("unable to add batch statement", exception);
            }

            statementsAdded++;

            if (statementsAdded % statementsBeforeExecute == 0) { executeBatch(); }
        }
    }

    /**
     * Constructs the query used as the prepared statement to insert data.
     *
     * @param table      the table to insert into
     * @param fieldNames the fields to insert into. If field names are unknown, supply an array of
     *                   nulls.
     */
    protected String constructInsertQuery(String table, String[] fieldNames) {
        if (fieldNames == null) {
            throw new IllegalArgumentException("Field names may not be null");
        }

        StringBuilder query = new StringBuilder();

        query.append("INSERT INTO ").append(table);

        if (fieldNames.length > 0 && fieldNames[0] != null) {
            query.append(" (");

            for (int i = 0; i < fieldNames.length; i++) {
                query.append(fieldNames[i]);

                if (i != fieldNames.length - 1) { query.append(","); }
            }

            query.append(")");

        }

        query.append(" VALUES (");

        for (int i = 0; i < fieldNames.length; i++) {
            query.append("?");

            if (i != fieldNames.length - 1) { query.append(","); }
        }

        query.append(")");
        
        boolean test = true;
        if (test) {
        	query.append(" ON DUPLICATE KEY UPDATE ");
        	
        	
            for (int i = 1; i < fieldNames.length; i++) {

                
                if ( (i != 1) ) { query.append(","); }
                //if (i != fieldNames.length - 1) { query.append(","); }
                //&& (i != fieldNames.length - 1)
                query.append(fieldNames[i]);
                query.append(" = ?");
                
                
            }
        }
        
        query.append(";");
        
        LOG.info(" ===================== " + query.toString());
        return query.toString();
    }

    protected String constructUpdateQuery(String table, String[] fieldNames, String[] updateNames) {
        if (fieldNames == null) {
            throw new IllegalArgumentException("field names may not be null");
        }

        Set<String> updateNamesSet = new HashSet<String>();
        Collections.addAll(updateNamesSet, updateNames);

        StringBuilder query = new StringBuilder();

        query.append("UPDATE ").append(table);

        query.append(" SET ");

        if (fieldNames.length > 0 && fieldNames[0] != null) {
            int count = 0;

            for (int i = 0; i < fieldNames.length; i++) {
                if (updateNamesSet.contains(fieldNames[i])) { continue; }

                if (count != 0) { query.append(","); }

                query.append(fieldNames[i]);
                query.append(" = ?");

                count++;
            }
        }

        query.append(" WHERE ");

        if (updateNames.length > 0 && updateNames[0] != null) {
            for (int i = 0; i < updateNames.length; i++) {
                query.append(updateNames[i]);
                query.append(" = ?");

                if (i != updateNames.length - 1) { query.append(" and "); }
            }
        }

        query.append(";");
        System.out.println("Update Query => " + query.toString());
        return query.toString();
    }

    /** {@inheritDoc} */
    public void checkOutputSpecs(FileSystem filesystem, JobConf job) throws IOException {
    }

    /** {@inheritDoc} */
    public RecordWriter<K, V> getRecordWriter(FileSystem filesystem, JobConf job, String name,
        Progressable progress) throws IOException {
        DBConfiguration dbConf = new DBConfiguration(job);

        String tableName = dbConf.getOutputTableName();
        String[] fieldNames = dbConf.getOutputFieldNames();
        String[] updateNames = dbConf.getOutputUpdateFieldNames();
        int batchStatements = dbConf.getBatchStatementsNum();

        Connection connection = dbConf.getConnection();

        configureConnection(connection);

        String sqlInsert = constructInsertQuery(tableName, fieldNames);
        PreparedStatement insertPreparedStatement;

        try {
            insertPreparedStatement = connection.prepareStatement(sqlInsert);
            insertPreparedStatement.setEscapeProcessing(true); // should be on by default
        } catch (SQLException exception) {
            throw new IOException("unable to create statement for: " + sqlInsert, exception);
        }

        String sqlUpdate =
            updateNames != null ? constructUpdateQuery(tableName, fieldNames, updateNames) : null;
        PreparedStatement updatePreparedStatement = null;

        try {
            updatePreparedStatement =
                sqlUpdate != null ? connection.prepareStatement(sqlUpdate) : null;
        } catch (SQLException exception) {
            throw new IOException("unable to create statement for: " + sqlUpdate, exception);
        }

        return new DBRecordWriter(connection, insertPreparedStatement, updatePreparedStatement, batchStatements);
    }

    protected void configureConnection(Connection connection) {
        setAutoCommit(connection);
    }

    protected void setAutoCommit(Connection connection) {
        try {
            connection.setAutoCommit(false);
        } catch (Exception exception) {
            throw new RuntimeException("unable to set auto commit", exception);
        }
    }

    /**
     * Initializes the reduce-part of the job with the appropriate output settings
     *
     * @param job                 The job
     * @param dbOutputFormatClass
     * @param tableName           The table to insert data into
     * @param fieldNames          The field names in the table. If unknown, supply the appropriate
     */
    public static void setOutput(JobConf job, Class<? extends DBOutputFormat> dbOutputFormatClass,
        String tableName, String[] fieldNames, String[] updateFields, int batchSize) {
        if (dbOutputFormatClass == null) { job.setOutputFormat(DBOutputFormat.class); } else {
            job.setOutputFormat(dbOutputFormatClass);
        }

        // writing doesn't always happen in reduce
        job.setReduceSpeculativeExecution(false);
        job.setMapSpeculativeExecution(false);

        DBConfiguration dbConf = new DBConfiguration(job);

        dbConf.setOutputTableName(tableName);
        dbConf.setOutputFieldNames(fieldNames);

        if (updateFields != null) { dbConf.setOutputUpdateFieldNames(updateFields); }

        if (batchSize != -1) { dbConf.setBatchStatementsNum(batchSize); }
    }
}