summaryrefslogtreecommitdiffstats
path: root/vespa-hadoop/src/main/java/com/yahoo/vespa/hadoop/mapreduce/VespaRecordWriter.java
blob: 6540aec342d7747d8560224a44d096ab0c72c1ad (plain) (blame)
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
// Copyright Verizon Media. Licensed under the terms of the Apache 2.0 license. See LICENSE in the project root.
package com.yahoo.vespa.hadoop.mapreduce;

import ai.vespa.feed.client.DocumentId;
import ai.vespa.feed.client.DryrunResult;
import ai.vespa.feed.client.FeedClient;
import ai.vespa.feed.client.FeedClientBuilder;
import ai.vespa.feed.client.JsonFeeder;
import ai.vespa.feed.client.OperationParseException;
import ai.vespa.feed.client.OperationParameters;
import ai.vespa.feed.client.OperationStats;
import ai.vespa.feed.client.Result;
import com.yahoo.vespa.hadoop.mapreduce.util.VespaConfiguration;
import com.yahoo.vespa.hadoop.mapreduce.util.VespaCounters;
import com.yahoo.vespa.http.client.config.FeedParams;
import org.apache.hadoop.mapreduce.RecordWriter;
import org.apache.hadoop.mapreduce.TaskAttemptContext;

import java.io.IOException;
import java.net.URI;
import java.time.Duration;
import java.util.Arrays;
import java.util.List;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ThreadLocalRandom;
import java.util.logging.Level;
import java.util.logging.Logger;

import static java.util.stream.Collectors.toList;

/**
 * {@link VespaRecordWriter} sends the output <key, value> to one or more Vespa endpoints using vespa-feed-client.
 *
 * @author bjorncs
 */
public class VespaRecordWriter extends RecordWriter<Object, Object> {

    private final static Logger log = Logger.getLogger(VespaRecordWriter.class.getCanonicalName());

    private final VespaCounters counters;
    private final VespaConfiguration config;

    private boolean initialized = false;
    private JsonFeeder feeder;

    protected VespaRecordWriter(VespaConfiguration config, VespaCounters counters) {
        this.counters = counters;
        this.config = config;
    }

    @Override
    public void write(Object key, Object data) throws IOException {
        initializeOnFirstWrite();
        String json = data.toString().trim();
        feeder.feedSingle(json)
                .whenComplete((result, error) -> {
                    if (error != null) {
                        if (error instanceof OperationParseException) {
                            counters.incrementDocumentsSkipped(1);
                        } else {
                            String msg = "Failed to feed single document: " + error;
                            System.out.println(msg);
                            System.err.println(msg);
                            log.log(Level.WARNING, msg, error);
                            counters.incrementDocumentsFailed(1);
                        }
                    } else {
                        counters.incrementDocumentsOk(1);
                    }
                });
        counters.incrementDocumentsSent(1);
        if (counters.getDocumentsSent() % config.progressInterval() == 0) {
            String progress = String.format("Feed progress: %d / %d / %d / %d (sent, ok, failed, skipped)",
                    counters.getDocumentsSent(),
                    counters.getDocumentsOk(),
                    counters.getDocumentsFailed(),
                    counters.getDocumentsSkipped());
            log.info(progress);
        }
    }

    @Override
    public void close(TaskAttemptContext context) throws IOException {
        if (feeder != null) {
            feeder.close();
            feeder = null;
            initialized = false;
        }
    }

    /** Override method to alter {@link FeedClient} configuration */
    protected void onFeedClientInitialization(FeedClientBuilder builder) {}

    private void initializeOnFirstWrite() {
        if (initialized) return;
        validateConfig();
        useRandomizedStartupDelayIfEnabled();
        feeder = createJsonStreamFeeder();
        initialized = true;
    }

    private void validateConfig() {
        if (!config.useSSL()) {
            throw new IllegalArgumentException("SSL is required for this feed client implementation");
        }
        if (config.dataFormat() != FeedParams.DataFormat.JSON_UTF8) {
            throw new IllegalArgumentException("Only JSON is support by this feed client implementation");
        }
        if (config.proxyHost() != null) {
            log.warning(String.format("Ignoring proxy config (host='%s', port=%d)", config.proxyHost(), config.proxyPort()));
        }
    }

    private void useRandomizedStartupDelayIfEnabled() {
        if (!config.dryrun() && config.randomStartupSleepMs() > 0) {
            int delay = ThreadLocalRandom.current().nextInt(config.randomStartupSleepMs());
            log.info("Delaying startup by " + delay + " ms");
            try {
                Thread.sleep(delay);
            } catch (Exception e) {}
        }
    }


    private JsonFeeder createJsonStreamFeeder() {
        FeedClient feedClient = createFeedClient();
            JsonFeeder.Builder builder = JsonFeeder.builder(feedClient)
                .withTimeout(Duration.ofMinutes(10));
        if (config.route() != null) {
            builder.withRoute(config.route());
        }
        return builder.build();

    }

    private FeedClient createFeedClient() {
        if (config.dryrun()) {
            return new DryrunClient();
        } else {
            List<URI> endpoints = endpointUris(config);
            log.info("Using endpoints " + endpoints);
            int streamsPerConnection = streamsPerConnection(config);
            log.log(Level.INFO, "Using {0} streams per connection", new Object[] {streamsPerConnection});
            log.log(Level.INFO, "Using {0} connections", new Object[] {config.numConnections()});
            FeedClientBuilder feedClientBuilder = FeedClientBuilder.create(endpoints)
                    .setConnectionsPerEndpoint(config.numConnections())
                    .setMaxStreamPerConnection(streamsPerConnection)
                    .setRetryStrategy(retryStrategy(config));

            onFeedClientInitialization(feedClientBuilder);
            return feedClientBuilder.build();
        }
    }

    private static FeedClient.RetryStrategy retryStrategy(VespaConfiguration config) {
        int maxRetries = config.numRetries();
        return new FeedClient.RetryStrategy() {
            @Override public int retries() { return maxRetries; }
        };
    }

    private static int streamsPerConnection(VespaConfiguration config) {
        return Math.min(256, config.maxInFlightRequests() / config.numConnections());
    }

    private static List<URI> endpointUris(VespaConfiguration config) {
        return Arrays.stream(config.endpoint().split(","))
                .map(hostname -> URI.create(String.format("https://%s:%d/", hostname, config.defaultPort())))
                .collect(toList());
    }

    private static class DryrunClient implements FeedClient {

        @Override
        public CompletableFuture<Result> put(DocumentId documentId, String documentJson, OperationParameters params) {
            return createSuccessResult(documentId);
        }

        @Override
        public CompletableFuture<Result> update(DocumentId documentId, String updateJson, OperationParameters params) {
            return createSuccessResult(documentId);
        }

        @Override
        public CompletableFuture<Result> remove(DocumentId documentId, OperationParameters params) {
            return createSuccessResult(documentId);
        }

        @Override public OperationStats stats() { return null; }
        @Override public void close(boolean graceful) {}

        private static CompletableFuture<Result> createSuccessResult(DocumentId documentId) {
            return CompletableFuture.completedFuture(DryrunResult.create(Result.Type.success, documentId, "ok", null));
        }
    }
}