[FLINK-35354] Support host mapping in Flink tikv cdc (#3336)
* [FLINK-35337][cdc] Keep up with the latest version of tikv client * [FLINK-35354]Support host mapping in Flink tikv cdc * [FLINK-35354] Add doc for host mapping feature * [FLINK-35354] fixed annotation importpull/3469/head
parent
ca1470d5dd
commit
302a691225
@ -0,0 +1,83 @@
|
||||
/*
|
||||
* 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.cdc.connectors.tidb.table.utils;
|
||||
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
import org.tikv.common.HostMapping;
|
||||
|
||||
import java.net.URI;
|
||||
import java.net.URISyntaxException;
|
||||
import java.util.Arrays;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.ConcurrentMap;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/** Get TiKV Host Mapping Function. {@link HostMapping}.* */
|
||||
public class UriHostMapping implements HostMapping {
|
||||
private static final Logger LOG = LoggerFactory.getLogger(UriHostMapping.class);
|
||||
private final ConcurrentMap<String, String> hostMapping;
|
||||
|
||||
public UriHostMapping(String hostMappingString) {
|
||||
if (hostMappingString == null || hostMappingString.isEmpty()) {
|
||||
hostMapping = null;
|
||||
return;
|
||||
}
|
||||
try {
|
||||
this.hostMapping =
|
||||
Arrays.stream(hostMappingString.split(";"))
|
||||
.map(
|
||||
s -> {
|
||||
String[] hostAndPort = s.split(":");
|
||||
return new ConcurrentHashMap.SimpleEntry<>(
|
||||
hostAndPort[0], hostAndPort[1]);
|
||||
})
|
||||
.collect(
|
||||
Collectors.toConcurrentMap(
|
||||
ConcurrentHashMap.SimpleEntry::getKey,
|
||||
ConcurrentHashMap.SimpleEntry::getValue));
|
||||
} catch (Exception e) {
|
||||
LOG.error("Invalid host mapping string: {}", hostMappingString, e);
|
||||
throw new IllegalArgumentException("Invalid host mapping string: " + hostMappingString);
|
||||
}
|
||||
}
|
||||
|
||||
public ConcurrentMap<String, String> getHostMapping() {
|
||||
return hostMapping;
|
||||
}
|
||||
|
||||
@Override
|
||||
public URI getMappedURI(URI uri) {
|
||||
if (hostMapping != null && hostMapping.containsKey(uri.getHost())) {
|
||||
try {
|
||||
return new URI(
|
||||
uri.getScheme(),
|
||||
uri.getUserInfo(),
|
||||
hostMapping.get(uri.getHost()),
|
||||
uri.getPort(),
|
||||
uri.getPath(),
|
||||
uri.getQuery(),
|
||||
uri.getFragment());
|
||||
} catch (URISyntaxException ex) {
|
||||
LOG.error("Failed to get mapped URI", ex);
|
||||
throw new IllegalArgumentException(ex);
|
||||
}
|
||||
}
|
||||
return uri;
|
||||
}
|
||||
}
|
@ -0,0 +1,60 @@
|
||||
/*
|
||||
* 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.cdc.connectors.tidb.table.utils;
|
||||
|
||||
import org.apache.flink.cdc.connectors.tidb.TDBSourceOptions;
|
||||
|
||||
import org.junit.Test;
|
||||
import org.tikv.common.TiConfiguration;
|
||||
|
||||
import java.util.HashMap;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
|
||||
/** Unit test for {@link UriHostMapping}. * */
|
||||
public class UriHostMappingTest {
|
||||
|
||||
@Test
|
||||
public void uriHostMappingTest() {
|
||||
final TiConfiguration tiConf =
|
||||
TDBSourceOptions.getTiConfiguration(
|
||||
"http://0.0.0.0:2347", "host1:1;host2:2;host3:3", new HashMap<>());
|
||||
UriHostMapping uriHostMapping = (UriHostMapping) tiConf.getHostMapping();
|
||||
assertEquals(uriHostMapping.getHostMapping().size(), 3);
|
||||
assertEquals(uriHostMapping.getHostMapping().get("host1"), "1");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void uriHostMappingEmpty() {
|
||||
final TiConfiguration tiConf =
|
||||
TDBSourceOptions.getTiConfiguration("http://0.0.0.0:2347", "", new HashMap<>());
|
||||
UriHostMapping uriHostMapping = (UriHostMapping) tiConf.getHostMapping();
|
||||
assertEquals(uriHostMapping.getHostMapping(), null);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void uriHostMappingError() {
|
||||
try {
|
||||
final TiConfiguration tiConf =
|
||||
TDBSourceOptions.getTiConfiguration(
|
||||
"http://0.0.0.0:2347", "host1=1;host2=2;host3=3", new HashMap<>());
|
||||
} catch (IllegalArgumentException e) {
|
||||
assertEquals(e.getMessage(), "Invalid host mapping string: host1=1;host2=2;host3=3");
|
||||
}
|
||||
}
|
||||
}
|
Loading…
Reference in New Issue